Modified: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java?rev=967130&r1=967129&r2=967130&view=diff
==============================================================================
--- lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java
(original)
+++ lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java
Fri Jul 23 15:26:25 2010
@@ -18,8 +18,11 @@ package org.apache.lucene.index.codecs.p
*/
import org.apache.lucene.store.*;
+import org.apache.lucene.document.*;
+import org.apache.lucene.analysis.*;
import org.apache.lucene.index.*;
import org.apache.lucene.index.codecs.*;
+import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
import org.apache.lucene.util.*;
import java.util.*;
@@ -30,8 +33,6 @@ import org.junit.Test;
public class TestSurrogates extends LuceneTestCaseJ4 {
- // chooses from a very limited alphabet to exacerbate the
- // surrogate seeking required
private static String makeDifficultRandomUnicodeString(Random r) {
final int end = r.nextInt(20);
if (end == 0) {
@@ -44,154 +45,295 @@ public class TestSurrogates extends Luce
if (0 == t && i < end - 1) {
// hi
- buffer[i++] = (char) 0xd800;
+ buffer[i++] = (char) (0xd800 + r.nextInt(2));
// lo
- buffer[i] = (char) 0xdc00;
+ buffer[i] = (char) (0xdc00 + r.nextInt(2));
} else if (t <= 3) {
- buffer[i] = 'a';
+ buffer[i] = (char) ('a' + r.nextInt(2));
} else if (4 == t) {
- buffer[i] = 0xe000;
+ buffer[i] = (char) (0xe000 + r.nextInt(2));
}
}
return new String(buffer, 0, end);
}
- private SegmentInfo makePreFlexSegment(Random r, String segName, Directory dir, FieldInfos
fieldInfos, Codec codec, List<Term> fieldTerms) throws IOException {
+ private String toHexString(Term t) {
+ return t.field() + ":" + UnicodeUtil.toHexString(t.text());
+ }
- final int numField = _TestUtil.nextInt(r, 2, 5);
+ private String getRandomString(Random r) {
+ String s;
+ if (r.nextInt(3) == 1) {
+ s = makeDifficultRandomUnicodeString(r);
+ } else {
+ s = _TestUtil.randomUnicodeString(r);
+ }
+ return s;
+ }
- List<Term> terms = new ArrayList<Term>();
+ private static class SortTermAsUTF16Comparator implements Comparator<Term> {
+ public int compare(Term o1, Term o2) {
+ return o1.compareToUTF16(o2);
+ }
+ }
- int tc = 0;
+ private static final SortTermAsUTF16Comparator termAsUTF16Comparator = new SortTermAsUTF16Comparator();
- for(int f=0;f<numField;f++) {
- String field = "f" + f;
- Term protoTerm = new Term(field);
+ // single straight enum
+ private void doTestStraightEnum(List<Term> fieldTerms, IndexReader reader, int uniqueTermCount)
throws IOException {
- fieldInfos.add(field, true, false, false, false, false, false, false);
- final int numTerms = 10000*_TestUtil.getRandomMultiplier();
- for(int i=0;i<numTerms;i++) {
- String s;
- if (r.nextInt(3) == 1) {
- s = makeDifficultRandomUnicodeString(r);
- } else {
- s = _TestUtil.randomUnicodeString(r);
+ if (VERBOSE) {
+ System.out.println("\nTEST: top now enum reader=" + reader);
+ }
+ FieldsEnum fieldsEnum = MultiFields.getFields(reader).iterator();
- // The surrogate dance uses 0xffff to seek-to-end
- // of blocks. Also, pre-4.0 indices are already
- // guaranteed to not contain the char 0xffff since
- // it's mapped during indexing:
- s = s.replace((char) 0xffff, (char) 0xfffe);
+ {
+ // Test straight enum:
+ String field;
+ int termCount = 0;
+ while((field = fieldsEnum.next()) != null) {
+ TermsEnum termsEnum = fieldsEnum.terms();
+ BytesRef text;
+ BytesRef lastText = null;
+ while((text = termsEnum.next()) != null) {
+ Term exp = fieldTerms.get(termCount);
+ if (VERBOSE) {
+ System.out.println(" got term=" + field + ":" + UnicodeUtil.toHexString(text.utf8ToString()));
+ System.out.println(" exp=" + exp.field() + ":" + UnicodeUtil.toHexString(exp.text().toString()));
+ System.out.println();
+ }
+ if (lastText == null) {
+ lastText = new BytesRef(text);
+ } else {
+ assertTrue(lastText.compareTo(text) < 0);
+ lastText.copy(text);
+ }
+ assertEquals(exp.field(), field);
+ assertEquals(exp.bytes(), text);
+ termCount++;
+ }
+ if (VERBOSE) {
+ System.out.println(" no more terms for field=" + field);
}
- terms.add(protoTerm.createTerm(s + "_" + (tc++)));
}
+ assertEquals(uniqueTermCount, termCount);
}
+ }
- fieldInfos.write(dir, segName);
+ // randomly seeks to term that we know exists, then next's
+ // from there
+ private void doTestSeekExists(Random r, List<Term> fieldTerms, IndexReader reader)
throws IOException {
- // sorts in UTF16 order, just like preflex:
- Collections.sort(terms, new Comparator<Term>() {
- public int compare(Term o1, Term o2) {
- return o1.compareToUTF16(o2);
- }
- });
+ final Map<String,TermsEnum> tes = new HashMap<String,TermsEnum>();
- TermInfosWriter w = new TermInfosWriter(dir, segName, fieldInfos, 128);
- TermInfo ti = new TermInfo();
- String lastText = null;
- int uniqueTermCount = 0;
+ // Test random seek to existing term, then enum:
if (VERBOSE) {
- System.out.println("TEST: utf16 order:");
+ System.out.println("\nTEST: top now seek");
}
- for(Term t : terms) {
- FieldInfo fi = fieldInfos.fieldInfo(t.field());
- String text = t.text();
- if (lastText != null && lastText.equals(text)) {
- continue;
- }
- fieldTerms.add(t);
- uniqueTermCount++;
- lastText = text;
+ for(int iter=0;iter<100*_TestUtil.getRandomMultiplier();iter++) {
+
+ // pick random field+term
+ int spot = r.nextInt(fieldTerms.size());
+ Term term = fieldTerms.get(spot);
+ String field = term.field();
if (VERBOSE) {
- System.out.println(" " + toHexString(t));
+ System.out.println("TEST: exist seek field=" + field + " term=" + UnicodeUtil.toHexString(term.text()));
+ }
+
+ // seek to it
+ TermsEnum te = tes.get(field);
+ if (te == null) {
+ te = MultiFields.getTerms(reader, field).iterator();
+ tes.put(field, te);
+ }
+
+ if (VERBOSE) {
+ System.out.println(" done get enum");
+ }
+
+ // seek should find the term
+ assertEquals(TermsEnum.SeekStatus.FOUND,
+ te.seek(term.bytes()));
+
+ // now .next() this many times:
+ int ct = _TestUtil.nextInt(r, 5, 100);
+ for(int i=0;i<ct;i++) {
+ if (VERBOSE) {
+ System.out.println("TEST: now next()");
+ }
+ if (1+spot+i >= fieldTerms.size()) {
+ break;
+ }
+ term = fieldTerms.get(1+spot+i);
+ if (term.field() != field) {
+ assertNull(te.next());
+ break;
+ } else {
+ BytesRef t = te.next();
+
+ if (VERBOSE) {
+ System.out.println(" got term=" + (t == null ? null : UnicodeUtil.toHexString(t.utf8ToString())));
+ System.out.println(" exp=" + UnicodeUtil.toHexString(term.text().toString()));
+ }
+
+ assertEquals(term.bytes(), t);
+ }
}
- w.add(fi.number, t.bytes().bytes, t.bytes().length, ti);
}
- w.close();
+ }
+
+ private void doTestSeekDoesNotExist(Random r, int numField, List<Term> fieldTerms,
Term[] fieldTermsArray, IndexReader reader) throws IOException {
+
+ final Map<String,TermsEnum> tes = new HashMap<String,TermsEnum>();
- Collections.sort(fieldTerms);
if (VERBOSE) {
- System.out.println("\nTEST: codepoint order");
- for(Term t: fieldTerms) {
- System.out.println(" " + t.field() + ":" + toHexString(t));
- }
+ System.out.println("TEST: top random seeks");
}
- dir.createOutput(segName + ".prx").close();
- dir.createOutput(segName + ".frq").close();
+ {
+ for(int iter=0;iter<100*_TestUtil.getRandomMultiplier();iter++) {
+
+ // seek to random spot
+ String field = ("f" + r.nextInt(numField)).intern();
+ Term tx = new Term(field, getRandomString(r));
+
+ int spot = Arrays.binarySearch(fieldTermsArray, tx);
+
+ if (spot < 0) {
+ if (VERBOSE) {
+ System.out.println("TEST: non-exist seek to " + field + ":" + UnicodeUtil.toHexString(tx.text()));
+ }
+
+ // term does not exist:
+ TermsEnum te = tes.get(field);
+ if (te == null) {
+ te = MultiFields.getTerms(reader, field).iterator();
+ tes.put(field, te);
+ }
+
+ if (VERBOSE) {
+ System.out.println(" got enum");
+ }
+
+ spot = -spot - 1;
+
+ if (spot == fieldTerms.size() || fieldTerms.get(spot).field() != field) {
+ assertEquals(TermsEnum.SeekStatus.END, te.seek(tx.bytes()));
+ } else {
+ assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seek(tx.bytes()));
+
+ if (VERBOSE) {
+ System.out.println(" got term=" + UnicodeUtil.toHexString(te.term().utf8ToString()));
+ System.out.println(" exp term=" + UnicodeUtil.toHexString(fieldTerms.get(spot).text()));
+ }
+
+ assertEquals(fieldTerms.get(spot).bytes(),
+ te.term());
+
+ // now .next() this many times:
+ int ct = _TestUtil.nextInt(r, 5, 100);
+ for(int i=0;i<ct;i++) {
+ if (VERBOSE) {
+ System.out.println("TEST: now next()");
+ }
+ if (1+spot+i >= fieldTerms.size()) {
+ break;
+ }
+ Term term = fieldTerms.get(1+spot+i);
+ if (term.field() != field) {
+ assertNull(te.next());
+ break;
+ } else {
+ BytesRef t = te.next();
+
+ if (VERBOSE) {
+ System.out.println(" got term=" + (t == null ? null : UnicodeUtil.toHexString(t.utf8ToString())));
+ System.out.println(" exp=" + UnicodeUtil.toHexString(term.text().toString()));
+ }
+
+ assertEquals(term.bytes(), t);
+ }
+ }
- // !!hack alert!! stuffing uniqueTermCount in as docCount
- return new SegmentInfo(segName, uniqueTermCount, dir, false, -1, null, false, true, codec);
+ }
+ }
+ }
+ }
}
- private String toHexString(Term t) {
- return t.field() + ":" + UnicodeUtil.toHexString(t.text());
- }
-
+
@Test
public void testSurrogatesOrder() throws Exception {
+ Random r = newRandom();
+
Directory dir = new MockRAMDirectory();
+ RandomIndexWriter w = new RandomIndexWriter(r,
+ dir,
+ new IndexWriterConfig(TEST_VERSION_CURRENT,
+ new MockAnalyzer()).setCodecProvider(_TestUtil.alwaysCodec(new
PreFlexRWCodec())));
+
+ final int numField = _TestUtil.nextInt(r, 2, 5);
- Codec codec = new PreFlexCodec();
+ int uniqueTermCount = 0;
+
+ int tc = 0;
- Random r = newRandom();
- FieldInfos fieldInfos = new FieldInfos();
List<Term> fieldTerms = new ArrayList<Term>();
- SegmentInfo si = makePreFlexSegment(r, "_0", dir, fieldInfos, codec, fieldTerms);
- // hack alert!!
- int uniqueTermCount = si.docCount;
+ for(int f=0;f<numField;f++) {
+ String field = "f" + f;
+ Term protoTerm = new Term(field);
- FieldsProducer fields = codec.fieldsProducer(new SegmentReadState(dir, si, fieldInfos,
1024, 1));
- assertNotNull(fields);
+ final int numTerms = 10000*_TestUtil.getRandomMultiplier();
- if (VERBOSE) {
- System.out.println("\nTEST: now enum");
+ final Set<String> uniqueTerms = new HashSet<String>();
+
+ for(int i=0;i<numTerms;i++) {
+ String term = getRandomString(r) + "_ " + (tc++);
+ uniqueTerms.add(term);
+ fieldTerms.add(new Term(field, term));
+ Document doc = new Document();
+ doc.add(new Field(field, term, Field.Store.NO, Field.Index.NOT_ANALYZED));
+ w.addDocument(doc);
+ }
+ uniqueTermCount += uniqueTerms.size();
}
- FieldsEnum fieldsEnum = fields.iterator();
- String field;
- UnicodeUtil.UTF16Result utf16 = new UnicodeUtil.UTF16Result();
-
- int termCount = 0;
- while((field = fieldsEnum.next()) != null) {
- TermsEnum termsEnum = fieldsEnum.terms();
- BytesRef text;
- BytesRef lastText = null;
- while((text = termsEnum.next()) != null) {
- if (VERBOSE) {
- UnicodeUtil.UTF8toUTF16(text.bytes, text.offset, text.length, utf16);
- System.out.println("got term=" + field + ":" + UnicodeUtil.toHexString(new String(utf16.result,
0, utf16.length)));
- System.out.println();
- }
- if (lastText == null) {
- lastText = new BytesRef(text);
- } else {
- assertTrue(lastText.compareTo(text) < 0);
- lastText.copy(text);
- }
- assertEquals(fieldTerms.get(termCount).field(), field);
- assertEquals(fieldTerms.get(termCount).bytes(), text);
- termCount++;
+
+ IndexReader reader = w.getReader();
+
+ if (VERBOSE) {
+ Collections.sort(fieldTerms, termAsUTF16Comparator);
+
+ System.out.println("\nTEST: UTF16 order");
+ for(Term t: fieldTerms) {
+ System.out.println(" " + toHexString(t));
}
- if (VERBOSE) {
- System.out.println(" no more terms for field=" + field);
+ }
+
+ // sorts in code point order:
+ Collections.sort(fieldTerms);
+
+ if (VERBOSE) {
+ System.out.println("\nTEST: codepoint order");
+ for(Term t: fieldTerms) {
+ System.out.println(" " + toHexString(t));
}
}
- assertEquals(uniqueTermCount, termCount);
- fields.close();
+ Term[] fieldTermsArray = fieldTerms.toArray(new Term[fieldTerms.size()]);
+
+ //SegmentInfo si = makePreFlexSegment(r, "_0", dir, fieldInfos, codec, fieldTerms);
+
+ //FieldsProducer fields = codec.fieldsProducer(new SegmentReadState(dir, si, fieldInfos,
1024, 1));
+ //assertNotNull(fields);
+
+ doTestStraightEnum(fieldTerms, reader, uniqueTermCount);
+ doTestSeekExists(r, fieldTerms, reader);
+ doTestSeekDoesNotExist(r, numField, fieldTerms, fieldTermsArray, reader);
+
+ reader.close();
}
}
Added: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java?rev=967130&view=auto
==============================================================================
--- lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
(added)
+++ lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
Fri Jul 23 15:26:25 2010
@@ -0,0 +1,212 @@
+package org.apache.lucene.index.codecs.preflexrw;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.TermsConsumer;
+import org.apache.lucene.index.codecs.PostingsConsumer;
+import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter;
+import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.codecs.preflex.TermInfo;
+import org.apache.lucene.store.IndexOutput;
+
+import java.io.IOException;
+import java.util.Comparator;
+
+class PreFlexFieldsWriter extends FieldsConsumer {
+
+ private final TermInfosWriter termsOut;
+ private final IndexOutput freqOut;
+ private final IndexOutput proxOut;
+ private final DefaultSkipListWriter skipListWriter;
+ private final int totalNumDocs;
+
+ public PreFlexFieldsWriter(SegmentWriteState state) throws IOException {
+ termsOut = new TermInfosWriter(state.directory,
+ state.segmentName,
+ state.fieldInfos,
+ state.termIndexInterval);
+ state.flushedFiles.add(IndexFileNames.segmentFileName(state.segmentName, "", PreFlexCodec.TERMS_EXTENSION));
+ state.flushedFiles.add(IndexFileNames.segmentFileName(state.segmentName, "", PreFlexCodec.TERMS_INDEX_EXTENSION));
+
+ final String freqFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexCodec.FREQ_EXTENSION);
+ freqOut = state.directory.createOutput(freqFile);
+ state.flushedFiles.add(freqFile);
+ totalNumDocs = state.numDocs;
+
+ if (state.fieldInfos.hasProx()) {
+ final String proxFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexCodec.PROX_EXTENSION);
+ proxOut = state.directory.createOutput(proxFile);
+ state.flushedFiles.add(proxFile);
+ } else {
+ proxOut = null;
+ }
+
+ skipListWriter = new DefaultSkipListWriter(termsOut.skipInterval,
+ termsOut.maxSkipLevels,
+ totalNumDocs,
+ freqOut,
+ proxOut);
+ //System.out.println("\nw start seg=" + segment);
+ }
+
+ @Override
+ public TermsConsumer addField(FieldInfo field) throws IOException {
+ assert field.number != -1;
+ //System.out.println("w field=" + field.name + " storePayload=" + field.storePayloads
+ " number=" + field.number);
+ return new PreFlexTermsWriter(field);
+ }
+
+ @Override
+ public void close() throws IOException {
+ termsOut.close();
+ freqOut.close();
+ if (proxOut != null) {
+ proxOut.close();
+ }
+ }
+
+ private class PreFlexTermsWriter extends TermsConsumer {
+ private final FieldInfo fieldInfo;
+ private final boolean omitTF;
+ private final boolean storePayloads;
+
+ private final TermInfo termInfo = new TermInfo();
+ private final PostingsWriter postingsWriter = new PostingsWriter();
+
+ public PreFlexTermsWriter(FieldInfo fieldInfo) {
+ this.fieldInfo = fieldInfo;
+ omitTF = fieldInfo.omitTermFreqAndPositions;
+ storePayloads = fieldInfo.storePayloads;
+ }
+
+ private class PostingsWriter extends PostingsConsumer {
+ private int lastDocID;
+ private int lastPayloadLength = -1;
+ private int lastPosition;
+ private int df;
+
+ public PostingsWriter reset() {
+ df = 0;
+ lastDocID = 0;
+ lastPayloadLength = -1;
+ return this;
+ }
+
+ @Override
+ public void startDoc(int docID, int termDocFreq) throws IOException {
+ //System.out.println(" w doc=" + docID);
+
+ final int delta = docID - lastDocID;
+ if (docID < 0 || (df > 0 && delta <= 0)) {
+ throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID
+ " )");
+ }
+
+ if ((++df % termsOut.skipInterval) == 0) {
+ skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
+ skipListWriter.bufferSkip(df);
+ }
+
+ lastDocID = docID;
+
+ assert docID < totalNumDocs: "docID=" + docID + " totalNumDocs=" + totalNumDocs;
+
+ if (omitTF) {
+ freqOut.writeVInt(delta);
+ } else {
+ final int code = delta << 1;
+ if (termDocFreq == 1) {
+ freqOut.writeVInt(code|1);
+ } else {
+ freqOut.writeVInt(code);
+ freqOut.writeVInt(termDocFreq);
+ }
+ }
+ lastPosition = 0;
+ }
+
+ @Override
+ public void addPosition(int position, BytesRef payload) throws IOException {
+ assert proxOut != null;
+
+ //System.out.println(" w pos=" + position + " payl=" + payload);
+ final int delta = position - lastPosition;
+ lastPosition = position;
+
+ if (storePayloads) {
+ final int payloadLength = payload == null ? 0 : payload.length;
+ if (payloadLength != lastPayloadLength) {
+ //System.out.println(" write payload len=" + payloadLength);
+ lastPayloadLength = payloadLength;
+ proxOut.writeVInt((delta<<1)|1);
+ proxOut.writeVInt(payloadLength);
+ } else {
+ proxOut.writeVInt(delta << 1);
+ }
+ if (payloadLength > 0) {
+ proxOut.writeBytes(payload.bytes, payload.offset, payload.length);
+ }
+ } else {
+ proxOut.writeVInt(delta);
+ }
+ }
+
+ @Override
+ public void finishDoc() throws IOException {
+ }
+ }
+
+ @Override
+ public PostingsConsumer startTerm(BytesRef text) throws IOException {
+ //System.out.println(" w term=" + text.utf8ToString());
+ skipListWriter.resetSkip();
+ termInfo.freqPointer = freqOut.getFilePointer();
+ if (proxOut != null) {
+ termInfo.proxPointer = proxOut.getFilePointer();
+ }
+ return postingsWriter.reset();
+ }
+
+ @Override
+ public void finishTerm(BytesRef text, int numDocs) throws IOException {
+ if (numDocs > 0) {
+ long skipPointer = skipListWriter.writeSkip(freqOut);
+ termInfo.docFreq = numDocs;
+ termInfo.skipOffset = (int) (skipPointer - termInfo.freqPointer);
+ //System.out.println(" w finish term=" + text.utf8ToString() + " fnum=" + fieldInfo.number);
+ termsOut.add(fieldInfo.number,
+ text,
+ termInfo);
+ }
+ }
+
+ @Override
+ public void finish() throws IOException {
+ }
+
+ @Override
+ public Comparator<BytesRef> getComparator() throws IOException {
+ return BytesRef.getUTF8SortedAsUTF16Comparator();
+ }
+ }
+}
\ No newline at end of file
Propchange: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
------------------------------------------------------------------------------
svn:eol-style = native
Added: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java?rev=967130&view=auto
==============================================================================
--- lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java
(added)
+++ lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java
Fri Jul 23 15:26:25 2010
@@ -0,0 +1,52 @@
+package org.apache.lucene.index.codecs.preflexrw;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
+import org.apache.lucene.index.codecs.preflex.PreFlexFields;
+import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.FieldsProducer;
+
+/** Codec, only for testing, that can write and read the
+ * pre-flex index format.
+ *
+ * @lucene.experimental
+ */
+public class PreFlexRWCodec extends PreFlexCodec {
+
+ public PreFlexRWCodec() {
+ // NOTE: we use same name as core PreFlex codec so that
+ // it can read the segments we write!
+ super();
+ }
+
+ @Override
+ public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ return new PreFlexFieldsWriter(state);
+ }
+
+ @Override
+ public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+ //System.out.println("preflexrw");
+ return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.readBufferSize,
state.termsIndexDivisor, false);
+ }
+}
Propchange: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java
------------------------------------------------------------------------------
svn:eol-style = native
Added: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java?rev=967130&view=auto
==============================================================================
--- lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
(added)
+++ lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
Fri Jul 23 15:26:25 2010
@@ -0,0 +1,227 @@
+package org.apache.lucene.index.codecs.preflexrw;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+import java.io.IOException;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.index.codecs.preflex.TermInfo;
+
+
+/** This stores a monotonically increasing set of <Term, TermInfo> pairs in a
+ Directory. A TermInfos can be written once, in order. */
+
+final class TermInfosWriter {
+ /** The file format version, a negative number. */
+ public static final int FORMAT = -3;
+
+ // Changed strings to true utf8 with length-in-bytes not
+ // length-in-chars
+ public static final int FORMAT_VERSION_UTF8_LENGTH_IN_BYTES = -4;
+
+ // NOTE: always change this if you switch to a new format!
+ public static final int FORMAT_CURRENT = FORMAT_VERSION_UTF8_LENGTH_IN_BYTES;
+
+ private FieldInfos fieldInfos;
+ private IndexOutput output;
+ private TermInfo lastTi = new TermInfo();
+ private long size;
+
+ // TODO: the default values for these two parameters should be settable from
+ // IndexWriter. However, once that's done, folks will start setting them to
+ // ridiculous values and complaining that things don't work well, as with
+ // mergeFactor. So, let's wait until a number of folks find that alternate
+ // values work better. Note that both of these values are stored in the
+ // segment, so that it's safe to change these w/o rebuilding all indexes.
+
+ /** Expert: The fraction of terms in the "dictionary" which should be stored
+ * in RAM. Smaller values use more memory, but make searching slightly
+ * faster, while larger values use less memory and make searching slightly
+ * slower. Searching is typically not dominated by dictionary lookup, so
+ * tweaking this is rarely useful.*/
+ int indexInterval = 128;
+
+ /** Expert: The fraction of {@link TermDocs} entries stored in skip tables,
+ * used to accelerate {@link TermDocs#skipTo(int)}. Larger values result in
+ * smaller indexes, greater acceleration, but fewer accelerable cases, while
+ * smaller values result in bigger indexes, less acceleration and more
+ * accelerable cases. More detailed experiments would be useful here. */
+ int skipInterval = 16;
+
+ /** Expert: The maximum number of skip levels. Smaller values result in
+ * slightly smaller indexes, but slower skipping in big posting lists.
+ */
+ int maxSkipLevels = 10;
+
+ private long lastIndexPointer;
+ private boolean isIndex;
+ private final BytesRef lastTerm = new BytesRef();
+ private int lastFieldNumber = -1;
+
+ private TermInfosWriter other;
+
+ TermInfosWriter(Directory directory, String segment, FieldInfos fis,
+ int interval)
+ throws IOException {
+ initialize(directory, segment, fis, interval, false);
+ other = new TermInfosWriter(directory, segment, fis, interval, true);
+ other.other = this;
+ }
+
+ private TermInfosWriter(Directory directory, String segment, FieldInfos fis,
+ int interval, boolean isIndex) throws IOException {
+ initialize(directory, segment, fis, interval, isIndex);
+ }
+
+ private void initialize(Directory directory, String segment, FieldInfos fis,
+ int interval, boolean isi) throws IOException {
+ indexInterval = interval;
+ fieldInfos = fis;
+ isIndex = isi;
+ output = directory.createOutput(segment + (isIndex ? ".tii" : ".tis"));
+ output.writeInt(FORMAT_CURRENT); // write format
+ output.writeLong(0); // leave space for size
+ output.writeInt(indexInterval); // write indexInterval
+ output.writeInt(skipInterval); // write skipInterval
+ output.writeInt(maxSkipLevels); // write maxSkipLevels
+ assert initUTF16Results();
+ }
+
+ // Currently used only by assert statements
+ UnicodeUtil.UTF16Result utf16Result1;
+ UnicodeUtil.UTF16Result utf16Result2;
+ private final BytesRef scratchBytes = new BytesRef();
+
+ // Currently used only by assert statements
+ private boolean initUTF16Results() {
+ utf16Result1 = new UnicodeUtil.UTF16Result();
+ utf16Result2 = new UnicodeUtil.UTF16Result();
+ return true;
+ }
+
+ // Currently used only by assert statement
+ private int compareToLastTerm(int fieldNumber, BytesRef term) {
+
+ if (lastFieldNumber != fieldNumber) {
+ final int cmp = fieldInfos.fieldName(lastFieldNumber).compareTo(fieldInfos.fieldName(fieldNumber));
+ // If there is a field named "" (empty string) then we
+ // will get 0 on this comparison, yet, it's "OK". But
+ // it's not OK if two different field numbers map to
+ // the same name.
+ if (cmp != 0 || lastFieldNumber != -1)
+ return cmp;
+ }
+
+ scratchBytes.copy(term);
+ assert lastTerm.offset == 0;
+ UnicodeUtil.UTF8toUTF16(lastTerm.bytes, 0, lastTerm.length, utf16Result1);
+
+ assert scratchBytes.offset == 0;
+ UnicodeUtil.UTF8toUTF16(scratchBytes.bytes, 0, scratchBytes.length, utf16Result2);
+
+ final int len;
+ if (utf16Result1.length < utf16Result2.length)
+ len = utf16Result1.length;
+ else
+ len = utf16Result2.length;
+
+ for(int i=0;i<len;i++) {
+ final char ch1 = utf16Result1.result[i];
+ final char ch2 = utf16Result2.result[i];
+ if (ch1 != ch2)
+ return ch1-ch2;
+ }
+ return utf16Result1.length - utf16Result2.length;
+ }
+
+ /** Adds a new <<fieldNumber, termBytes>, TermInfo> pair to the set.
+ Term must be lexicographically greater than all previous Terms added.
+ TermInfo pointers must be positive and greater than all previous.*/
+ public void add(int fieldNumber, BytesRef term, TermInfo ti)
+ throws IOException {
+
+ assert compareToLastTerm(fieldNumber, term) < 0 ||
+ (isIndex && term.length == 0 && lastTerm.length == 0) :
+ "Terms are out of order: field=" + fieldInfos.fieldName(fieldNumber) + " (number "
+ fieldNumber + ")" +
+ " lastField=" + fieldInfos.fieldName(lastFieldNumber) + " (number " + lastFieldNumber
+ ")" +
+ " text=" + term.utf8ToString() + " lastText=" + lastTerm.utf8ToString();
+
+ assert ti.freqPointer >= lastTi.freqPointer: "freqPointer out of order (" + ti.freqPointer
+ " < " + lastTi.freqPointer + ")";
+ assert ti.proxPointer >= lastTi.proxPointer: "proxPointer out of order (" + ti.proxPointer
+ " < " + lastTi.proxPointer + ")";
+
+ if (!isIndex && size % indexInterval == 0)
+ other.add(lastFieldNumber, lastTerm, lastTi); // add an index
term
+
+ writeTerm(fieldNumber, term); // write term
+
+ output.writeVInt(ti.docFreq); // write doc freq
+ output.writeVLong(ti.freqPointer - lastTi.freqPointer); // write pointers
+ output.writeVLong(ti.proxPointer - lastTi.proxPointer);
+
+ if (ti.docFreq >= skipInterval) {
+ output.writeVInt(ti.skipOffset);
+ }
+
+ if (isIndex) {
+ output.writeVLong(other.output.getFilePointer() - lastIndexPointer);
+ lastIndexPointer = other.output.getFilePointer(); // write pointer
+ }
+
+ lastFieldNumber = fieldNumber;
+ lastTi.set(ti);
+ size++;
+ }
+
+ private void writeTerm(int fieldNumber, BytesRef term)
+ throws IOException {
+
+ //System.out.println(" tiw.write field=" + fieldNumber + " term=" + term.utf8ToString());
+
+ // TODO: UTF16toUTF8 could tell us this prefix
+ // Compute prefix in common with last term:
+ int start = 0;
+ final int limit = term.length < lastTerm.length ? term.length : lastTerm.length;
+ while(start < limit) {
+ if (term.bytes[start+term.offset] != lastTerm.bytes[start+lastTerm.offset])
+ break;
+ start++;
+ }
+
+ final int length = term.length - start;
+ output.writeVInt(start); // write shared prefix length
+ output.writeVInt(length); // write delta length
+ output.writeBytes(term.bytes, start+term.offset, length); // write delta bytes
+ output.writeVInt(fieldNumber); // write field num
+ lastTerm.copy(term);
+ }
+
+ /** Called to complete TermInfos creation. */
+ void close() throws IOException {
+ output.seek(4); // write size after format
+ output.writeLong(size);
+ output.close();
+
+ if (!isIndex)
+ other.close();
+ }
+
+}
Propchange: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java?rev=967130&r1=967129&r2=967130&view=diff
==============================================================================
--- lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java
(original)
+++ lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java
Fri Jul 23 15:26:25 2010
@@ -34,6 +34,8 @@ import org.apache.lucene.search.BooleanQ
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldCache.CacheEntry;
import org.apache.lucene.util.FieldCacheSanityChecker.Insanity;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
/**
* Base class for all Lucene unit tests.
@@ -72,6 +74,8 @@ public abstract class LuceneTestCase ext
private volatile Thread.UncaughtExceptionHandler savedUncaughtExceptionHandler = null;
+ private String savedDefaultCodec;
+
/** Used to track if setUp and tearDown are called correctly from subclasses */
private boolean setup;
@@ -110,6 +114,8 @@ public abstract class LuceneTestCase ext
ConcurrentMergeScheduler.setTestMode();
savedBoolMaxClauseCount = BooleanQuery.getMaxClauseCount();
+ savedDefaultCodec = CodecProvider.getDefaultCodec();
+ CodecProvider.setDefaultCodec(_TestUtil.getTestCodec());
}
/**
@@ -135,7 +141,8 @@ public abstract class LuceneTestCase ext
assertTrue("ensure your setUp() calls super.setUp()!!!", setup);
setup = false;
BooleanQuery.setMaxClauseCount(savedBoolMaxClauseCount);
-
+ CodecProvider.setDefaultCodec(savedDefaultCodec);
+
try {
Thread.setDefaultUncaughtExceptionHandler(savedUncaughtExceptionHandler);
if (!uncaughtExceptions.isEmpty()) {
@@ -298,4 +305,10 @@ public abstract class LuceneTestCase ext
// static members
private static final Random seedRnd = new Random();
+
+ // register preflex-rw statically.
+ static {
+ CodecProvider.getDefault().register(new PreFlexRWCodec());
+ }
+
}
Modified: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java?rev=967130&r1=967129&r2=967130&view=diff
==============================================================================
--- lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java
(original)
+++ lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java
Fri Jul 23 15:26:25 2010
@@ -22,9 +22,14 @@ import org.apache.lucene.search.BooleanQ
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldCache.CacheEntry;
import org.apache.lucene.util.FieldCacheSanityChecker.Insanity;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
+
import org.junit.After;
+import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
+import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestWatchman;
@@ -127,6 +132,20 @@ public class LuceneTestCaseJ4 {
private static final Map<Class<? extends LuceneTestCaseJ4>,Object> checkedClasses
=
Collections.synchronizedMap(new WeakHashMap<Class<? extends LuceneTestCaseJ4>,Object>());
+ // saves default codec: we do this statically as many build indexes in @beforeClass
+ private static String savedDefaultCodec;
+
+ @BeforeClass
+ public static void beforeClassLuceneTestCaseJ4() {
+ savedDefaultCodec = CodecProvider.getDefaultCodec();
+ CodecProvider.setDefaultCodec(_TestUtil.getTestCodec());
+ }
+
+ @AfterClass
+ public static void afterClassLuceneTestCaseJ4() {
+ CodecProvider.setDefaultCodec(savedDefaultCodec);
+ }
+
// This is how we get control when errors occur.
// Think of this as start/end/success/failed
// events.
@@ -405,4 +424,8 @@ public class LuceneTestCaseJ4 {
private String name = "<unknown>";
+ // register PreFlexRWCodec statically
+ static {
+ CodecProvider.getDefault().register(new PreFlexRWCodec());
+ }
}
Modified: lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/_TestUtil.java?rev=967130&r1=967129&r2=967130&view=diff
==============================================================================
--- lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/_TestUtil.java
(original)
+++ lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/_TestUtil.java
Fri Jul 23 15:26:25 2010
@@ -23,6 +23,9 @@ import org.apache.lucene.index.IndexWrit
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.CheckIndex;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.Directory;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
@@ -129,8 +132,24 @@ public class _TestUtil {
}
final char[] buffer = new char[end];
for (int i = 0; i < end; i++) {
+
int t = r.nextInt(5);
+
//buffer[i] = (char) (97 + r.nextInt(26));
+
+ /*
+ if (0 == t && i < end - 1) {
+ // hi
+ buffer[i++] = (char) 0xd800;
+ // lo
+ buffer[i] = (char) 0xdc00;
+ } else if (t <= 3) {
+ buffer[i] = 'a';
+ } else if (4 == t) {
+ buffer[i] = 0xe000;
+ }
+ */
+
if (0 == t && i < end - 1) {
// Make a surrogate pair
// High surrogate
@@ -218,4 +237,27 @@ public class _TestUtil {
public static int getRandomMultiplier() {
return Integer.parseInt(System.getProperty("random.multiplier", "1"));
}
+
+ /** gets the codec to run tests with */
+ public static String getTestCodec() {
+ return System.getProperty("tests.codec", "Standard");
+ }
+
+ public static CodecProvider alwaysCodec(final Codec c) {
+ return new CodecProvider() {
+ @Override
+ public Codec getWriter(SegmentWriteState state) {
+ return c;
+ }
+
+ @Override
+ public Codec lookup(String name) {
+ return c;
+ }
+ };
+ }
+
+ public static CodecProvider alwaysCodec(final String codec) {
+ return alwaysCodec(CodecProvider.getDefault().lookup(codec));
+ }
}
|