You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/04/29 19:18:44 UTC
svn commit: r1591025 [2/2] - in /lucene/dev/trunk: ./ lucene/ lucene/codecs/
lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/
lucene/core/src/java/org/apache/lucene/analysis/
lucene/core/src/java/org/apache/lucene/codecs/ lucene...
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java Tue Apr 29 17:18:41 2014
@@ -26,18 +26,15 @@ import org.apache.lucene.util.BytesRefHa
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IntBlockPool;
import org.apache.lucene.util.BytesRefHash.BytesStartArray;
-import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException;
-final class TermsHashPerField extends InvertedDocConsumerPerField {
+abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
private static final int HASH_INIT_SIZE = 4;
- final TermsHashConsumerPerField consumer;
-
final TermsHash termsHash;
final TermsHashPerField nextPerField;
- final DocumentsWriterPerThread.DocState docState;
- final FieldInvertState fieldState;
+ protected final DocumentsWriterPerThread.DocState docState;
+ protected final FieldInvertState fieldState;
TermToBytesRefAttribute termAtt;
BytesRef termBytesRef;
@@ -49,44 +46,37 @@ final class TermsHashPerField extends In
final int streamCount;
final int numPostingInt;
- final FieldInfo fieldInfo;
+ protected final FieldInfo fieldInfo;
final BytesRefHash bytesHash;
ParallelPostingsArray postingsArray;
private final Counter bytesUsed;
- public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHash termsHash, final TermsHash nextTermsHash, final FieldInfo fieldInfo) {
+ /** streamCount: how many streams this field stores per term.
+ * E.g. doc(+freq) is 1 stream, prox+offset is a second. */
+
+ public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
intPool = termsHash.intPool;
bytePool = termsHash.bytePool;
termBytePool = termsHash.termBytePool;
docState = termsHash.docState;
this.termsHash = termsHash;
bytesUsed = termsHash.bytesUsed;
- fieldState = docInverterPerField.fieldState;
- this.consumer = termsHash.consumer.addField(this, fieldInfo);
- PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
- bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
- streamCount = consumer.getStreamCount();
+ this.fieldState = fieldState;
+ this.streamCount = streamCount;
numPostingInt = 2*streamCount;
this.fieldInfo = fieldInfo;
- if (nextTermsHash != null)
- nextPerField = (TermsHashPerField) nextTermsHash.addField(docInverterPerField, fieldInfo);
- else
- nextPerField = null;
+ this.nextPerField = nextPerField;
+ PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
+ bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
}
void reset() {
bytesHash.clear(false);
- if (nextPerField != null)
+ if (nextPerField != null) {
nextPerField.reset();
- }
-
- @Override
- public void abort() {
- reset();
- if (nextPerField != null)
- nextPerField.abort();
+ }
}
public void initReader(ByteSliceReader reader, int termID, int stream) {
@@ -99,34 +89,17 @@ final class TermsHashPerField extends In
ints[upto+stream]);
}
- /** Collapse the hash table & sort in-place. */
+ int[] sortedTermIDs;
+
+ /** Collapse the hash table & sort in-place; also sets
+ * this.sortedTermIDs to the results */
public int[] sortPostings() {
- return bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
+ sortedTermIDs = bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
+ return sortedTermIDs;
}
- private boolean doCall;
private boolean doNextCall;
- @Override
- void start(IndexableField f) {
- termAtt = fieldState.attributeSource.getAttribute(TermToBytesRefAttribute.class);
- termBytesRef = termAtt.getBytesRef();
- consumer.start(f);
- if (nextPerField != null) {
- nextPerField.start(f);
- }
- }
-
- @Override
- boolean start(IndexableField[] fields, int count) throws IOException {
- doCall = consumer.start(fields, count);
- bytesHash.reinit();
- if (nextPerField != null) {
- doNextCall = nextPerField.start(fields, count);
- }
- return doCall || doNextCall;
- }
-
// Secondary entry point (for 2nd & subsequent TermsHash),
// because token text has already been "interned" into
// textStart, so we hash by textStart. term vectors use
@@ -137,8 +110,9 @@ final class TermsHashPerField extends In
// First time we are seeing this token since we last
// flushed the hash.
// Init stream slices
- if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE)
+ if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {
intPool.nextBuffer();
+ }
if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) {
bytePool.nextBuffer();
@@ -156,20 +130,20 @@ final class TermsHashPerField extends In
}
postingsArray.byteStarts[termID] = intUptos[intUptoStart];
- consumer.newTerm(termID);
+ newTerm(termID);
} else {
termID = (-termID)-1;
int intStart = postingsArray.intStarts[termID];
intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
- consumer.addTerm(termID);
+ addTerm(termID);
}
}
- // Primary entry point (for first TermsHash); postings use
- // this API.
- @Override
+ /** Called once per inverted token. This is the primary
+ * entry point (for first TermsHash); postings use this
+ * API. */
void add() throws IOException {
termAtt.fillBytesRef();
@@ -177,27 +151,10 @@ final class TermsHashPerField extends In
// We are first in the chain so we must "intern" the
// term text into textStart address
// Get the text & hash of this term.
- int termID;
- try {
- termID = bytesHash.add(termBytesRef);
- } catch (MaxBytesLengthExceededException e) {
- // Term is too large; record this here (can't throw an
- // exc because DocInverterPerField will then abort the
- // entire segment) and then throw an exc later in
- // DocInverterPerField.java. LengthFilter can always be
- // used to prune the term before indexing:
- if (docState.maxTermPrefix == null) {
- final int saved = termBytesRef.length;
- try {
- termBytesRef.length = Math.min(30, DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8);
- docState.maxTermPrefix = termBytesRef.toString();
- } finally {
- termBytesRef.length = saved;
- }
- }
- consumer.skippingLongTerm();
- return;
- }
+ int termID = bytesHash.add(termBytesRef);
+
+ //System.out.println("add term=" + termBytesRef.utf8ToString() + " doc=" + docState.docID + " termID=" + termID);
+
if (termID >= 0) {// New posting
bytesHash.byteStart(termID);
// Init stream slices
@@ -221,18 +178,19 @@ final class TermsHashPerField extends In
}
postingsArray.byteStarts[termID] = intUptos[intUptoStart];
- consumer.newTerm(termID);
+ newTerm(termID);
} else {
termID = (-termID)-1;
- final int intStart = postingsArray.intStarts[termID];
+ int intStart = postingsArray.intStarts[termID];
intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
- consumer.addTerm(termID);
+ addTerm(termID);
}
- if (doNextCall)
+ if (doNextCall) {
nextPerField.add(postingsArray.textStarts[termID]);
+ }
}
int[] intUptos;
@@ -269,13 +227,6 @@ final class TermsHashPerField extends In
writeByte(stream, (byte) i);
}
- @Override
- void finish() throws IOException {
- consumer.finish();
- if (nextPerField != null)
- nextPerField.finish();
- }
-
private static final class PostingsBytesStartArray extends BytesStartArray {
private final TermsHashPerField perField;
@@ -290,7 +241,8 @@ final class TermsHashPerField extends In
@Override
public int[] init() {
if (perField.postingsArray == null) {
- perField.postingsArray = perField.consumer.createPostingsArray(2);
+ perField.postingsArray = perField.createPostingsArray(2);
+ perField.newPostingsArray();
bytesUsed.addAndGet(perField.postingsArray.size * perField.postingsArray.bytesPerPosting());
}
return perField.postingsArray.textStarts;
@@ -301,15 +253,17 @@ final class TermsHashPerField extends In
ParallelPostingsArray postingsArray = perField.postingsArray;
final int oldSize = perField.postingsArray.size;
postingsArray = perField.postingsArray = postingsArray.grow();
+ perField.newPostingsArray();
bytesUsed.addAndGet((postingsArray.bytesPerPosting() * (postingsArray.size - oldSize)));
return postingsArray.textStarts;
}
@Override
public int[] clear() {
- if(perField.postingsArray != null) {
+ if (perField.postingsArray != null) {
bytesUsed.addAndGet(-(perField.postingsArray.size * perField.postingsArray.bytesPerPosting()));
perField.postingsArray = null;
+ perField.newPostingsArray();
}
return null;
}
@@ -318,7 +272,47 @@ final class TermsHashPerField extends In
public Counter bytesUsed() {
return bytesUsed;
}
+ }
+
+ @Override
+ public int compareTo(TermsHashPerField other) {
+ return fieldInfo.name.compareTo(other.fieldInfo.name);
+ }
+
+ /** Finish adding all instances of this field to the
+ * current document. */
+ void finish() throws IOException {
+ if (nextPerField != null) {
+ nextPerField.finish();
+ }
+ }
+ /** Start adding a new field instance; first is true if
+ * this is the first time this field name was seen in the
+ * document. */
+ boolean start(IndexableField field, boolean first) {
+ termAtt = fieldState.termAttribute;
+ // EmptyTokenStream can have null term att
+ if (termAtt != null) {
+ termBytesRef = termAtt.getBytesRef();
+ }
+ if (nextPerField != null) {
+ doNextCall = nextPerField.start(field, first);
+ }
+
+ return true;
}
+ /** Called when a term is seen for the first time. */
+ abstract void newTerm(int termID) throws IOException;
+
+ /** Called when a previously seen term is seen again. */
+ abstract void addTerm(int termID) throws IOException;
+
+ /** Called when the postings array is initialized or
+ * resized. */
+ abstract void newPostingsArray();
+
+ /** Creates a new postings array of the specified size. */
+ abstract ParallelPostingsArray createPostingsArray(int size);
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java Tue Apr 29 17:18:41 2014
@@ -171,7 +171,7 @@ public class RAMDirectory extends BaseDi
existing.directory = null;
}
fileMap.put(name, file);
- return new RAMOutputStream(file);
+ return new RAMOutputStream(file, true);
}
/**
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Tue Apr 29 17:18:41 2014
@@ -38,20 +38,25 @@ public class RAMOutputStream extends Ind
private long bufferStart;
private int bufferLength;
- private Checksum crc = new BufferedChecksum(new CRC32());
+ private final Checksum crc;
/** Construct an empty output buffer. */
public RAMOutputStream() {
- this(new RAMFile());
+ this(new RAMFile(), false);
}
- public RAMOutputStream(RAMFile f) {
+ public RAMOutputStream(RAMFile f, boolean checksum) {
file = f;
// make sure that we switch to the
// first needed buffer lazily
currentBufferIndex = -1;
currentBuffer = null;
+ if (checksum) {
+ crc = new BufferedChecksum(new CRC32());
+ } else {
+ crc = null;
+ }
}
/** Copy the current contents of this buffer to the named output. */
@@ -99,7 +104,9 @@ public class RAMOutputStream extends Ind
bufferStart = 0;
bufferLength = 0;
file.setLength(0);
- crc.reset();
+ if (crc != null) {
+ crc.reset();
+ }
}
@Override
@@ -113,14 +120,18 @@ public class RAMOutputStream extends Ind
currentBufferIndex++;
switchCurrentBuffer();
}
- crc.update(b);
+ if (crc != null) {
+ crc.update(b);
+ }
currentBuffer[bufferPosition++] = b;
}
@Override
public void writeBytes(byte[] b, int offset, int len) throws IOException {
assert b != null;
- crc.update(b, offset, len);
+ if (crc != null) {
+ crc.update(b, offset, len);
+ }
while (len > 0) {
if (bufferPosition == bufferLength) {
currentBufferIndex++;
@@ -171,6 +182,10 @@ public class RAMOutputStream extends Ind
@Override
public long getChecksum() throws IOException {
- return crc.getValue();
+ if (crc == null) {
+ throw new IllegalStateException("internal RAMOutputStream created with checksum disabled");
+ } else {
+ return crc.getValue();
+ }
}
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java Tue Apr 29 17:18:41 2014
@@ -287,22 +287,20 @@ public class AttributeSource {
}
/**
- * The caller must pass in a Class<? extends Attribute> value.
* Returns the instance of the passed in Attribute contained in this AttributeSource
+ * <p>
+ * The caller must pass in a Class<? extends Attribute> value.
*
- * @throws IllegalArgumentException if this AttributeSource does not contain the
- * Attribute. It is recommended to always use {@link #addAttribute} even in consumers
- * of TokenStreams, because you cannot know if a specific TokenStream really uses
- * a specific Attribute. {@link #addAttribute} will automatically make the attribute
- * available. If you want to only use the attribute, if it is available (to optimize
+ * @return instance of the passed in Attribute, or {@code null} if this AttributeSource
+ * does not contain the Attribute. It is recommended to always use
+ * {@link #addAttribute} even in consumers of TokenStreams, because you cannot
+ * know if a specific TokenStream really uses a specific Attribute.
+ * {@link #addAttribute} will automatically make the attribute available.
+ * If you want to only use the attribute, if it is available (to optimize
* consuming), use {@link #hasAttribute}.
*/
public final <T extends Attribute> T getAttribute(Class<T> attClass) {
- AttributeImpl attImpl = attributes.get(attClass);
- if (attImpl == null) {
- throw new IllegalArgumentException("This AttributeSource does not have the attribute '" + attClass.getName() + "'.");
- }
- return attClass.cast(attImpl);
+ return attClass.cast(attributes.get(attClass));
}
private State getCurrentState() {
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java Tue Apr 29 17:18:41 2014
@@ -80,8 +80,7 @@ public final class BytesRefHash {
/**
* Creates a new {@link BytesRefHash}
*/
- public BytesRefHash(ByteBlockPool pool, int capacity,
- BytesStartArray bytesStartArray) {
+ public BytesRefHash(ByteBlockPool pool, int capacity, BytesStartArray bytesStartArray) {
hashSize = capacity;
hashHalfSize = hashSize >> 1;
hashMask = hashSize - 1;
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java Tue Apr 29 17:18:41 2014
@@ -207,12 +207,8 @@ public class QueryBuilder {
buffer = new CachingTokenFilter(source);
buffer.reset();
- if (buffer.hasAttribute(TermToBytesRefAttribute.class)) {
- termAtt = buffer.getAttribute(TermToBytesRefAttribute.class);
- }
- if (buffer.hasAttribute(PositionIncrementAttribute.class)) {
- posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
- }
+ termAtt = buffer.getAttribute(TermToBytesRefAttribute.class);
+ posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
if (termAtt != null) {
try {
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java Tue Apr 29 17:18:41 2014
@@ -31,10 +31,12 @@ public class TestNumericTokenStream exte
public void testLongStream() throws Exception {
final NumericTokenStream stream=new NumericTokenStream().setLongValue(lvalue);
- // use getAttribute to test if attributes really exist, if not an IAE will be throwed
final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+ assertNotNull(bytesAtt);
final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
+ assertNotNull(typeAtt);
final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
+ assertNotNull(numericAtt);
final BytesRef bytes = bytesAtt.getBytesRef();
stream.reset();
assertEquals(64, numericAtt.getValueSize());
@@ -53,10 +55,12 @@ public class TestNumericTokenStream exte
public void testIntStream() throws Exception {
final NumericTokenStream stream=new NumericTokenStream().setIntValue(ivalue);
- // use getAttribute to test if attributes really exist, if not an IAE will be throwed
final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+ assertNotNull(bytesAtt);
final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
+ assertNotNull(typeAtt);
final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
+ assertNotNull(numericAtt);
final BytesRef bytes = bytesAtt.getBytesRef();
stream.reset();
assertEquals(32, numericAtt.getValueSize());
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java Tue Apr 29 17:18:41 2014
@@ -131,7 +131,7 @@ public class TestDocument extends Lucene
assertEquals(0, doc.getFields().size());
}
- public void testConstructorExceptions() {
+ public void testConstructorExceptions() throws Exception {
FieldType ft = new FieldType();
ft.setStored(true);
new Field("name", "value", ft); // okay
@@ -142,16 +142,23 @@ public class TestDocument extends Lucene
} catch (IllegalArgumentException e) {
// expected exception
}
+
+ Directory dir = newDirectory();
+ RandomIndexWriter w = new RandomIndexWriter(random(), dir);
new Field("name", "value", ft); // okay
+ Document doc = new Document();
+ FieldType ft2 = new FieldType();
+ ft2.setStored(true);
+ ft2.setStoreTermVectors(true);
+ doc.add(new Field("name", "value", ft2));
try {
- FieldType ft2 = new FieldType();
- ft2.setStored(true);
- ft2.setStoreTermVectors(true);
- new Field("name", "value", ft2);
+ w.addDocument(doc);
fail();
} catch (IllegalArgumentException e) {
// expected exception
}
+ w.close();
+ dir.close();
}
public void testClearDocument() {
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java Tue Apr 29 17:18:41 2014
@@ -31,11 +31,10 @@ import org.apache.lucene.analysis.Canned
*/
public final class BinaryTokenStream extends TokenStream {
private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
- private final BytesRef bytes;
private boolean available = true;
public BinaryTokenStream(BytesRef bytes) {
- this.bytes = bytes;
+ bytesAtt.setBytesRef(bytes);
}
@Override
@@ -43,7 +42,6 @@ public final class BinaryTokenStream ext
if (available) {
clearAttributes();
available = false;
- bytesAtt.setBytesRef(bytes);
return true;
}
return false;
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java Tue Apr 29 17:18:41 2014
@@ -279,21 +279,14 @@ public class TestDocumentWriter extends
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
TEST_VERSION_CURRENT, new MockAnalyzer(random())));
- writer.addDocument(doc);
+ try {
+ writer.addDocument(doc);
+ fail("did not hit exception");
+ } catch (IllegalArgumentException iae) {
+ assertEquals("all instances of a given field name must have the same term vectors settings (storeTermVectors changed for field=\"f1\")",
+ iae.getMessage());
+ }
writer.shutdown();
-
- TestUtil.checkIndex(dir);
-
- IndexReader reader = DirectoryReader.open(dir);
- // f1
- Terms tfv1 = reader.getTermVectors(0).terms("f1");
- assertNotNull(tfv1);
- assertEquals("the 'with_tv' setting should rule!",2,tfv1.size());
- // f2
- Terms tfv2 = reader.getTermVectors(0).terms("f2");
- assertNotNull(tfv2);
- assertEquals("the 'with_tv' setting should rule!",2,tfv2.size());
- reader.close();
}
/**
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java Tue Apr 29 17:18:41 2014
@@ -20,11 +20,14 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.io.Reader;
import java.io.StringReader;
+import java.util.Collections;
import java.util.Iterator;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
@@ -378,4 +381,63 @@ public class TestIndexableField extends
r.close();
dir.close();
}
+
+ private static class CustomField implements StorableField {
+ @Override
+ public BytesRef binaryValue() {
+ return null;
+ }
+
+ @Override
+ public String stringValue() {
+ return "foobar";
+ }
+
+ @Override
+ public Reader readerValue() {
+ return null;
+ }
+
+ @Override
+ public Number numericValue() {
+ return null;
+ }
+
+ @Override
+ public String name() {
+ return "field";
+ }
+
+ @Override
+ public IndexableFieldType fieldType() {
+ FieldType ft = new FieldType(StoredField.TYPE);
+ ft.setStoreTermVectors(true);
+ ft.freeze();
+ return ft;
+ }
+ }
+
+ // LUCENE-5611
+ public void testNotIndexedTermVectors() throws Exception {
+ Directory dir = newDirectory();
+ RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+ try {
+ w.addDocument(
+ new IndexDocument() {
+ @Override
+ public Iterable<IndexableField> indexableFields() {
+ return Collections.emptyList();
+ }
+ @Override
+ public Iterable<StorableField> storableFields() {
+ return Collections.<StorableField>singletonList(new CustomField());
+ }
+ });
+ fail("didn't hit exception");
+ } catch (IllegalArgumentException iae) {
+ // expected
+ }
+ w.close();
+ dir.close();
+ }
}
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestParallelReaderEmptyIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestParallelReaderEmptyIndex.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestParallelReaderEmptyIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestParallelReaderEmptyIndex.java Tue Apr 29 17:18:41 2014
@@ -98,7 +98,7 @@ public class TestParallelReaderEmptyInde
doc.add(newField("test", "", customType));
idField.setStringValue("1");
iw.addDocument(doc);
- doc.add(newTextField("test", "", Field.Store.NO));
+ doc.add(newField("test", "", customType));
idField.setStringValue("2");
iw.addDocument(doc);
iw.shutdown();
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java Tue Apr 29 17:18:41 2014
@@ -639,8 +639,8 @@ public class TestStressIndexing2 extends
int freq1 = dpEnum1.freq();
int freq2 = dpEnum2.freq();
assertEquals(freq1, freq2);
- OffsetAttribute offsetAtt1 = dpEnum1.attributes().hasAttribute(OffsetAttribute.class) ? dpEnum1.attributes().getAttribute(OffsetAttribute.class) : null;
- OffsetAttribute offsetAtt2 = dpEnum2.attributes().hasAttribute(OffsetAttribute.class) ? dpEnum2.attributes().getAttribute(OffsetAttribute.class) : null;
+ OffsetAttribute offsetAtt1 = dpEnum1.attributes().getAttribute(OffsetAttribute.class);
+ OffsetAttribute offsetAtt2 = dpEnum2.attributes().getAttribute(OffsetAttribute.class);
if (offsetAtt1 != null) {
assertNotNull(offsetAtt2);
@@ -773,24 +773,39 @@ public class TestStressIndexing2 extends
Field idField = newField("id", idString, customType1);
fields.add(idField);
+ Map<String,FieldType> tvTypes = new HashMap<>();
+
int nFields = nextInt(maxFields);
for (int i=0; i<nFields; i++) {
- FieldType customType = new FieldType();
- switch (nextInt(4)) {
- case 0:
- break;
- case 1:
- customType.setStoreTermVectors(true);
- break;
- case 2:
- customType.setStoreTermVectors(true);
- customType.setStoreTermVectorPositions(true);
- break;
- case 3:
- customType.setStoreTermVectors(true);
- customType.setStoreTermVectorOffsets(true);
- break;
+ String fieldName = "f" + nextInt(100);
+ FieldType customType;
+
+ // Use the same term vector settings if we already
+ // added this field to the doc:
+ FieldType oldTVType = tvTypes.get(fieldName);
+ if (oldTVType != null) {
+ customType = new FieldType(oldTVType);
+ } else {
+ customType = new FieldType();
+ switch (nextInt(4)) {
+ case 0:
+ break;
+ case 1:
+ customType.setStoreTermVectors(true);
+ break;
+ case 2:
+ customType.setStoreTermVectors(true);
+ customType.setStoreTermVectorPositions(true);
+ break;
+ case 3:
+ customType.setStoreTermVectors(true);
+ customType.setStoreTermVectorOffsets(true);
+ break;
+ }
+ FieldType newType = new FieldType(customType);
+ newType.freeze();
+ tvTypes.put(fieldName, newType);
}
switch (nextInt(4)) {
@@ -798,26 +813,30 @@ public class TestStressIndexing2 extends
customType.setStored(true);
customType.setOmitNorms(true);
customType.setIndexed(true);
- fields.add(newField("f" + nextInt(100), getString(1), customType));
+ customType.freeze();
+ fields.add(newField(fieldName, getString(1), customType));
break;
case 1:
customType.setIndexed(true);
customType.setTokenized(true);
- fields.add(newField("f" + nextInt(100), getString(0), customType));
+ customType.freeze();
+ fields.add(newField(fieldName, getString(0), customType));
break;
case 2:
customType.setStored(true);
customType.setStoreTermVectors(false);
customType.setStoreTermVectorOffsets(false);
customType.setStoreTermVectorPositions(false);
- fields.add(newField("f" + nextInt(100), getString(0), customType));
+ customType.freeze();
+ fields.add(newField(fieldName, getString(0), customType));
break;
case 3:
customType.setStored(true);
customType.setIndexed(true);
customType.setTokenized(true);
- fields.add(newField("f" + nextInt(100), getString(bigFieldSize), customType));
- break;
+ customType.freeze();
+ fields.add(newField(fieldName, getString(bigFieldSize), customType));
+ break;
}
}
@@ -872,8 +891,7 @@ public class TestStressIndexing2 extends
}
}
} catch (Throwable e) {
- e.printStackTrace();
- Assert.fail(e.toString());
+ throw new RuntimeException(e);
}
synchronized (this) {
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java Tue Apr 29 17:18:41 2014
@@ -29,6 +29,7 @@ import org.apache.lucene.codecs.TermVect
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
@@ -109,6 +110,7 @@ public class TestTermVectorsReader exten
}
else if (!testFieldsStorePos[i] && testFieldsStoreOff[i]) {
customType.setStoreTermVectors(true);
+ customType.setStoreTermVectorPositions(true);
customType.setStoreTermVectorOffsets(true);
}
else {
@@ -333,7 +335,9 @@ public class TestTermVectorsReader exten
public void testIllegalIndexableField() throws Exception {
Directory dir = newDirectory();
- RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+ MockAnalyzer a = new MockAnalyzer(random());
+ a.setEnableChecks(false);
+ RandomIndexWriter w = new RandomIndexWriter(random(), dir, a);
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
ft.setStoreTermVectors(true);
ft.setStoreTermVectorPayloads(true);
@@ -386,6 +390,67 @@ public class TestTermVectorsReader exten
assertEquals("cannot index term vector payloads when term vectors are not indexed (field=\"field\")", iae.getMessage());
}
+ ft = new FieldType(TextField.TYPE_NOT_STORED);
+ ft.setStoreTermVectors(true);
+ ft.setStoreTermVectorPayloads(true);
+ doc = new Document();
+ doc.add(new Field("field", "value", ft));
+ try {
+ w.addDocument(doc);
+ fail("did not hit exception");
+ } catch (IllegalArgumentException iae) {
+ // Expected
+ assertEquals("cannot index term vector payloads without term vector positions (field=\"field\")", iae.getMessage());
+ }
+
+ ft = new FieldType(StoredField.TYPE);
+ ft.setStoreTermVectors(true);
+ doc = new Document();
+ doc.add(new Field("field", "value", ft));
+ try {
+ w.addDocument(doc);
+ fail("did not hit exception");
+ } catch (IllegalArgumentException iae) {
+ // Expected
+ assertEquals("cannot store term vectors for a field that is not indexed (field=\"field\")", iae.getMessage());
+ }
+
+ ft = new FieldType(StoredField.TYPE);
+ ft.setStoreTermVectorPositions(true);
+ doc = new Document();
+ doc.add(new Field("field", "value", ft));
+ try {
+ w.addDocument(doc);
+ fail("did not hit exception");
+ } catch (IllegalArgumentException iae) {
+ // Expected
+ assertEquals("cannot store term vector positions for a field that is not indexed (field=\"field\")", iae.getMessage());
+ }
+
+ ft = new FieldType(StoredField.TYPE);
+ ft.setStoreTermVectorOffsets(true);
+ doc = new Document();
+ doc.add(new Field("field", "value", ft));
+ try {
+ w.addDocument(doc);
+ fail("did not hit exception");
+ } catch (IllegalArgumentException iae) {
+ // Expected
+ assertEquals("cannot store term vector offsets for a field that is not indexed (field=\"field\")", iae.getMessage());
+ }
+
+ ft = new FieldType(StoredField.TYPE);
+ ft.setStoreTermVectorPayloads(true);
+ doc = new Document();
+ doc.add(new Field("field", "value", ft));
+ try {
+ w.addDocument(doc);
+ fail("did not hit exception");
+ } catch (IllegalArgumentException iae) {
+ // Expected
+ assertEquals("cannot store term vector payloads for a field that is not indexed (field=\"field\")", iae.getMessage());
+ }
+
w.shutdown();
dir.close();
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java Tue Apr 29 17:18:41 2014
@@ -534,6 +534,7 @@ public class TestTermVectorsWriter exten
FieldType customType = new FieldType(StringField.TYPE_NOT_STORED);
customType.setStoreTermVectors(true);
+ document = new Document();
document.add(newField("tvtest", "a b c", customType));
iw.addDocument(document);
// Make 2nd segment
@@ -567,6 +568,7 @@ public class TestTermVectorsWriter exten
FieldType customType2 = new FieldType(StringField.TYPE_NOT_STORED);
customType2.setStoreTermVectors(true);
document.add(newField("tvtest", "a b c", customType2));
+ document = new Document();
iw.addDocument(document);
// Make 2nd segment
iw.commit();
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java Tue Apr 29 17:18:41 2014
@@ -26,15 +26,10 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.English;
import org.apache.lucene.util.IOUtils;
@@ -116,44 +111,14 @@ public class TestTermVectors extends Luc
doc.add(newField("field", "one", ft3));
doc.add(newField("field", "one", ft4));
doc.add(newField("field", "one", ft5));
- writer.addDocument(doc);
- IndexReader reader = writer.getReader();
- writer.shutdown();
-
- IndexSearcher searcher = newSearcher(reader);
-
- Query query = new TermQuery(new Term("field", "one"));
- ScoreDoc[] hits = searcher.search(query, null, 1000).scoreDocs;
- assertEquals(1, hits.length);
-
- Fields vectors = searcher.reader.getTermVectors(hits[0].doc);
- assertNotNull(vectors);
- assertEquals(1, vectors.size());
- Terms vector = vectors.terms("field");
- assertNotNull(vector);
- assertEquals(1, vector.size());
- TermsEnum termsEnum = vector.iterator(null);
- assertNotNull(termsEnum.next());
- assertEquals("one", termsEnum.term().utf8ToString());
- assertEquals(5, termsEnum.totalTermFreq());
- DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
- assertNotNull(dpEnum);
- assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
- assertEquals(5, dpEnum.freq());
- for(int i=0;i<5;i++) {
- assertEquals(i, dpEnum.nextPosition());
- }
-
- dpEnum = termsEnum.docsAndPositions(null, dpEnum);
- assertNotNull(dpEnum);
- assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
- assertEquals(5, dpEnum.freq());
- for(int i=0;i<5;i++) {
- dpEnum.nextPosition();
- assertEquals(4*i, dpEnum.startOffset());
- assertEquals(4*i+3, dpEnum.endOffset());
+ try {
+ writer.addDocument(doc);
+ fail("should have hit exception");
+ } catch (IllegalArgumentException iae) {
+ assertEquals("all instances of a given field name must have the same term vectors settings (storeTermVectors changed for field=\"field\")",
+ iae.getMessage());
}
- reader.close();
+ writer.shutdown();
}
private IndexWriter createWriter(Directory dir) throws IOException {
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java Tue Apr 29 17:18:41 2014
@@ -54,7 +54,7 @@ public class TestHugeRamFile extends Luc
public void testHugeFile() throws IOException {
DenseRAMFile f = new DenseRAMFile();
// output part
- RAMOutputStream out = new RAMOutputStream(f);
+ RAMOutputStream out = new RAMOutputStream(f, true);
byte b1[] = new byte[RAMOutputStream.BUFFER_SIZE];
byte b2[] = new byte[RAMOutputStream.BUFFER_SIZE / 3];
for (int i = 0; i < b1.length; i++) {
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java Tue Apr 29 17:18:41 2014
@@ -90,7 +90,9 @@ public class TestAttributeSource extends
assertFalse("No more attributes", it.hasNext());
final FlagsAttribute flagsAtt2 = clone.getAttribute(FlagsAttribute.class);
+ assertNotNull(flagsAtt2);
final TypeAttribute typeAtt2 = clone.getAttribute(TypeAttribute.class);
+ assertNotNull(typeAtt2);
assertNotSame("FlagsAttribute of original and clone must be different instances", flagsAtt2, flagsAtt);
assertNotSame("TypeAttribute of original and clone must be different instances", typeAtt2, typeAtt);
assertEquals("FlagsAttribute of original and clone must be equal", flagsAtt2, flagsAtt);
Modified: lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java Tue Apr 29 17:18:41 2014
@@ -582,7 +582,7 @@ public class SortingAtomicReader extends
file = new RAMFile();
sorter = new DocOffsetSorter(maxDoc);
}
- final IndexOutput out = new RAMOutputStream(file);
+ final IndexOutput out = new RAMOutputStream(file, false);
int doc;
int i = 0;
while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Tue Apr 29 17:18:41 2014
@@ -685,11 +685,11 @@ public abstract class BaseTokenStreamTes
int remainder = random.nextInt(10);
Reader reader = new StringReader(text);
TokenStream ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
- CharTermAttribute termAtt = ts.hasAttribute(CharTermAttribute.class) ? ts.getAttribute(CharTermAttribute.class) : null;
- OffsetAttribute offsetAtt = ts.hasAttribute(OffsetAttribute.class) ? ts.getAttribute(OffsetAttribute.class) : null;
- PositionIncrementAttribute posIncAtt = ts.hasAttribute(PositionIncrementAttribute.class) ? ts.getAttribute(PositionIncrementAttribute.class) : null;
- PositionLengthAttribute posLengthAtt = ts.hasAttribute(PositionLengthAttribute.class) ? ts.getAttribute(PositionLengthAttribute.class) : null;
- TypeAttribute typeAtt = ts.hasAttribute(TypeAttribute.class) ? ts.getAttribute(TypeAttribute.class) : null;
+ CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
+ OffsetAttribute offsetAtt = ts.getAttribute(OffsetAttribute.class);
+ PositionIncrementAttribute posIncAtt = ts.getAttribute(PositionIncrementAttribute.class);
+ PositionLengthAttribute posLengthAtt = ts.getAttribute(PositionLengthAttribute.class);
+ TypeAttribute typeAtt = ts.getAttribute(TypeAttribute.class);
List<String> tokens = new ArrayList<>();
List<String> types = new ArrayList<>();
List<Integer> positions = new ArrayList<>();
Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java Tue Apr 29 17:18:41 2014
@@ -25,7 +25,6 @@ import org.apache.lucene.analysis.tokena
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
-import org.apache.lucene.util.Attribute;
// TODO: rename to OffsetsXXXTF? ie we only validate
// offsets (now anyway...)
@@ -47,23 +46,14 @@ public final class ValidatingTokenFilter
private final Map<Integer,Integer> posToStartOffset = new HashMap<>();
private final Map<Integer,Integer> posToEndOffset = new HashMap<>();
- private final PositionIncrementAttribute posIncAtt = getAttrIfExists(PositionIncrementAttribute.class);
- private final PositionLengthAttribute posLenAtt = getAttrIfExists(PositionLengthAttribute.class);
- private final OffsetAttribute offsetAtt = getAttrIfExists(OffsetAttribute.class);
- private final CharTermAttribute termAtt = getAttrIfExists(CharTermAttribute.class);
+ private final PositionIncrementAttribute posIncAtt = getAttribute(PositionIncrementAttribute.class);
+ private final PositionLengthAttribute posLenAtt = getAttribute(PositionLengthAttribute.class);
+ private final OffsetAttribute offsetAtt = getAttribute(OffsetAttribute.class);
+ private final CharTermAttribute termAtt = getAttribute(CharTermAttribute.class);
private final boolean offsetsAreCorrect;
private final String name;
- // Returns null if the attr wasn't already added
- private <A extends Attribute> A getAttrIfExists(Class<A> att) {
- if (hasAttribute(att)) {
- return getAttribute(att);
- } else {
- return null;
- }
- }
-
/** The name arg is used to identify this stage when
* throwing exceptions (useful if you have more than one
* instance in your chain). */
Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java Tue Apr 29 17:18:41 2014
@@ -90,7 +90,6 @@ public class AssertingStoredFieldsFormat
static class AssertingStoredFieldsWriter extends StoredFieldsWriter {
private final StoredFieldsWriter in;
private int numWritten;
- private int fieldCount;
private Status docStatus;
AssertingStoredFieldsWriter(StoredFieldsWriter in) {
@@ -99,11 +98,9 @@ public class AssertingStoredFieldsFormat
}
@Override
- public void startDocument(int numStoredFields) throws IOException {
+ public void startDocument() throws IOException {
assert docStatus != Status.STARTED;
- in.startDocument(numStoredFields);
- assert fieldCount == 0;
- fieldCount = numStoredFields;
+ in.startDocument();
numWritten++;
docStatus = Status.STARTED;
}
@@ -111,7 +108,6 @@ public class AssertingStoredFieldsFormat
@Override
public void finishDocument() throws IOException {
assert docStatus == Status.STARTED;
- assert fieldCount == 0;
in.finishDocument();
docStatus = Status.FINISHED;
}
@@ -120,8 +116,6 @@ public class AssertingStoredFieldsFormat
public void writeField(FieldInfo info, StorableField field) throws IOException {
assert docStatus == Status.STARTED;
in.writeField(info, field);
- assert fieldCount > 0;
- fieldCount--;
}
@Override
@@ -133,7 +127,6 @@ public class AssertingStoredFieldsFormat
public void finish(FieldInfos fis, int numDocs) throws IOException {
assert docStatus == (numDocs > 0 ? Status.FINISHED : Status.UNDEFINED);
in.finish(fis, numDocs);
- assert fieldCount == 0;
assert numDocs == numWritten;
}
Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Tue Apr 29 17:18:41 2014
@@ -17,9 +17,6 @@ package org.apache.lucene.util;
* limitations under the License.
*/
-import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsBoolean;
-import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsInt;
-
import java.io.Closeable;
import java.io.File;
import java.io.FileNotFoundException;
@@ -40,14 +37,17 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.EnumSet;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
+import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.TimeZone;
import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@@ -56,8 +56,8 @@ import java.util.logging.Logger;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
@@ -76,8 +76,8 @@ import org.apache.lucene.index.FieldFilt
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
+import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LogByteSizeMergePolicy;
import org.apache.lucene.index.LogDocMergePolicy;
@@ -98,25 +98,25 @@ import org.apache.lucene.index.SortedSet
import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.StoredDocument;
import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.AssertingIndexSearcher;
import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldCache.CacheEntry;
+import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.QueryUtils.FCInvisibleMultiReader;
import org.apache.lucene.store.BaseDirectoryWrapper;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.FlushInfo;
-import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IOContext.Context;
+import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.LockFactory;
import org.apache.lucene.store.MergeInfo;
-import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
+import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.NRTCachingDirectory;
import org.apache.lucene.store.RateLimitedDirectoryWrapper;
import org.apache.lucene.util.FieldCacheSanityChecker.Insanity;
@@ -134,7 +134,6 @@ import org.junit.Test;
import org.junit.rules.RuleChain;
import org.junit.rules.TestRule;
import org.junit.runner.RunWith;
-
import com.carrotsearch.randomizedtesting.JUnit4MethodProvider;
import com.carrotsearch.randomizedtesting.LifecycleScope;
import com.carrotsearch.randomizedtesting.MixWithSuiteName;
@@ -145,16 +144,16 @@ import com.carrotsearch.randomizedtestin
import com.carrotsearch.randomizedtesting.annotations.SeedDecorators;
import com.carrotsearch.randomizedtesting.annotations.TestGroup;
import com.carrotsearch.randomizedtesting.annotations.TestMethodProviders;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup.Group;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Consequence;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies;
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
@@ -163,6 +162,9 @@ import com.carrotsearch.randomizedtestin
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsBoolean;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsInt;
+
/**
* Base class for all Lucene unit tests, Junit3 or Junit4 variant.
*
@@ -601,6 +603,8 @@ public abstract class LuceneTestCase ext
.around(new TestRuleFieldCacheSanity())
.around(parentChainCallRule);
+ private static final Map<String,FieldType> fieldToType = new HashMap<String,FieldType>();
+
// -----------------------------------------------------------------
// Suite and test case setup/ cleanup.
// -----------------------------------------------------------------
@@ -619,6 +623,7 @@ public abstract class LuceneTestCase ext
@After
public void tearDown() throws Exception {
parentChainCallRule.teardownCalled = true;
+ fieldToType.clear();
}
@@ -1168,11 +1173,43 @@ public abstract class LuceneTestCase ext
public static Field newField(String name, String value, FieldType type) {
return newField(random(), name, value, type);
}
-
- public static Field newField(Random random, String name, String value, FieldType type) {
+
+ /** Returns a FieldType derived from newType but whose
+ * term vector options match the old type */
+ private static FieldType mergeTermVectorOptions(FieldType newType, FieldType oldType) {
+ if (newType.indexed() && oldType.storeTermVectors() == true && newType.storeTermVectors() == false) {
+ newType = new FieldType(newType);
+ newType.setStoreTermVectors(oldType.storeTermVectors());
+ newType.setStoreTermVectorPositions(oldType.storeTermVectorPositions());
+ newType.setStoreTermVectorOffsets(oldType.storeTermVectorOffsets());
+ newType.setStoreTermVectorPayloads(oldType.storeTermVectorPayloads());
+ newType.freeze();
+ }
+
+ return newType;
+ }
+
+ // TODO: if we can pull out the "make term vector options
+ // consistent across all instances of the same field name"
+ // write-once schema sort of helper class then we can
+ // remove the sync here. We can also fold the random
+ // "enable norms" (now commented out, below) into that:
+ public synchronized static Field newField(Random random, String name, String value, FieldType type) {
+
+ // Defeat any consumers that illegally rely on intern'd
+ // strings (we removed this from Lucene a while back):
name = new String(name);
- if (usually(random) || !type.indexed()) {
+
+ FieldType prevType = fieldToType.get(name);
+
+ if (usually(random) || !type.indexed() || prevType != null) {
// most of the time, don't modify the params
+ if (prevType == null) {
+ fieldToType.put(name, new FieldType(type));
+ } else {
+ type = mergeTermVectorOptions(type, prevType);
+ }
+
return new Field(name, value, type);
}
@@ -1185,19 +1222,29 @@ public abstract class LuceneTestCase ext
newType.setStored(true); // randomly store it
}
+ // Randomly turn on term vector options, but always do
+ // so consistently for the same field name:
if (!newType.storeTermVectors() && random.nextBoolean()) {
newType.setStoreTermVectors(true);
- if (!newType.storeTermVectorOffsets()) {
- newType.setStoreTermVectorOffsets(random.nextBoolean());
- }
if (!newType.storeTermVectorPositions()) {
newType.setStoreTermVectorPositions(random.nextBoolean());
- if (newType.storeTermVectorPositions() && !newType.storeTermVectorPayloads()) {
- newType.setStoreTermVectorPayloads(random.nextBoolean());
+ if (newType.storeTermVectorPositions()) {
+ if (!newType.storeTermVectorPayloads()) {
+ newType.setStoreTermVectorPayloads(random.nextBoolean());
+ }
+ if (!newType.storeTermVectorOffsets()) {
+ newType.setStoreTermVectorOffsets(random.nextBoolean());
+ }
}
}
+
+ if (VERBOSE) {
+ System.out.println("NOTE: LuceneTestCase: upgrade name=" + name + " type=" + newType);
+ }
}
+ newType.freeze();
+ fieldToType.put(name, newType);
// TODO: we need to do this, but smarter, ie, most of
// the time we set the same value for a given field but
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/JsonPreAnalyzedParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/JsonPreAnalyzedParser.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/JsonPreAnalyzedParser.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/JsonPreAnalyzedParser.java Tue Apr 29 17:18:41 2014
@@ -230,10 +230,10 @@ public class JsonPreAnalyzedParser imple
Map<String,Object> tok = new TreeMap<>();
while (it.hasNext()) {
Class<? extends Attribute> cl = it.next();
- if (!ts.hasAttribute(cl)) {
+ Attribute att = ts.getAttribute(cl);
+ if (att == null) {
continue;
}
- Attribute att = ts.getAttribute(cl);
if (cl.isAssignableFrom(CharTermAttribute.class)) {
CharTermAttribute catt = (CharTermAttribute)att;
cTerm = new String(catt.buffer(), 0, catt.length());
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/SimplePreAnalyzedParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/SimplePreAnalyzedParser.java?rev=1591025&r1=1591024&r2=1591025&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/SimplePreAnalyzedParser.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/SimplePreAnalyzedParser.java Tue Apr 29 17:18:41 2014
@@ -482,10 +482,10 @@ public final class SimplePreAnalyzedPars
String tTerm = null;
while (it.hasNext()) {
Class<? extends Attribute> cl = it.next();
- if (!ts.hasAttribute(cl)) {
+ Attribute att = ts.getAttribute(cl);
+ if (att == null) {
continue;
}
- Attribute att = ts.getAttribute(cl);
if (cl.isAssignableFrom(CharTermAttribute.class)) {
CharTermAttribute catt = (CharTermAttribute)att;
cTerm = escape(catt.buffer(), catt.length());