You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/10/21 20:58:44 UTC
svn commit: r1534320 [26/39] - in /lucene/dev/branches/lucene4956: ./
dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/
dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/
dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Mon Oct 21 18:58:24 2013
@@ -17,21 +17,21 @@ package org.apache.lucene.index;
* limitations under the License.
*/
-import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
-import java.util.Map;
import java.util.Map.Entry;
+import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@@ -52,11 +52,14 @@ import org.apache.lucene.search.ScoreDoc
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
+import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
+
/**
* Abstract class to do basic tests for a docvalues format.
* NOTE: This test focuses on the docvalues impl, nothing else.
@@ -648,6 +651,43 @@ public abstract class BaseDocValuesForma
ireader.close();
directory.close();
}
+
+ public void testSortedMergeAwayAllValues() throws IOException {
+ Directory directory = newDirectory();
+ Analyzer analyzer = new MockAnalyzer(random());
+ IndexWriterConfig iwconfig = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+ iwconfig.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, iwconfig);
+
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.NO));
+ iwriter.addDocument(doc);
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.NO));
+ doc.add(new SortedDocValuesField("field", new BytesRef("hello")));
+ iwriter.addDocument(doc);
+ iwriter.commit();
+ iwriter.deleteDocuments(new Term("id", "1"));
+ iwriter.forceMerge(1);
+
+ DirectoryReader ireader = iwriter.getReader();
+ iwriter.close();
+
+ SortedDocValues dv = getOnlySegmentReader(ireader).getSortedDocValues("field");
+ if (defaultCodecSupportsDocsWithField()) {
+ assertEquals(-1, dv.getOrd(0));
+ assertEquals(0, dv.getValueCount());
+ } else {
+ assertEquals(0, dv.getOrd(0));
+ assertEquals(1, dv.getValueCount());
+ BytesRef ref = new BytesRef();
+ dv.lookupOrd(0, ref);
+ assertEquals(new BytesRef(), ref);
+ }
+
+ ireader.close();
+ directory.close();
+ }
public void testBytesWithNewline() throws IOException {
Analyzer analyzer = new MockAnalyzer(random());
@@ -694,7 +734,10 @@ public abstract class BaseDocValuesForma
BytesRef scratch = new BytesRef();
dv.lookupOrd(dv.getOrd(0), scratch);
assertEquals(new BytesRef("hello world 2"), scratch);
- dv.lookupOrd(dv.getOrd(1), scratch);
+ if (defaultCodecSupportsDocsWithField()) {
+ assertEquals(-1, dv.getOrd(1));
+ }
+ dv.get(1, scratch);
assertEquals(new BytesRef(""), scratch);
ireader.close();
directory.close();
@@ -1049,6 +1092,11 @@ public abstract class BaseDocValuesForma
public void testRandomSortedBytes() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig cfg = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+ if (!defaultCodecSupportsDocsWithField()) {
+ // if the codec doesnt support missing, we expect missing to be mapped to byte[]
+ // by the impersonator, but we have to give it a chance to merge them to this
+ cfg.setMergePolicy(newLogMergePolicy());
+ }
RandomIndexWriter w = new RandomIndexWriter(random(), dir, cfg);
int numDocs = atLeast(100);
BytesRefHash hash = new BytesRefHash();
@@ -1073,11 +1121,18 @@ public abstract class BaseDocValuesForma
doc.add(newTextField("id", "noValue", Field.Store.YES));
w.addDocument(doc);
}
- BytesRef bytesRef = new BytesRef();
- hash.add(bytesRef); // add empty value for the gaps
+ if (!defaultCodecSupportsDocsWithField()) {
+ BytesRef bytesRef = new BytesRef();
+ hash.add(bytesRef); // add empty value for the gaps
+ }
if (rarely()) {
w.commit();
}
+ if (!defaultCodecSupportsDocsWithField()) {
+ // if the codec doesnt support missing, we expect missing to be mapped to byte[]
+ // by the impersonator, but we have to give it a chance to merge them to this
+ w.forceMerge(1);
+ }
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
String id = "" + i + numDocs;
@@ -1188,6 +1243,73 @@ public abstract class BaseDocValuesForma
dir.close();
}
+ private void doTestMissingVsFieldCache(final long minValue, final long maxValue) throws Exception {
+ doTestMissingVsFieldCache(new LongProducer() {
+ @Override
+ long next() {
+ return _TestUtil.nextLong(random(), minValue, maxValue);
+ }
+ });
+ }
+
+ private void doTestMissingVsFieldCache(LongProducer longs) throws Exception {
+ assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+ Directory dir = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+ RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+ Field idField = new StringField("id", "", Field.Store.NO);
+ Field indexedField = newStringField("indexed", "", Field.Store.NO);
+ Field dvField = new NumericDocValuesField("dv", 0);
+
+
+ // index some docs
+ int numDocs = atLeast(300);
+ // numDocs should be always > 256 so that in case of a codec that optimizes
+ // for numbers of values <= 256, all storage layouts are tested
+ assert numDocs > 256;
+ for (int i = 0; i < numDocs; i++) {
+ idField.setStringValue(Integer.toString(i));
+ long value = longs.next();
+ indexedField.setStringValue(Long.toString(value));
+ dvField.setLongValue(value);
+ Document doc = new Document();
+ doc.add(idField);
+ // 1/4 of the time we neglect to add the fields
+ if (random().nextInt(4) > 0) {
+ doc.add(indexedField);
+ doc.add(dvField);
+ }
+ writer.addDocument(doc);
+ if (random().nextInt(31) == 0) {
+ writer.commit();
+ }
+ }
+
+ // delete some docs
+ int numDeletions = random().nextInt(numDocs/10);
+ for (int i = 0; i < numDeletions; i++) {
+ int id = random().nextInt(numDocs);
+ writer.deleteDocuments(new Term("id", Integer.toString(id)));
+ }
+
+ // merge some segments and ensure that at least one of them has more than
+ // 256 values
+ writer.forceMerge(numDocs / 256);
+
+ writer.close();
+
+ // compare
+ DirectoryReader ir = DirectoryReader.open(dir);
+ for (AtomicReaderContext context : ir.leaves()) {
+ AtomicReader r = context.reader();
+ Bits expected = FieldCache.DEFAULT.getDocsWithField(r, "indexed");
+ Bits actual = FieldCache.DEFAULT.getDocsWithField(r, "dv");
+ assertEquals(expected, actual);
+ }
+ ir.close();
+ dir.close();
+ }
+
public void testBooleanNumericsVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
@@ -1202,6 +1324,13 @@ public abstract class BaseDocValuesForma
}
}
+ public void testByteMissingVsFieldCache() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestMissingVsFieldCache(Byte.MIN_VALUE, Byte.MAX_VALUE);
+ }
+ }
+
public void testShortNumericsVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
@@ -1209,6 +1338,13 @@ public abstract class BaseDocValuesForma
}
}
+ public void testShortMissingVsFieldCache() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestMissingVsFieldCache(Short.MIN_VALUE, Short.MAX_VALUE);
+ }
+ }
+
public void testIntNumericsVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
@@ -1216,6 +1352,13 @@ public abstract class BaseDocValuesForma
}
}
+ public void testIntMissingVsFieldCache() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestMissingVsFieldCache(Integer.MIN_VALUE, Integer.MAX_VALUE);
+ }
+ }
+
public void testLongNumericsVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
@@ -1223,6 +1366,13 @@ public abstract class BaseDocValuesForma
}
}
+ public void testLongMissingVsFieldCache() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestMissingVsFieldCache(Long.MIN_VALUE, Long.MAX_VALUE);
+ }
+ }
+
private void doTestBinaryVsStoredFields(int minLength, int maxLength) throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
@@ -1282,7 +1432,7 @@ public abstract class BaseDocValuesForma
public void testBinaryFixedLengthVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
- int fixedLength = _TestUtil.nextInt(random(), 1, 10);
+ int fixedLength = _TestUtil.nextInt(random(), 0, 10);
doTestBinaryVsStoredFields(fixedLength, fixedLength);
}
}
@@ -1290,7 +1440,7 @@ public abstract class BaseDocValuesForma
public void testBinaryVariableLengthVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
- doTestBinaryVsStoredFields(1, 10);
+ doTestBinaryVsStoredFields(0, 10);
}
}
@@ -1970,6 +2120,13 @@ public abstract class BaseDocValuesForma
doTestSortedSetVsStoredFields(1, 10);
}
}
+
+ private void assertEquals(Bits expected, Bits actual) throws Exception {
+ assertEquals(expected.length(), actual.length());
+ for (int i = 0; i < expected.length(); i++) {
+ assertEquals(expected.get(i), actual.get(i));
+ }
+ }
private void assertEquals(int maxDoc, SortedDocValues expected, SortedDocValues actual) throws Exception {
assertEquals(maxDoc, new SingletonSortedSetDocValues(expected), new SingletonSortedSetDocValues(actual));
@@ -2197,5 +2354,648 @@ public abstract class BaseDocValuesForma
doTestNumericsVsStoredFields(longs);
}
}
+
+ public void testTwoNumbersOneMissing() throws IOException {
+ assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+ Directory directory = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+ conf.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.YES));
+ doc.add(new NumericDocValuesField("dv1", 0));
+ iw.addDocument(doc);
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.YES));
+ iw.addDocument(doc);
+ iw.forceMerge(1);
+ iw.close();
+
+ IndexReader ir = DirectoryReader.open(directory);
+ assertEquals(1, ir.leaves().size());
+ AtomicReader ar = ir.leaves().get(0).reader();
+ NumericDocValues dv = ar.getNumericDocValues("dv1");
+ assertEquals(0, dv.get(0));
+ assertEquals(0, dv.get(1));
+ Bits docsWithField = ar.getDocsWithField("dv1");
+ assertTrue(docsWithField.get(0));
+ assertFalse(docsWithField.get(1));
+ ir.close();
+ directory.close();
+ }
+
+ public void testTwoNumbersOneMissingWithMerging() throws IOException {
+ assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+ Directory directory = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+ conf.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.YES));
+ doc.add(new NumericDocValuesField("dv1", 0));
+ iw.addDocument(doc);
+ iw.commit();
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.YES));
+ iw.addDocument(doc);
+ iw.forceMerge(1);
+ iw.close();
+
+ IndexReader ir = DirectoryReader.open(directory);
+ assertEquals(1, ir.leaves().size());
+ AtomicReader ar = ir.leaves().get(0).reader();
+ NumericDocValues dv = ar.getNumericDocValues("dv1");
+ assertEquals(0, dv.get(0));
+ assertEquals(0, dv.get(1));
+ Bits docsWithField = ar.getDocsWithField("dv1");
+ assertTrue(docsWithField.get(0));
+ assertFalse(docsWithField.get(1));
+ ir.close();
+ directory.close();
+ }
+
+ public void testThreeNumbersOneMissingWithMerging() throws IOException {
+ assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+ Directory directory = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+ conf.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.YES));
+ doc.add(new NumericDocValuesField("dv1", 0));
+ iw.addDocument(doc);
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.YES));
+ iw.addDocument(doc);
+ iw.commit();
+ doc = new Document();
+ doc.add(new StringField("id", "2", Field.Store.YES));
+ doc.add(new NumericDocValuesField("dv1", 5));
+ iw.addDocument(doc);
+ iw.forceMerge(1);
+ iw.close();
+
+ IndexReader ir = DirectoryReader.open(directory);
+ assertEquals(1, ir.leaves().size());
+ AtomicReader ar = ir.leaves().get(0).reader();
+ NumericDocValues dv = ar.getNumericDocValues("dv1");
+ assertEquals(0, dv.get(0));
+ assertEquals(0, dv.get(1));
+ assertEquals(5, dv.get(2));
+ Bits docsWithField = ar.getDocsWithField("dv1");
+ assertTrue(docsWithField.get(0));
+ assertFalse(docsWithField.get(1));
+ assertTrue(docsWithField.get(2));
+ ir.close();
+ directory.close();
+ }
+
+ public void testTwoBytesOneMissing() throws IOException {
+ assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+ Directory directory = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+ conf.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.YES));
+ doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+ iw.addDocument(doc);
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.YES));
+ iw.addDocument(doc);
+ iw.forceMerge(1);
+ iw.close();
+
+ IndexReader ir = DirectoryReader.open(directory);
+ assertEquals(1, ir.leaves().size());
+ AtomicReader ar = ir.leaves().get(0).reader();
+ BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+ BytesRef ref = new BytesRef();
+ dv.get(0, ref);
+ assertEquals(new BytesRef(), ref);
+ dv.get(1, ref);
+ assertEquals(new BytesRef(), ref);
+ Bits docsWithField = ar.getDocsWithField("dv1");
+ assertTrue(docsWithField.get(0));
+ assertFalse(docsWithField.get(1));
+ ir.close();
+ directory.close();
+ }
+
+ public void testTwoBytesOneMissingWithMerging() throws IOException {
+ assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+ Directory directory = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+ conf.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.YES));
+ doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+ iw.addDocument(doc);
+ iw.commit();
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.YES));
+ iw.addDocument(doc);
+ iw.forceMerge(1);
+ iw.close();
+
+ IndexReader ir = DirectoryReader.open(directory);
+ assertEquals(1, ir.leaves().size());
+ AtomicReader ar = ir.leaves().get(0).reader();
+ BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+ BytesRef ref = new BytesRef();
+ dv.get(0, ref);
+ assertEquals(new BytesRef(), ref);
+ dv.get(1, ref);
+ assertEquals(new BytesRef(), ref);
+ Bits docsWithField = ar.getDocsWithField("dv1");
+ assertTrue(docsWithField.get(0));
+ assertFalse(docsWithField.get(1));
+ ir.close();
+ directory.close();
+ }
+
+ public void testThreeBytesOneMissingWithMerging() throws IOException {
+ assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+ Directory directory = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+ conf.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.YES));
+ doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+ iw.addDocument(doc);
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.YES));
+ iw.addDocument(doc);
+ iw.commit();
+ doc = new Document();
+ doc.add(new StringField("id", "2", Field.Store.YES));
+ doc.add(new BinaryDocValuesField("dv1", new BytesRef("boo")));
+ iw.addDocument(doc);
+ iw.forceMerge(1);
+ iw.close();
+
+ IndexReader ir = DirectoryReader.open(directory);
+ assertEquals(1, ir.leaves().size());
+ AtomicReader ar = ir.leaves().get(0).reader();
+ BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+ BytesRef ref = new BytesRef();
+ dv.get(0, ref);
+ assertEquals(new BytesRef(), ref);
+ dv.get(1, ref);
+ assertEquals(new BytesRef(), ref);
+ dv.get(2, ref);
+ assertEquals(new BytesRef("boo"), ref);
+ Bits docsWithField = ar.getDocsWithField("dv1");
+ assertTrue(docsWithField.get(0));
+ assertFalse(docsWithField.get(1));
+ assertTrue(docsWithField.get(2));
+ ir.close();
+ directory.close();
+ }
+
+ // LUCENE-4853
+ public void testHugeBinaryValues() throws Exception {
+ Analyzer analyzer = new MockAnalyzer(random());
+ // FSDirectory because SimpleText will consume gobbs of
+ // space when storing big binary values:
+ Directory d = newFSDirectory(_TestUtil.getTempDir("hugeBinaryValues"));
+ boolean doFixed = random().nextBoolean();
+ int numDocs;
+ int fixedLength = 0;
+ if (doFixed) {
+ // Sometimes make all values fixed length since some
+ // codecs have different code paths for this:
+ numDocs = _TestUtil.nextInt(random(), 10, 20);
+ fixedLength = _TestUtil.nextInt(random(), 65537, 256*1024);
+ } else {
+ numDocs = _TestUtil.nextInt(random(), 100, 200);
+ }
+ IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
+ List<byte[]> docBytes = new ArrayList<byte[]>();
+ long totalBytes = 0;
+ for(int docID=0;docID<numDocs;docID++) {
+ // we don't use RandomIndexWriter because it might add
+ // more docvalues than we expect !!!!
+
+ // Must be > 64KB in size to ensure more than 2 pages in
+ // PagedBytes would be needed:
+ int numBytes;
+ if (doFixed) {
+ numBytes = fixedLength;
+ } else if (docID == 0 || random().nextInt(5) == 3) {
+ numBytes = _TestUtil.nextInt(random(), 65537, 3*1024*1024);
+ } else {
+ numBytes = _TestUtil.nextInt(random(), 1, 1024*1024);
+ }
+ totalBytes += numBytes;
+ if (totalBytes > 5 * 1024*1024) {
+ break;
+ }
+ byte[] bytes = new byte[numBytes];
+ random().nextBytes(bytes);
+ docBytes.add(bytes);
+ Document doc = new Document();
+ BytesRef b = new BytesRef(bytes);
+ b.length = bytes.length;
+ doc.add(new BinaryDocValuesField("field", b));
+ doc.add(new StringField("id", ""+docID, Field.Store.YES));
+ try {
+ w.addDocument(doc);
+ } catch (IllegalArgumentException iae) {
+ if (iae.getMessage().indexOf("is too large") == -1) {
+ throw iae;
+ } else {
+ // OK: some codecs can't handle binary DV > 32K
+ assertFalse(codecAcceptsHugeBinaryValues("field"));
+ w.rollback();
+ d.close();
+ return;
+ }
+ }
+ }
+
+ DirectoryReader r;
+ try {
+ r = w.getReader();
+ } catch (IllegalArgumentException iae) {
+ if (iae.getMessage().indexOf("is too large") == -1) {
+ throw iae;
+ } else {
+ assertFalse(codecAcceptsHugeBinaryValues("field"));
+
+ // OK: some codecs can't handle binary DV > 32K
+ w.rollback();
+ d.close();
+ return;
+ }
+ }
+ w.close();
+
+ AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+
+ BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field", false);
+ for(int docID=0;docID<docBytes.size();docID++) {
+ StoredDocument doc = ar.document(docID);
+ BytesRef bytes = new BytesRef();
+ s.get(docID, bytes);
+ byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
+ assertEquals(expected.length, bytes.length);
+ assertEquals(new BytesRef(expected), bytes);
+ }
+
+ assertTrue(codecAcceptsHugeBinaryValues("field"));
+
+ ar.close();
+ d.close();
+ }
+
+ // TODO: get this out of here and into the deprecated codecs (4.0, 4.2)
+ public void testHugeBinaryValueLimit() throws Exception {
+ // We only test DVFormats that have a limit
+ assumeFalse("test requires codec with limits on max binary field length", codecAcceptsHugeBinaryValues("field"));
+ Analyzer analyzer = new MockAnalyzer(random());
+ // FSDirectory because SimpleText will consume gobbs of
+ // space when storing big binary values:
+ Directory d = newFSDirectory(_TestUtil.getTempDir("hugeBinaryValues"));
+ boolean doFixed = random().nextBoolean();
+ int numDocs;
+ int fixedLength = 0;
+ if (doFixed) {
+ // Sometimes make all values fixed length since some
+ // codecs have different code paths for this:
+ numDocs = _TestUtil.nextInt(random(), 10, 20);
+ fixedLength = Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH;
+ } else {
+ numDocs = _TestUtil.nextInt(random(), 100, 200);
+ }
+ IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
+ List<byte[]> docBytes = new ArrayList<byte[]>();
+ long totalBytes = 0;
+ for(int docID=0;docID<numDocs;docID++) {
+ // we don't use RandomIndexWriter because it might add
+ // more docvalues than we expect !!!!
+
+ // Must be > 64KB in size to ensure more than 2 pages in
+ // PagedBytes would be needed:
+ int numBytes;
+ if (doFixed) {
+ numBytes = fixedLength;
+ } else if (docID == 0 || random().nextInt(5) == 3) {
+ numBytes = Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH;
+ } else {
+ numBytes = _TestUtil.nextInt(random(), 1, Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH);
+ }
+ totalBytes += numBytes;
+ if (totalBytes > 5 * 1024*1024) {
+ break;
+ }
+ byte[] bytes = new byte[numBytes];
+ random().nextBytes(bytes);
+ docBytes.add(bytes);
+ Document doc = new Document();
+ BytesRef b = new BytesRef(bytes);
+ b.length = bytes.length;
+ doc.add(new BinaryDocValuesField("field", b));
+ doc.add(new StringField("id", ""+docID, Field.Store.YES));
+ w.addDocument(doc);
+ }
+
+ DirectoryReader r = w.getReader();
+ w.close();
+
+ AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+
+ BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field", false);
+ for(int docID=0;docID<docBytes.size();docID++) {
+ StoredDocument doc = ar.document(docID);
+ BytesRef bytes = new BytesRef();
+ s.get(docID, bytes);
+ byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
+ assertEquals(expected.length, bytes.length);
+ assertEquals(new BytesRef(expected), bytes);
+ }
+
+ ar.close();
+ d.close();
+ }
+
+ /** Tests dv against stored fields with threads (binary/numeric/sorted, no missing) */
+ public void testThreads() throws Exception {
+ Directory dir = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+ RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+ Document doc = new Document();
+ Field idField = new StringField("id", "", Field.Store.NO);
+ Field storedBinField = new StoredField("storedBin", new byte[0]);
+ Field dvBinField = new BinaryDocValuesField("dvBin", new BytesRef());
+ Field dvSortedField = new SortedDocValuesField("dvSorted", new BytesRef());
+ Field storedNumericField = new StoredField("storedNum", "");
+ Field dvNumericField = new NumericDocValuesField("dvNum", 0);
+ doc.add(idField);
+ doc.add(storedBinField);
+ doc.add(dvBinField);
+ doc.add(dvSortedField);
+ doc.add(storedNumericField);
+ doc.add(dvNumericField);
+
+ // index some docs
+ int numDocs = atLeast(300);
+ for (int i = 0; i < numDocs; i++) {
+ idField.setStringValue(Integer.toString(i));
+ int length = _TestUtil.nextInt(random(), 0, 8);
+ byte buffer[] = new byte[length];
+ random().nextBytes(buffer);
+ storedBinField.setBytesValue(buffer);
+ dvBinField.setBytesValue(buffer);
+ dvSortedField.setBytesValue(buffer);
+ long numericValue = random().nextLong();
+ storedNumericField.setStringValue(Long.toString(numericValue));
+ dvNumericField.setLongValue(numericValue);
+ writer.addDocument(doc);
+ if (random().nextInt(31) == 0) {
+ writer.commit();
+ }
+ }
+
+ // delete some docs
+ int numDeletions = random().nextInt(numDocs/10);
+ for (int i = 0; i < numDeletions; i++) {
+ int id = random().nextInt(numDocs);
+ writer.deleteDocuments(new Term("id", Integer.toString(id)));
+ }
+ writer.close();
+
+ // compare
+ final DirectoryReader ir = DirectoryReader.open(dir);
+ int numThreads = _TestUtil.nextInt(random(), 2, 7);
+ Thread threads[] = new Thread[numThreads];
+ final CountDownLatch startingGun = new CountDownLatch(1);
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread() {
+ @Override
+ public void run() {
+ try {
+ startingGun.await();
+ for (AtomicReaderContext context : ir.leaves()) {
+ AtomicReader r = context.reader();
+ BinaryDocValues binaries = r.getBinaryDocValues("dvBin");
+ SortedDocValues sorted = r.getSortedDocValues("dvSorted");
+ NumericDocValues numerics = r.getNumericDocValues("dvNum");
+ for (int j = 0; j < r.maxDoc(); j++) {
+ BytesRef binaryValue = r.document(j).getBinaryValue("storedBin");
+ BytesRef scratch = new BytesRef();
+ binaries.get(j, scratch);
+ assertEquals(binaryValue, scratch);
+ sorted.get(j, scratch);
+ assertEquals(binaryValue, scratch);
+ String expected = r.document(j).get("storedNum");
+ assertEquals(Long.parseLong(expected), numerics.get(j));
+ }
+ }
+ _TestUtil.checkReader(ir);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ };
+ threads[i].start();
+ }
+ startingGun.countDown();
+ for (Thread t : threads) {
+ t.join();
+ }
+ ir.close();
+ dir.close();
+ }
+
+ /** Tests dv against stored fields with threads (all types + missing) */
+ public void testThreads2() throws Exception {
+ assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+ assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
+ Directory dir = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+ RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+ Field idField = new StringField("id", "", Field.Store.NO);
+ Field storedBinField = new StoredField("storedBin", new byte[0]);
+ Field dvBinField = new BinaryDocValuesField("dvBin", new BytesRef());
+ Field dvSortedField = new SortedDocValuesField("dvSorted", new BytesRef());
+ Field storedNumericField = new StoredField("storedNum", "");
+ Field dvNumericField = new NumericDocValuesField("dvNum", 0);
+
+ // index some docs
+ int numDocs = atLeast(300);
+ for (int i = 0; i < numDocs; i++) {
+ idField.setStringValue(Integer.toString(i));
+ int length = _TestUtil.nextInt(random(), 0, 8);
+ byte buffer[] = new byte[length];
+ random().nextBytes(buffer);
+ storedBinField.setBytesValue(buffer);
+ dvBinField.setBytesValue(buffer);
+ dvSortedField.setBytesValue(buffer);
+ long numericValue = random().nextLong();
+ storedNumericField.setStringValue(Long.toString(numericValue));
+ dvNumericField.setLongValue(numericValue);
+ Document doc = new Document();
+ doc.add(idField);
+ if (random().nextInt(4) > 0) {
+ doc.add(storedBinField);
+ doc.add(dvBinField);
+ doc.add(dvSortedField);
+ }
+ if (random().nextInt(4) > 0) {
+ doc.add(storedNumericField);
+ doc.add(dvNumericField);
+ }
+ int numSortedSetFields = random().nextInt(3);
+ Set<String> values = new TreeSet<String>();
+ for (int j = 0; j < numSortedSetFields; j++) {
+ values.add(_TestUtil.randomSimpleString(random()));
+ }
+ for (String v : values) {
+ doc.add(new SortedSetDocValuesField("dvSortedSet", new BytesRef(v)));
+ doc.add(new StoredField("storedSortedSet", v));
+ }
+ writer.addDocument(doc);
+ if (random().nextInt(31) == 0) {
+ writer.commit();
+ }
+ }
+
+ // delete some docs
+ int numDeletions = random().nextInt(numDocs/10);
+ for (int i = 0; i < numDeletions; i++) {
+ int id = random().nextInt(numDocs);
+ writer.deleteDocuments(new Term("id", Integer.toString(id)));
+ }
+ writer.close();
+
+ // compare
+ final DirectoryReader ir = DirectoryReader.open(dir);
+ int numThreads = _TestUtil.nextInt(random(), 2, 7);
+ Thread threads[] = new Thread[numThreads];
+ final CountDownLatch startingGun = new CountDownLatch(1);
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread() {
+ @Override
+ public void run() {
+ try {
+ startingGun.await();
+ for (AtomicReaderContext context : ir.leaves()) {
+ AtomicReader r = context.reader();
+ BinaryDocValues binaries = r.getBinaryDocValues("dvBin");
+ Bits binaryBits = r.getDocsWithField("dvBin");
+ SortedDocValues sorted = r.getSortedDocValues("dvSorted");
+ Bits sortedBits = r.getDocsWithField("dvSorted");
+ NumericDocValues numerics = r.getNumericDocValues("dvNum");
+ Bits numericBits = r.getDocsWithField("dvNum");
+ SortedSetDocValues sortedSet = r.getSortedSetDocValues("dvSortedSet");
+ Bits sortedSetBits = r.getDocsWithField("dvSortedSet");
+ for (int j = 0; j < r.maxDoc(); j++) {
+ BytesRef binaryValue = r.document(j).getBinaryValue("storedBin");
+ if (binaryValue != null) {
+ if (binaries != null) {
+ BytesRef scratch = new BytesRef();
+ binaries.get(j, scratch);
+ assertEquals(binaryValue, scratch);
+ sorted.get(j, scratch);
+ assertEquals(binaryValue, scratch);
+ assertTrue(binaryBits.get(j));
+ assertTrue(sortedBits.get(j));
+ }
+ } else if (binaries != null) {
+ assertFalse(binaryBits.get(j));
+ assertFalse(sortedBits.get(j));
+ assertEquals(-1, sorted.getOrd(j));
+ }
+
+ String number = r.document(j).get("storedNum");
+ if (number != null) {
+ if (numerics != null) {
+ assertEquals(Long.parseLong(number), numerics.get(j));
+ }
+ } else if (numerics != null) {
+ assertFalse(numericBits.get(j));
+ assertEquals(0, numerics.get(j));
+ }
+
+ String values[] = r.document(j).getValues("storedSortedSet");
+ if (values.length > 0) {
+ assertNotNull(sortedSet);
+ sortedSet.setDocument(j);
+ for (int i = 0; i < values.length; i++) {
+ long ord = sortedSet.nextOrd();
+ assertTrue(ord != SortedSetDocValues.NO_MORE_ORDS);
+ BytesRef value = new BytesRef();
+ sortedSet.lookupOrd(ord, value);
+ assertEquals(values[i], value.utf8ToString());
+ }
+ assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
+ assertTrue(sortedSetBits.get(j));
+ } else if (sortedSet != null) {
+ sortedSet.setDocument(j);
+ assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
+ assertFalse(sortedSetBits.get(j));
+ }
+ }
+ }
+ _TestUtil.checkReader(ir);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ };
+ threads[i].start();
+ }
+ startingGun.countDown();
+ for (Thread t : threads) {
+ t.join();
+ }
+ ir.close();
+ dir.close();
+ }
+ // LUCENE-5218
+ public void testEmptyBinaryValueOnPageSizes() throws Exception {
+ // Test larger and larger power-of-two sized values,
+ // followed by empty string value:
+ for(int i=0;i<20;i++) {
+ if (i > 14 && codecAcceptsHugeBinaryValues("field") == false) {
+ break;
+ }
+ Directory dir = newDirectory();
+ RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+ BytesRef bytes = new BytesRef();
+ bytes.bytes = new byte[1<<i];
+ bytes.length = 1<<i;
+ for(int j=0;j<4;j++) {
+ Document doc = new Document();
+ doc.add(new BinaryDocValuesField("field", bytes));
+ w.addDocument(doc);
+ }
+ Document doc = new Document();
+ doc.add(new StoredField("id", "5"));
+ doc.add(new BinaryDocValuesField("field", new BytesRef()));
+ w.addDocument(doc);
+ IndexReader r = w.getReader();
+ w.close();
+
+ AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+ BinaryDocValues values = ar.getBinaryDocValues("field");
+ BytesRef result = new BytesRef();
+ for(int j=0;j<5;j++) {
+ values.get(0, result);
+ assertTrue(result.length == 0 || result.length == 1<<i);
+ }
+ ar.close();
+ dir.close();
+ }
+ }
+
+ protected boolean codecAcceptsHugeBinaryValues(String field) {
+ return true;
+ }
}
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Mon Oct 21 18:58:24 2013
@@ -23,6 +23,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
@@ -30,14 +31,17 @@ import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Random;
import java.util.Set;
+import java.util.SortedMap;
import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PostingsConsumer;
-import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.FieldInfo.DocValuesType;
@@ -49,7 +53,9 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util._TestUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@@ -126,6 +132,7 @@ public abstract class BasePostingsFormat
private final BytesRef payload;
private final IndexOptions options;
private final boolean doPositions;
+ private final boolean allowPayloads;
private int docID;
private int freq;
@@ -138,11 +145,12 @@ public abstract class BasePostingsFormat
private int posSpacing;
private int posUpto;
- public SeedPostings(long seed, int minDocFreq, int maxDocFreq, Bits liveDocs, IndexOptions options) {
+ public SeedPostings(long seed, int minDocFreq, int maxDocFreq, Bits liveDocs, IndexOptions options, boolean allowPayloads) {
random = new Random(seed);
docRandom = new Random(random.nextLong());
docFreq = _TestUtil.nextInt(random, minDocFreq, maxDocFreq);
this.liveDocs = liveDocs;
+ this.allowPayloads = allowPayloads;
// TODO: more realistic to inversely tie this to numDocs:
maxDocSpacing = _TestUtil.nextInt(random, 1, 100);
@@ -249,6 +257,9 @@ public abstract class BasePostingsFormat
} else {
payload.length = 0;
}
+ if (!allowPayloads) {
+ payload.length = 0;
+ }
startOffset = offset + random.nextInt(5);
endOffset = startOffset + random.nextInt(10);
@@ -295,7 +306,7 @@ public abstract class BasePostingsFormat
}
// Holds all postings:
- private static Map<String,Map<BytesRef,Long>> fields;
+ private static Map<String,SortedMap<BytesRef,Long>> fields;
private static FieldInfos fieldInfos;
@@ -307,7 +318,7 @@ public abstract class BasePostingsFormat
private static long totalPostings;
private static long totalPayloadBytes;
- private static SeedPostings getSeedPostings(String term, long seed, boolean withLiveDocs, IndexOptions options) {
+ private static SeedPostings getSeedPostings(String term, long seed, boolean withLiveDocs, IndexOptions options, boolean allowPayloads) {
int minDocFreq, maxDocFreq;
if (term.startsWith("big_")) {
minDocFreq = RANDOM_MULTIPLIER * 50000;
@@ -323,14 +334,14 @@ public abstract class BasePostingsFormat
maxDocFreq = 3;
}
- return new SeedPostings(seed, minDocFreq, maxDocFreq, withLiveDocs ? globalLiveDocs : null, options);
+ return new SeedPostings(seed, minDocFreq, maxDocFreq, withLiveDocs ? globalLiveDocs : null, options, allowPayloads);
}
@BeforeClass
public static void createPostings() throws IOException {
totalPostings = 0;
totalPayloadBytes = 0;
- fields = new TreeMap<String,Map<BytesRef,Long>>();
+ fields = new TreeMap<String,SortedMap<BytesRef,Long>>();
final int numFields = _TestUtil.nextInt(random(), 1, 5);
if (VERBOSE) {
@@ -351,7 +362,7 @@ public abstract class BasePostingsFormat
null, DocValuesType.NUMERIC, null);
fieldUpto++;
- Map<BytesRef,Long> postings = new TreeMap<BytesRef,Long>();
+ SortedMap<BytesRef,Long> postings = new TreeMap<BytesRef,Long>();
fields.put(field, postings);
Set<String> seenTerms = new HashSet<String>();
@@ -388,7 +399,7 @@ public abstract class BasePostingsFormat
// NOTE: sort of silly: we enum all the docs just to
// get the maxDoc
- DocsEnum docsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS_ONLY);
+ DocsEnum docsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS_ONLY, true);
int doc;
int lastDoc = 0;
while((doc = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
@@ -412,7 +423,7 @@ public abstract class BasePostingsFormat
}
allTerms = new ArrayList<FieldAndTerm>();
- for(Map.Entry<String,Map<BytesRef,Long>> fieldEnt : fields.entrySet()) {
+ for(Map.Entry<String,SortedMap<BytesRef,Long>> fieldEnt : fields.entrySet()) {
String field = fieldEnt.getKey();
for(Map.Entry<BytesRef,Long> termEnt : fieldEnt.getValue().entrySet()) {
allTerms.add(new FieldAndTerm(field, termEnt.getKey()));
@@ -432,6 +443,208 @@ public abstract class BasePostingsFormat
globalLiveDocs = null;
}
+ private static class SeedFields extends Fields {
+ final Map<String,SortedMap<BytesRef,Long>> fields;
+ final FieldInfos fieldInfos;
+ final IndexOptions maxAllowed;
+ final boolean allowPayloads;
+
+ public SeedFields(Map<String,SortedMap<BytesRef,Long>> fields, FieldInfos fieldInfos, IndexOptions maxAllowed, boolean allowPayloads) {
+ this.fields = fields;
+ this.fieldInfos = fieldInfos;
+ this.maxAllowed = maxAllowed;
+ this.allowPayloads = allowPayloads;
+ }
+
+ @Override
+ public Iterator<String> iterator() {
+ return fields.keySet().iterator();
+ }
+
+ @Override
+ public Terms terms(String field) {
+ SortedMap<BytesRef,Long> terms = fields.get(field);
+ if (terms == null) {
+ return null;
+ } else {
+ return new SeedTerms(terms, fieldInfos.fieldInfo(field), maxAllowed, allowPayloads);
+ }
+ }
+
+ @Override
+ public int size() {
+ return fields.size();
+ }
+ }
+
+ private static class SeedTerms extends Terms {
+ final SortedMap<BytesRef,Long> terms;
+ final FieldInfo fieldInfo;
+ final IndexOptions maxAllowed;
+ final boolean allowPayloads;
+
+ public SeedTerms(SortedMap<BytesRef,Long> terms, FieldInfo fieldInfo, IndexOptions maxAllowed, boolean allowPayloads) {
+ this.terms = terms;
+ this.fieldInfo = fieldInfo;
+ this.maxAllowed = maxAllowed;
+ this.allowPayloads = allowPayloads;
+ }
+
+ @Override
+ public TermsEnum iterator(TermsEnum reuse) {
+ SeedTermsEnum termsEnum;
+ if (reuse != null && reuse instanceof SeedTermsEnum) {
+ termsEnum = (SeedTermsEnum) reuse;
+ if (termsEnum.terms != terms) {
+ termsEnum = new SeedTermsEnum(terms, maxAllowed, allowPayloads);
+ }
+ } else {
+ termsEnum = new SeedTermsEnum(terms, maxAllowed, allowPayloads);
+ }
+ termsEnum.reset();
+
+ return termsEnum;
+ }
+
+ @Override
+ public long size() {
+ return terms.size();
+ }
+
+ @Override
+ public long getSumTotalTermFreq() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public long getSumDocFreq() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public int getDocCount() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean hasFreqs() {
+ return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+ }
+
+ @Override
+ public boolean hasOffsets() {
+ return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+ }
+
+ @Override
+ public boolean hasPositions() {
+ return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+ }
+
+ @Override
+ public boolean hasPayloads() {
+ return allowPayloads && fieldInfo.hasPayloads();
+ }
+ }
+
+ private static class SeedTermsEnum extends TermsEnum {
+ final SortedMap<BytesRef,Long> terms;
+ final IndexOptions maxAllowed;
+ final boolean allowPayloads;
+
+ private Iterator<Map.Entry<BytesRef,Long>> iterator;
+
+ private Map.Entry<BytesRef,Long> current;
+
+ public SeedTermsEnum(SortedMap<BytesRef,Long> terms, IndexOptions maxAllowed, boolean allowPayloads) {
+ this.terms = terms;
+ this.maxAllowed = maxAllowed;
+ this.allowPayloads = allowPayloads;
+ }
+
+ void reset() {
+ iterator = terms.entrySet().iterator();
+ }
+
+ @Override
+ public SeekStatus seekCeil(BytesRef text) {
+ SortedMap<BytesRef,Long> tailMap = terms.tailMap(text);
+ if (tailMap.isEmpty()) {
+ return SeekStatus.END;
+ } else {
+ iterator = tailMap.entrySet().iterator();
+ if (tailMap.firstKey().equals(text)) {
+ return SeekStatus.FOUND;
+ } else {
+ return SeekStatus.NOT_FOUND;
+ }
+ }
+ }
+
+ @Override
+ public BytesRef next() {
+ if (iterator.hasNext()) {
+ current = iterator.next();
+ return term();
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public void seekExact(long ord) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public BytesRef term() {
+ return current.getKey();
+ }
+
+ @Override
+ public long ord() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public int docFreq() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public long totalTermFreq() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public final DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+ if (liveDocs != null) {
+ throw new IllegalArgumentException("liveDocs must be null");
+ }
+ if ((flags & DocsEnum.FLAG_FREQS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS) < 0) {
+ return null;
+ }
+ return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads);
+ }
+
+ @Override
+ public final DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+ if (liveDocs != null) {
+ throw new IllegalArgumentException("liveDocs must be null");
+ }
+ if (maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
+ return null;
+ }
+ if ((flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
+ return null;
+ }
+ if ((flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0 && allowPayloads == false) {
+ return null;
+ }
+ return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads);
+ }
+ }
+
// TODO maybe instead of @BeforeClass just make a single test run: build postings & index & test it?
private FieldInfos currentFieldInfos;
@@ -440,7 +653,7 @@ public abstract class BasePostingsFormat
// randomly index at lower IndexOption
private FieldsProducer buildIndex(Directory dir, IndexOptions maxAllowed, boolean allowPayloads, boolean alwaysTestMax) throws IOException {
Codec codec = getCodec();
- SegmentInfo segmentInfo = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, "_0", maxDoc, false, codec, null, null);
+ SegmentInfo segmentInfo = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, "_0", maxDoc, false, codec, null);
int maxIndexOption = Arrays.asList(IndexOptions.values()).indexOf(maxAllowed);
if (VERBOSE) {
@@ -489,79 +702,10 @@ public abstract class BasePostingsFormat
SegmentWriteState writeState = new SegmentWriteState(null, dir,
segmentInfo, newFieldInfos,
null, new IOContext(new FlushInfo(maxDoc, bytes)));
- FieldsConsumer fieldsConsumer = codec.postingsFormat().fieldsConsumer(writeState);
-
- for(Map.Entry<String,Map<BytesRef,Long>> fieldEnt : fields.entrySet()) {
- String field = fieldEnt.getKey();
- Map<BytesRef,Long> terms = fieldEnt.getValue();
-
- FieldInfo fieldInfo = newFieldInfos.fieldInfo(field);
-
- IndexOptions indexOptions = fieldInfo.getIndexOptions();
- if (VERBOSE) {
- System.out.println("field=" + field + " indexOtions=" + indexOptions);
- }
+ Fields seedFields = new SeedFields(fields, newFieldInfos, maxAllowed, allowPayloads);
- boolean doFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
- boolean doPos = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
- boolean doPayloads = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 && allowPayloads;
- boolean doOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
-
- TermsConsumer termsConsumer = fieldsConsumer.addField(fieldInfo);
- long sumTotalTF = 0;
- long sumDF = 0;
- FixedBitSet seenDocs = new FixedBitSet(maxDoc);
- for(Map.Entry<BytesRef,Long> termEnt : terms.entrySet()) {
- BytesRef term = termEnt.getKey();
- SeedPostings postings = getSeedPostings(term.utf8ToString(), termEnt.getValue(), false, maxAllowed);
- if (VERBOSE) {
- System.out.println(" term=" + field + ":" + term.utf8ToString() + " docFreq=" + postings.docFreq + " seed=" + termEnt.getValue());
- }
-
- PostingsConsumer postingsConsumer = termsConsumer.startTerm(term);
- long totalTF = 0;
- int docID = 0;
- while((docID = postings.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
- final int freq = postings.freq();
- if (VERBOSE) {
- System.out.println(" " + postings.upto + ": docID=" + docID + " freq=" + postings.freq);
- }
- postingsConsumer.startDoc(docID, doFreq ? postings.freq : -1);
- seenDocs.set(docID);
- if (doPos) {
- totalTF += postings.freq;
- for(int posUpto=0;posUpto<freq;posUpto++) {
- int pos = postings.nextPosition();
- BytesRef payload = postings.getPayload();
-
- if (VERBOSE) {
- if (doPayloads) {
- System.out.println(" pos=" + pos + " payload=" + (payload == null ? "null" : payload.length + " bytes"));
- } else {
- System.out.println(" pos=" + pos);
- }
- }
- postingsConsumer.addPosition(pos, doPayloads ? payload : null,
- doOffsets ? postings.startOffset() : -1,
- doOffsets ? postings.endOffset() : -1);
- }
- } else if (doFreq) {
- totalTF += freq;
- } else {
- totalTF++;
- }
- postingsConsumer.finishDoc();
- }
- termsConsumer.finishTerm(term, new TermStats(postings.docFreq, doFreq ? totalTF : -1));
- sumTotalTF += totalTF;
- sumDF += postings.docFreq;
- }
-
- termsConsumer.finish(doFreq ? sumTotalTF : -1, sumDF, seenDocs.cardinality());
- }
-
- fieldsConsumer.close();
+ codec.postingsFormat().fieldsConsumer(writeState).write(seedFields);
if (VERBOSE) {
System.out.println("TEST: after indexing: files=");
@@ -572,7 +716,7 @@ public abstract class BasePostingsFormat
currentFieldInfos = newFieldInfos;
- SegmentReadState readState = new SegmentReadState(dir, segmentInfo, newFieldInfos, IOContext.DEFAULT);
+ SegmentReadState readState = new SegmentReadState(dir, segmentInfo, newFieldInfos, IOContext.READ);
return codec.postingsFormat().fieldsProducer(readState);
}
@@ -625,7 +769,8 @@ public abstract class BasePostingsFormat
SeedPostings expected = getSeedPostings(term.utf8ToString(),
fields.get(field).get(term),
useLiveDocs,
- maxIndexOptions);
+ maxIndexOptions,
+ true);
assertEquals(expected.docFreq, termsEnum.docFreq());
boolean allowFreqs = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0 &&
@@ -1224,4 +1369,197 @@ public abstract class BasePostingsFormat
iw.close();
dir.close();
}
+
+ private static class TermFreqs {
+ long totalTermFreq;
+ int docFreq;
+ };
+
+ // LUCENE-5123: make sure we can visit postings twice
+ // during flush/merge
+ public void testInvertedWrite() throws Exception {
+ Directory dir = newDirectory();
+ IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+
+ final Map<String,TermFreqs> termFreqs = new HashMap<String,TermFreqs>();
+ final AtomicLong sumDocFreq = new AtomicLong();
+ final AtomicLong sumTotalTermFreq = new AtomicLong();
+
+ // TODO: would be better to use / delegate to the current
+ // Codec returned by getCodec()
+
+ iwc.setCodec(new Lucene46Codec() {
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+
+ PostingsFormat p = getCodec().postingsFormat();
+ if (p instanceof PerFieldPostingsFormat) {
+ p = ((PerFieldPostingsFormat) p).getPostingsFormatForField(field);
+ }
+ final PostingsFormat defaultPostingsFormat = p;
+
+ if (field.equals("body")) {
+
+ // A PF that counts up some stats and then in
+ // the end we verify the stats match what the
+ // final IndexReader says, just to exercise the
+ // new freedom of iterating the postings more
+ // than once at flush/merge:
+
+ return new PostingsFormat(defaultPostingsFormat.getName()) {
+
+ @Override
+ public FieldsConsumer fieldsConsumer(final SegmentWriteState state) throws IOException {
+
+ final FieldsConsumer fieldsConsumer = defaultPostingsFormat.fieldsConsumer(state);
+
+ return new FieldsConsumer() {
+ @Override
+ public void write(Fields fields) throws IOException {
+ fieldsConsumer.write(fields);
+
+ boolean isMerge = state.context.context == IOContext.Context.MERGE;
+
+ boolean addOnSecondPass = random().nextBoolean();
+
+ //System.out.println("write isMerge=" + isMerge + " 2ndPass=" + addOnSecondPass);
+
+ // Gather our own stats:
+ Terms terms = fields.terms("body");
+ assert terms != null;
+
+ TermsEnum termsEnum = terms.iterator(null);
+ DocsEnum docs = null;
+ while(termsEnum.next() != null) {
+ BytesRef term = termsEnum.term();
+ if (random().nextBoolean()) {
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_FREQS);
+ } else if (docs instanceof DocsAndPositionsEnum) {
+ docs = termsEnum.docsAndPositions(null, (DocsAndPositionsEnum) docs, 0);
+ } else {
+ docs = termsEnum.docsAndPositions(null, null, 0);
+ }
+ int docFreq = 0;
+ long totalTermFreq = 0;
+ while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+ docFreq++;
+ totalTermFreq += docs.freq();
+ if (docs instanceof DocsAndPositionsEnum) {
+ DocsAndPositionsEnum posEnum = (DocsAndPositionsEnum) docs;
+ int limit = _TestUtil.nextInt(random(), 1, docs.freq());
+ for(int i=0;i<limit;i++) {
+ posEnum.nextPosition();
+ }
+ }
+ }
+
+ String termString = term.utf8ToString();
+
+ // During merge we should only see terms
+ // we had already seen during flush:
+ assertTrue(isMerge==false || termFreqs.containsKey(termString));
+
+ if (isMerge == false && addOnSecondPass == false) {
+ TermFreqs tf = termFreqs.get(termString);
+ if (tf == null) {
+ tf = new TermFreqs();
+ termFreqs.put(termString, tf);
+ }
+ tf.docFreq += docFreq;
+ tf.totalTermFreq += totalTermFreq;
+ sumDocFreq.addAndGet(docFreq);
+ sumTotalTermFreq.addAndGet(totalTermFreq);
+ } else if (termFreqs.containsKey(termString) == false) {
+ termFreqs.put(termString, new TermFreqs());
+ }
+ }
+
+ // Also test seeking the TermsEnum:
+ for(String term : termFreqs.keySet()) {
+ if (termsEnum.seekExact(new BytesRef(term))) {
+ if (random().nextBoolean()) {
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_FREQS);
+ } else if (docs instanceof DocsAndPositionsEnum) {
+ docs = termsEnum.docsAndPositions(null, (DocsAndPositionsEnum) docs, 0);
+ } else {
+ docs = termsEnum.docsAndPositions(null, null, 0);
+ }
+
+ int docFreq = 0;
+ long totalTermFreq = 0;
+ while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+ docFreq++;
+ totalTermFreq += docs.freq();
+ if (docs instanceof DocsAndPositionsEnum) {
+ DocsAndPositionsEnum posEnum = (DocsAndPositionsEnum) docs;
+ int limit = _TestUtil.nextInt(random(), 1, docs.freq());
+ for(int i=0;i<limit;i++) {
+ posEnum.nextPosition();
+ }
+ }
+ }
+
+ if (isMerge == false && addOnSecondPass) {
+ TermFreqs tf = termFreqs.get(term);
+ if (tf == null) {
+ tf = new TermFreqs();
+ termFreqs.put(term, tf);
+ }
+ tf.docFreq += docFreq;
+ tf.totalTermFreq += totalTermFreq;
+ sumDocFreq.addAndGet(docFreq);
+ sumTotalTermFreq.addAndGet(totalTermFreq);
+ }
+
+ //System.out.println(" term=" + term + " docFreq=" + docFreq + " ttDF=" + termToDocFreq.get(term));
+ assertTrue(docFreq <= termFreqs.get(term).docFreq);
+ assertTrue(totalTermFreq <= termFreqs.get(term).totalTermFreq);
+ }
+ }
+ }
+ };
+ }
+
+ @Override
+ public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+ return defaultPostingsFormat.fieldsProducer(state);
+ }
+ };
+ } else {
+ return defaultPostingsFormat;
+ }
+ }
+ });
+
+ RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+
+ LineFileDocs docs = new LineFileDocs(random());
+ int bytesToIndex = atLeast(100) * 1024;
+ int bytesIndexed = 0;
+ while (bytesIndexed < bytesToIndex) {
+ Document doc = docs.nextDoc();
+ w.addDocument(doc);
+ bytesIndexed += RamUsageEstimator.sizeOf(doc);
+ }
+
+ IndexReader r = w.getReader();
+ w.close();
+
+ Terms terms = MultiFields.getTerms(r, "body");
+ assertEquals(sumDocFreq.get(), terms.getSumDocFreq());
+ assertEquals(sumTotalTermFreq.get(), terms.getSumTotalTermFreq());
+
+ TermsEnum termsEnum = terms.iterator(null);
+ long termCount = 0;
+ while(termsEnum.next() != null) {
+ BytesRef term = termsEnum.term();
+ termCount++;
+ assertEquals(termFreqs.get(term.utf8ToString()).docFreq, termsEnum.docFreq());
+ assertEquals(termFreqs.get(term.utf8ToString()).totalTermFreq, termsEnum.totalTermFreq());
+ }
+ assertEquals(termFreqs.size(), termCount);
+
+ r.close();
+ dir.close();
+ }
}
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java Mon Oct 21 18:58:24 2013
@@ -31,21 +31,20 @@ import java.util.concurrent.atomic.Atomi
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.compressing.CompressingCodec;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoubleField;
import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FieldType.NumericType;
import org.apache.lucene.document.FloatField;
import org.apache.lucene.document.IntField;
import org.apache.lucene.document.LongField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.FieldType.NumericType;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.NumericRangeQuery;
@@ -59,7 +58,6 @@ import org.apache.lucene.store.MockDirec
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
-import org.apache.lucene.util.LuceneTestCase.Nightly;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
@@ -78,6 +76,7 @@ public abstract class BaseStoredFieldsFo
*/
protected abstract Codec getCodec();
+ @Override
public void setUp() throws Exception {
super.setUp();
// set the default codec, so adding test cases to this isn't fragile
@@ -85,6 +84,7 @@ public abstract class BaseStoredFieldsFo
Codec.setDefault(getCodec());
}
+ @Override
public void tearDown() throws Exception {
Codec.setDefault(savedCodec); // restore
super.tearDown();
@@ -502,7 +502,7 @@ public abstract class BaseStoredFieldsFo
// get another codec, other than the default: so we are merging segments across different codecs
final Codec otherCodec;
if ("SimpleText".equals(Codec.getDefault().getName())) {
- otherCodec = new Lucene42Codec();
+ otherCodec = new Lucene46Codec();
} else {
otherCodec = new SimpleTextCodec();
}
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java Mon Oct 21 18:58:24 2013
@@ -406,7 +406,7 @@ public abstract class BaseTermVectorsFor
uniqueTerms.add(new BytesRef(term));
}
final BytesRef[] sortedTerms = uniqueTerms.toArray(new BytesRef[0]);
- Arrays.sort(sortedTerms, terms.getComparator());
+ Arrays.sort(sortedTerms);
final TermsEnum termsEnum = terms.iterator(random().nextBoolean() ? null : this.termsEnum.get());
this.termsEnum.set(termsEnum);
for (int i = 0; i < sortedTerms.length; ++i) {
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java Mon Oct 21 18:58:24 2013
@@ -22,6 +22,7 @@ import java.util.ArrayList;
import java.util.Iterator;
import java.util.Set;
+import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FilterIterator;
/**
@@ -137,6 +138,11 @@ public final class FieldFilterAtomicRead
}
@Override
+ public Bits getDocsWithField(String field) throws IOException {
+ return hasField(field) ? super.getDocsWithField(field) : null;
+ }
+
+ @Override
public String toString() {
final StringBuilder sb = new StringBuilder("FieldFilterAtomicReader(reader=");
sb.append(in).append(", fields=");
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Mon Oct 21 18:58:24 2013
@@ -27,20 +27,24 @@ import java.util.Map;
import java.util.Random;
import java.util.Set;
-import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
+import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
-import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
-import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
-import org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat;
-import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat;
+import org.apache.lucene.codecs.memory.FSTPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat;
+import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
@@ -48,8 +52,8 @@ import org.apache.lucene.codecs.mockrand
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
-import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat;
+import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
@@ -62,7 +66,7 @@ import org.apache.lucene.util._TestUtil;
* documents in different orders and the test will still be deterministic
* and reproducable.
*/
-public class RandomCodec extends Lucene42Codec {
+public class RandomCodec extends Lucene46Codec {
/** Shuffled list of postings formats to use for new mappings */
private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();
@@ -125,6 +129,10 @@ public class RandomCodec extends Lucene4
add(avoidCodecs,
new Lucene41PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
+ new FSTPostingsFormat(),
+ new FSTOrdPostingsFormat(),
+ new FSTPulsing41PostingsFormat(1 + random.nextInt(20)),
+ new FSTOrdPulsing41PostingsFormat(1 + random.nextInt(20)),
new DirectPostingsFormat(LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : maxItemsPerBlock),
LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : lowFreqCutoff)),
new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
@@ -148,11 +156,11 @@ public class RandomCodec extends Lucene4
new MemoryPostingsFormat(false, random.nextFloat()));
addDocValues(avoidCodecs,
- new Lucene42DocValuesFormat(),
+ new Lucene45DocValuesFormat(),
new DiskDocValuesFormat(),
+ new MemoryDocValuesFormat(),
new SimpleTextDocValuesFormat(),
- new AssertingDocValuesFormat(),
- new CheapBastardDocValuesFormat());
+ new AssertingDocValuesFormat());
Collections.shuffle(formats, random);
Collections.shuffle(dvFormats, random);
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java Mon Oct 21 18:58:24 2013
@@ -221,6 +221,10 @@ public class RandomIndexWriter implement
w.addIndexes(readers);
}
+ public void updateNumericDocValue(Term term, String field, Long value) throws IOException {
+ w.updateNumericDocValue(term, field, value);
+ }
+
public void deleteDocuments(Term term) throws IOException {
w.deleteDocuments(term);
}
@@ -340,7 +344,7 @@ public class RandomIndexWriter implement
w.forceMerge(maxSegmentCount);
}
- private static final class TestPointInfoStream extends InfoStream {
+ static final class TestPointInfoStream extends InfoStream {
private final InfoStream delegate;
private final TestPoint testPoint;
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Mon Oct 21 18:58:24 2013
@@ -45,6 +45,7 @@ import org.apache.lucene.util.FailOnNonB
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.NamedThreadFactory;
+import org.apache.lucene.util.PrintStreamInfoStream;
import org.apache.lucene.util._TestUtil;
// TODO
@@ -327,6 +328,9 @@ public abstract class ThreadedIndexingAn
searchThreads[thread] = new Thread() {
@Override
public void run() {
+ if (VERBOSE) {
+ System.out.println(Thread.currentThread().getName() + ": launch search thread");
+ }
while (System.currentTimeMillis() < stopTimeMS) {
try {
final IndexSearcher s = getCurrentSearcher();
@@ -478,6 +482,17 @@ public abstract class ThreadedIndexingAn
}
});
+ if (VERBOSE) {
+ conf.setInfoStream(new PrintStreamInfoStream(System.out) {
+ @Override
+ public void message(String component, String message) {
+ if ("TP".equals(component)) {
+ return; // ignore test points!
+ }
+ super.message(component, message);
+ }
+ });
+ }
writer = new IndexWriter(dir, conf);
_TestUtil.reduceOpenFiles(writer);
@@ -498,7 +513,7 @@ public abstract class ThreadedIndexingAn
final Thread[] indexThreads = launchIndexingThreads(docs, NUM_INDEX_THREADS, stopTime, delIDs, delPackIDs, allSubDocs);
if (VERBOSE) {
- System.out.println("TEST: DONE start indexing threads [" + (System.currentTimeMillis()-t0) + " ms]");
+ System.out.println("TEST: DONE start " + NUM_INDEX_THREADS + " indexing threads [" + (System.currentTimeMillis()-t0) + " ms]");
}
// Let index build up a bit
@@ -599,7 +614,7 @@ public abstract class ThreadedIndexingAn
if (!delIDs.contains(stringID)) {
final TopDocs hits = s.search(new TermQuery(new Term("docid", stringID)), 1);
if (hits.totalHits != 1) {
- System.out.println("doc id=" + stringID + " is not supposed to be deleted, but got hitCount=" + hits.totalHits);
+ System.out.println("doc id=" + stringID + " is not supposed to be deleted, but got hitCount=" + hits.totalHits + "; delIDs=" + delIDs);
doFail = true;
}
}
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java Mon Oct 21 18:58:24 2013
@@ -25,6 +25,7 @@ import junit.framework.Assert;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
+import org.apache.lucene.index.AllDeletedFilterReader;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
@@ -38,7 +39,6 @@ import org.apache.lucene.store.MockDirec
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
@@ -132,7 +132,7 @@ public class QueryUtils {
public static void purgeFieldCache(IndexReader r) throws IOException {
// this is just a hack, to get an atomic reader that contains all subreaders for insanity checks
- FieldCache.DEFAULT.purge(SlowCompositeReaderWrapper.wrap(r));
+ FieldCache.DEFAULT.purgeByCacheKey(SlowCompositeReaderWrapper.wrap(r).getCoreCacheKey());
}
/** This is a MultiReader that can be used for randomly wrapping other readers
@@ -193,7 +193,7 @@ public class QueryUtils {
static final IndexReader[] emptyReaders = new IndexReader[8];
static {
try {
- emptyReaders[0] = makeEmptyIndex(new Random(0), 0);
+ emptyReaders[0] = new MultiReader();
emptyReaders[4] = makeEmptyIndex(new Random(0), 4);
emptyReaders[5] = makeEmptyIndex(new Random(0), 5);
emptyReaders[7] = makeEmptyIndex(new Random(0), 7);
@@ -202,31 +202,18 @@ public class QueryUtils {
}
}
- private static DirectoryReader makeEmptyIndex(Random random, final int numDeletedDocs)
- throws IOException {
+ private static IndexReader makeEmptyIndex(Random random, final int numDocs) throws IOException {
+ assert numDocs > 0;
Directory d = new MockDirectoryWrapper(random, new RAMDirectory());
- IndexWriter w = new IndexWriter(d, new IndexWriterConfig(
- TEST_VERSION_CURRENT, new MockAnalyzer(random)));
- for (int i = 0; i < numDeletedDocs; i++) {
- w.addDocument(new Document());
- }
- w.commit();
- w.deleteDocuments( new MatchAllDocsQuery() );
- _TestUtil.keepFullyDeletedSegments(w);
- w.commit();
-
- if (0 < numDeletedDocs)
- Assert.assertTrue("writer has no deletions", w.hasDeletions());
-
- Assert.assertEquals("writer is missing some deleted docs",
- numDeletedDocs, w.maxDoc());
- Assert.assertEquals("writer has non-deleted docs",
- 0, w.numDocs());
- w.close();
- DirectoryReader r = DirectoryReader.open(d);
- Assert.assertEquals("reader has wrong number of deleted docs",
- numDeletedDocs, r.numDeletedDocs());
- return r;
+ IndexWriter w = new IndexWriter(d, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+ for (int i = 0; i < numDocs; i++) {
+ w.addDocument(new Document());
+ }
+ w.forceMerge(1);
+ w.commit();
+ w.close();
+ DirectoryReader reader = DirectoryReader.open(d);
+ return new AllDeletedFilterReader(LuceneTestCase.getOnlySegmentReader(reader));
}
/** alternate scorer skipTo(),skipTo(),next(),next(),skipTo(),skipTo(), etc
Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.store;
*/
import java.io.IOException;
-import java.util.Collection;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.util._TestUtil;
@@ -29,15 +28,14 @@ import org.apache.lucene.util._TestUtil;
// do NOT make any methods in this class synchronized, volatile
// do NOT import anything from the concurrency package.
// no randoms, no nothing.
-public class BaseDirectoryWrapper extends Directory {
- /** our in directory */
- protected final Directory delegate;
+public class BaseDirectoryWrapper extends FilterDirectory {
private boolean checkIndexOnClose = true;
private boolean crossCheckTermVectorsOnClose = true;
+ protected volatile boolean isOpen = true;
public BaseDirectoryWrapper(Directory delegate) {
- this.delegate = delegate;
+ super(delegate);
}
@Override
@@ -46,7 +44,7 @@ public class BaseDirectoryWrapper extend
if (checkIndexOnClose && DirectoryReader.indexExists(this)) {
_TestUtil.checkIndex(this, crossCheckTermVectorsOnClose);
}
- delegate.close();
+ super.close();
}
public boolean isOpen() {
@@ -73,80 +71,13 @@ public class BaseDirectoryWrapper extend
return crossCheckTermVectorsOnClose;
}
- // directory methods: delegate
-
- @Override
- public String[] listAll() throws IOException {
- return delegate.listAll();
- }
-
- @Override
- public boolean fileExists(String name) throws IOException {
- return delegate.fileExists(name);
- }
-
- @Override
- public void deleteFile(String name) throws IOException {
- delegate.deleteFile(name);
- }
-
- @Override
- public long fileLength(String name) throws IOException {
- return delegate.fileLength(name);
- }
-
- @Override
- public IndexOutput createOutput(String name, IOContext context) throws IOException {
- return delegate.createOutput(name, context);
- }
-
- @Override
- public void sync(Collection<String> names) throws IOException {
- delegate.sync(names);
- }
-
- @Override
- public IndexInput openInput(String name, IOContext context) throws IOException {
- return delegate.openInput(name, context);
- }
-
- @Override
- public Lock makeLock(String name) {
- return delegate.makeLock(name);
- }
-
- @Override
- public void clearLock(String name) throws IOException {
- delegate.clearLock(name);
- }
-
- @Override
- public void setLockFactory(LockFactory lockFactory) throws IOException {
- delegate.setLockFactory(lockFactory);
- }
-
- @Override
- public LockFactory getLockFactory() {
- return delegate.getLockFactory();
- }
-
- @Override
- public String getLockID() {
- return delegate.getLockID();
- }
-
- @Override
- public String toString() {
- return "BaseDirectoryWrapper(" + delegate.toString() + ")";
- }
-
@Override
public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
- delegate.copy(to, src, dest, context);
+ in.copy(to, src, dest, context);
}
@Override
public IndexInputSlicer createSlicer(String name, IOContext context) throws IOException {
- return delegate.createSlicer(name, context);
+ return in.createSlicer(name, context);
}
}