This class also interacts with an instance of {@link
+ * TermsIndexReaderBase}, to abstract away the specific
+ * implementation of the terms dict index.
+ * @lucene.experimental */
+
+public class BlockTermsReader extends FieldsProducer {
+ // Open input to the main terms dict file (_X.tis)
+ private final IndexInput in;
+
+ // Reads the terms dict entries, to gather state to
+ // produce DocsEnum on demand
+ private final PostingsReaderBase postingsReader;
+
+ private final TreeMap fields = new TreeMap();
+
+ // Comparator that orders our terms
+ private final Comparator termComp;
+
+ // Caches the most recently looked-up field + terms:
+ private final DoubleBarrelLRUCache termsCache;
+
+ // Reads the terms index
+ private TermsIndexReaderBase indexReader;
+
+ // keeps the dirStart offset
+ protected long dirOffset;
+
+ // Used as key for the terms cache
+ private static class FieldAndTerm extends DoubleBarrelLRUCache.CloneableKey {
+ String field;
+ BytesRef term;
+
+ public FieldAndTerm() {
+ }
+
+ public FieldAndTerm(FieldAndTerm other) {
+ field = other.field;
+ term = new BytesRef(other.term);
+ }
+
+ @Override
+ public boolean equals(Object _other) {
+ FieldAndTerm other = (FieldAndTerm) _other;
+ return other.field == field && term.bytesEquals(other.term);
+ }
+
+ @Override
+ public Object clone() {
+ return new FieldAndTerm(this);
+ }
+
+ @Override
+ public int hashCode() {
+ return field.hashCode() * 31 + term.hashCode();
+ }
+ }
+
+ private String segment;
+
+ public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
+ Comparator termComp, int termsCacheSize, String codecId)
+ throws IOException {
+
+ this.postingsReader = postingsReader;
+ termsCache = new DoubleBarrelLRUCache(termsCacheSize);
+
+ this.termComp = termComp;
+ this.segment = segment;
+ in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
+ readBufferSize);
+
+ boolean success = false;
+ try {
+ readHeader(in);
+
+ // Have PostingsReader init itself
+ postingsReader.init(in);
+
+ // Read per-field details
+ seekDir(in, dirOffset);
+
+ final int numFields = in.readVInt();
+
+ for(int i=0;i= 0;
+ final long termsStartPointer = in.readVLong();
+ final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+ final long sumTotalTermFreq = fieldInfo.omitTermFreqAndPositions ? -1 : in.readVLong();
+ assert !fields.containsKey(fieldInfo.name);
+ fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, termsStartPointer, sumTotalTermFreq));
+ }
+ success = true;
+ } finally {
+ if (!success) {
+ in.close();
+ }
+ }
+
+ this.indexReader = indexReader;
+ }
+
+ protected void readHeader(IndexInput input) throws IOException {
+ CodecUtil.checkHeader(in, BlockTermsWriter.CODEC_NAME,
+ BlockTermsWriter.VERSION_START,
+ BlockTermsWriter.VERSION_CURRENT);
+ dirOffset = in.readLong();
+ }
+
+ protected void seekDir(IndexInput input, long dirOffset)
+ throws IOException {
+ input.seek(dirOffset);
+ }
+
+ @Override
+ public void loadTermsIndex(int indexDivisor) throws IOException {
+ indexReader.loadTermsIndex(indexDivisor);
+ }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ try {
+ if (indexReader != null) {
+ indexReader.close();
+ }
+ } finally {
+ // null so if an app hangs on to us (ie, we are not
+ // GCable, despite being closed) we still free most
+ // ram
+ indexReader = null;
+ if (in != null) {
+ in.close();
+ }
+ }
+ } finally {
+ try {
+ if (postingsReader != null) {
+ postingsReader.close();
+ }
+ } finally {
+ for(FieldReader field : fields.values()) {
+ field.close();
+ }
+ }
+ }
+ }
+
+ public static void files(Directory dir, SegmentInfo segmentInfo, String id, Collection files) {
+ files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, BlockTermsWriter.TERMS_EXTENSION));
+ }
+
+ public static void getExtensions(Collection extensions) {
+ extensions.add(BlockTermsWriter.TERMS_EXTENSION);
+ }
+
+ @Override
+ public FieldsEnum iterator() {
+ return new TermFieldsEnum();
+ }
+
+ @Override
+ public Terms terms(String field) throws IOException {
+ return fields.get(field);
+ }
+
+ // Iterates through all fields
+ private class TermFieldsEnum extends FieldsEnum {
+ final Iterator it;
+ FieldReader current;
+
+ TermFieldsEnum() {
+ it = fields.values().iterator();
+ }
+
+ @Override
+ public String next() {
+ if (it.hasNext()) {
+ current = it.next();
+ return current.fieldInfo.name;
+ } else {
+ current = null;
+ return null;
+ }
+ }
+
+ @Override
+ public TermsEnum terms() throws IOException {
+ return current.iterator();
+ }
+ }
+
+ private class FieldReader extends Terms implements Closeable {
+ final long numTerms;
+ final FieldInfo fieldInfo;
+ final long termsStartPointer;
+ final long sumTotalTermFreq;
+
+ FieldReader(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq) {
+ assert numTerms > 0;
+ this.fieldInfo = fieldInfo;
+ this.numTerms = numTerms;
+ this.termsStartPointer = termsStartPointer;
+ this.sumTotalTermFreq = sumTotalTermFreq;
+ }
+
+ @Override
+ public Comparator getComparator() {
+ return termComp;
+ }
+
+ @Override
+ public void close() {
+ super.close();
+ }
+
+ @Override
+ public TermsEnum iterator() throws IOException {
+ return new SegmentTermsEnum();
+ }
+
+ @Override
+ public long getUniqueTermCount() {
+ return numTerms;
+ }
+
+ @Override
+ public long getSumTotalTermFreq() {
+ return sumTotalTermFreq;
+ }
+
+ // Iterates through terms in this field
+ private final class SegmentTermsEnum extends TermsEnum {
+ private final IndexInput in;
+ private final BlockTermState state;
+ private final boolean doOrd;
+ private final FieldAndTerm fieldTerm = new FieldAndTerm();
+ private final TermsIndexReaderBase.FieldIndexEnum indexEnum;
+ private final BytesRef term = new BytesRef();
+
+ /* This is true if indexEnum is "still" seek'd to the index term
+ for the current term. We set it to true on seeking, and then it
+ remains valid until next() is called enough times to load another
+ terms block: */
+ private boolean indexIsCurrent;
+
+ /* True if we've already called .next() on the indexEnum, to "bracket"
+ the current block of terms: */
+ private boolean didIndexNext;
+
+ /* Next index term, bracketing the current block of terms; this is
+ only valid if didIndexNext is true: */
+ private BytesRef nextIndexTerm;
+
+ /* True after seek(TermState), do defer seeking. If the app then
+ calls next() (which is not "typical"), then we'll do the real seek */
+ private boolean seekPending;
+
+ /* How many blocks we've read since last seek. Once this
+ is >= indexEnum.getDivisor() we set indexIsCurrent to false (since
+ the index can no long bracket seek-within-block). */
+ private int blocksSinceSeek;
+
+ private byte[] termSuffixes;
+ private ByteArrayDataInput termSuffixesReader = new ByteArrayDataInput(null);
+
+ /* Common prefix used for all terms in this block. */
+ private int termBlockPrefix;
+
+ private byte[] docFreqBytes;
+ private final ByteArrayDataInput freqReader = new ByteArrayDataInput(null);
+ private int metaDataUpto;
+
+ public SegmentTermsEnum() throws IOException {
+ in = (IndexInput) BlockTermsReader.this.in.clone();
+ in.seek(termsStartPointer);
+ indexEnum = indexReader.getFieldEnum(fieldInfo);
+ doOrd = indexReader.supportsOrd();
+ fieldTerm.field = fieldInfo.name;
+ state = postingsReader.newTermState();
+ state.totalTermFreq = -1;
+ state.ord = -1;
+
+ termSuffixes = new byte[128];
+ docFreqBytes = new byte[64];
+ //System.out.println("BTR.enum init this=" + this + " postingsReader=" + postingsReader);
+ }
+
+ @Override
+ public Comparator getComparator() {
+ return termComp;
+ }
+
+ @Override
+ public SeekStatus seek(final BytesRef target, final boolean useCache) throws IOException {
+
+ if (indexEnum == null) {
+ throw new IllegalStateException("terms index was not loaded");
+ }
+
+ //System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this=" + this);
+ /*
+ if (didIndexNext) {
+ if (nextIndexTerm == null) {
+ //System.out.println(" nextIndexTerm=null");
+ } else {
+ //System.out.println(" nextIndexTerm=" + nextIndexTerm.utf8ToString());
+ }
+ }
+ */
+
+ // Check cache
+ if (useCache) {
+ fieldTerm.term = target;
+ // TODO: should we differentiate "frozen"
+ // TermState (ie one that was cloned and
+ // cached/returned by termState()) from the
+ // malleable (primary) one?
+ final TermState cachedState = termsCache.get(fieldTerm);
+ if (cachedState != null) {
+ seekPending = true;
+ //System.out.println(" cached!");
+ seek(target, cachedState);
+ //System.out.println(" term=" + term.utf8ToString());
+ return SeekStatus.FOUND;
+ }
+ }
+
+ boolean doSeek = true;
+
+ // See if we can avoid seeking, because target term
+ // is after current term but before next index term:
+ if (indexIsCurrent) {
+
+ final int cmp = termComp.compare(term, target);
+
+ if (cmp == 0) {
+ // Already at the requested term
+ return SeekStatus.FOUND;
+ } else if (cmp < 0) {
+
+ // Target term is after current term
+ if (!didIndexNext) {
+ if (indexEnum.next() == -1) {
+ nextIndexTerm = null;
+ } else {
+ nextIndexTerm = indexEnum.term();
+ }
+ //System.out.println(" now do index next() nextIndexTerm=" + (nextIndexTerm == null ? "null" : nextIndexTerm.utf8ToString()));
+ didIndexNext = true;
+ }
+
+ if (nextIndexTerm == null || termComp.compare(target, nextIndexTerm) < 0) {
+ // Optimization: requested term is within the
+ // same term block we are now in; skip seeking
+ // (but do scanning):
+ doSeek = false;
+ //System.out.println(" skip seek: nextIndexTerm=" + (nextIndexTerm == null ? "null" : nextIndexTerm.utf8ToString()));
+ }
+ }
+ }
+
+ if (doSeek) {
+ //System.out.println(" seek");
+
+ // Ask terms index to find biggest indexed term (=
+ // first term in a block) that's <= our text:
+ in.seek(indexEnum.seek(target));
+ boolean result = nextBlock();
+
+ // Block must exist since, at least, the indexed term
+ // is in the block:
+ assert result;
+
+ indexIsCurrent = true;
+ didIndexNext = false;
+ blocksSinceSeek = 0;
+
+ if (doOrd) {
+ state.ord = indexEnum.ord()-1;
+ }
+
+ // NOTE: the first _next() after an index seek is
+ // a bit wasteful, since it redundantly reads some
+ // suffix bytes into the buffer. We could avoid storing
+ // those bytes in the primary file, but then when
+ // next()ing over an index term we'd have to
+ // special case it:
+ term.copy(indexEnum.term());
+ //System.out.println(" seek: term=" + term.utf8ToString());
+ } else {
+ ////System.out.println(" skip seek");
+ }
+
+ seekPending = false;
+
+ // Now scan:
+ while (_next() != null) {
+ final int cmp = termComp.compare(term, target);
+ if (cmp == 0) {
+ // Match!
+ if (useCache) {
+ // Store in cache
+ decodeMetaData();
+ termsCache.put(new FieldAndTerm(fieldTerm), (BlockTermState) state.clone());
+ }
+ //System.out.println(" FOUND");
+ return SeekStatus.FOUND;
+ } else if (cmp > 0) {
+ //System.out.println(" NOT_FOUND term=" + term.utf8ToString());
+ return SeekStatus.NOT_FOUND;
+ }
+
+ // The purpose of the terms dict index is to seek
+ // the enum to the closest index term before the
+ // term we are looking for. So, we should never
+ // cross another index term (besides the first
+ // one) while we are scanning:
+ assert indexIsCurrent;
+ }
+
+ indexIsCurrent = false;
+ //System.out.println(" END");
+ return SeekStatus.END;
+ }
+
+ @Override
+ public BytesRef next() throws IOException {
+ //System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termCount);
+
+ // If seek was previously called and the term was cached,
+ // usually caller is just going to pull a D/&PEnum or get
+ // docFreq, etc. But, if they then call next(),
+ // this method catches up all internal state so next()
+ // works properly:
+ if (seekPending) {
+ assert !indexIsCurrent;
+ in.seek(state.blockFilePointer);
+ final int pendingSeekCount = state.termCount;
+ boolean result = nextBlock();
+
+ final long savOrd = state.ord;
+
+ // Block must exist since seek(TermState) was called w/ a
+ // TermState previously returned by this enum when positioned
+ // on a real term:
+ assert result;
+
+ while(state.termCount < pendingSeekCount) {
+ BytesRef nextResult = _next();
+ assert nextResult != null;
+ }
+ seekPending = false;
+ state.ord = savOrd;
+ }
+ return _next();
+ }
+
+ /* Decodes only the term bytes of the next term. If caller then asks for
+ metadata, ie docFreq, totalTermFreq or pulls a D/&PEnum, we then (lazily)
+ decode all metadata up to the current term. */
+ private BytesRef _next() throws IOException {
+ //System.out.println("BTR._next this=" + this + " termCount=" + state.termCount + " (vs " + state.blockTermCount + ")");
+ if (state.termCount == state.blockTermCount) {
+ if (!nextBlock()) {
+ //System.out.println(" eof");
+ indexIsCurrent = false;
+ return null;
+ }
+ }
+
+ // TODO: cutover to something better for these ints! simple64?
+ final int suffix = termSuffixesReader.readVInt();
+ //System.out.println(" suffix=" + suffix);
+
+ term.length = termBlockPrefix + suffix;
+ if (term.bytes.length < term.length) {
+ term.grow(term.length);
+ }
+ termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+ state.termCount++;
+
+ // NOTE: meaningless in the non-ord case
+ state.ord++;
+
+ //System.out.println(" return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term);
+ return term;
+ }
+
+ @Override
+ public BytesRef term() {
+ return term;
+ }
+
+ @Override
+ public int docFreq() throws IOException {
+ //System.out.println("BTR.docFreq");
+ decodeMetaData();
+ //System.out.println(" return " + state.docFreq);
+ return state.docFreq;
+ }
+
+ @Override
+ public long totalTermFreq() throws IOException {
+ decodeMetaData();
+ return state.totalTermFreq;
+ }
+
+ @Override
+ public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
+ //System.out.println("BTR.docs this=" + this);
+ decodeMetaData();
+ //System.out.println(" state.docFreq=" + state.docFreq);
+ final DocsEnum docsEnum = postingsReader.docs(fieldInfo, state, skipDocs, reuse);
+ assert docsEnum != null;
+ return docsEnum;
+ }
+
+ @Override
+ public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+ //System.out.println("BTR.d&p this=" + this);
+ decodeMetaData();
+ if (fieldInfo.omitTermFreqAndPositions) {
+ return null;
+ } else {
+ DocsAndPositionsEnum dpe = postingsReader.docsAndPositions(fieldInfo, state, skipDocs, reuse);
+ //System.out.println(" return d&pe=" + dpe);
+ return dpe;
+ }
+ }
+
+ @Override
+ public void seek(BytesRef target, TermState otherState) throws IOException {
+ //System.out.println("BTR.seek termState target=" + target.utf8ToString() + " " + target + " this=" + this);
+ assert otherState != null && otherState instanceof BlockTermState;
+ assert !doOrd || ((BlockTermState) otherState).ord < numTerms;
+ state.copyFrom(otherState);
+ seekPending = true;
+ indexIsCurrent = false;
+ term.copy(target);
+ }
+
+ @Override
+ public TermState termState() throws IOException {
+ //System.out.println("BTR.termState this=" + this);
+ decodeMetaData();
+ TermState ts = (TermState) state.clone();
+ //System.out.println(" return ts=" + ts);
+ return ts;
+ }
+
+ @Override
+ public SeekStatus seek(long ord) throws IOException {
+ //System.out.println("BTR.seek by ord ord=" + ord);
+ if (indexEnum == null) {
+ throw new IllegalStateException("terms index was not loaded");
+ }
+
+ if (ord >= numTerms) {
+ state.ord = numTerms-1;
+ return SeekStatus.END;
+ }
+
+ // TODO: if ord is in same terms block and
+ // after current ord, we should avoid this seek just
+ // like we do in the seek(BytesRef) case
+ in.seek(indexEnum.seek(ord));
+ boolean result = nextBlock();
+
+ // Block must exist since ord < numTerms:
+ assert result;
+
+ indexIsCurrent = true;
+ didIndexNext = false;
+ blocksSinceSeek = 0;
+ seekPending = false;
+
+ state.ord = indexEnum.ord()-1;
+ assert state.ord >= -1: "ord=" + state.ord;
+ term.copy(indexEnum.term());
+
+ // Now, scan:
+ int left = (int) (ord - state.ord);
+ while(left > 0) {
+ final BytesRef term = _next();
+ assert term != null;
+ left--;
+ assert indexIsCurrent;
+ }
+
+ // always found
+ return SeekStatus.FOUND;
+ }
+
+ public long ord() {
+ if (!doOrd) {
+ throw new UnsupportedOperationException();
+ }
+ return state.ord;
+ }
+
+ private void doPendingSeek() {
+ }
+
+ /* Does initial decode of next block of terms; this
+ doesn't actually decode the docFreq, totalTermFreq,
+ postings details (frq/prx offset, etc.) metadata;
+ it just loads them as byte[] blobs which are then
+ decoded on-demand if the metadata is ever requested
+ for any term in this block. This enables terms-only
+ intensive consumes (eg certain MTQs, respelling) to
+ not pay the price of decoding metadata they won't
+ use. */
+ private boolean nextBlock() throws IOException {
+
+ // TODO: we still lazy-decode the byte[] for each
+ // term (the suffix), but, if we decoded
+ // all N terms up front then seeking could do a fast
+ // bsearch w/in the block...
+
+ //System.out.println("BTR.nextBlock() fp=" + in.getFilePointer() + " this=" + this);
+ state.blockFilePointer = in.getFilePointer();
+ state.blockTermCount = in.readVInt();
+ //System.out.println(" blockTermCount=" + state.blockTermCount);
+ if (state.blockTermCount == 0) {
+ return false;
+ }
+ termBlockPrefix = in.readVInt();
+
+ // term suffixes:
+ int len = in.readVInt();
+ if (termSuffixes.length < len) {
+ termSuffixes = new byte[ArrayUtil.oversize(len, 1)];
+ }
+ //System.out.println(" termSuffixes len=" + len);
+ in.readBytes(termSuffixes, 0, len);
+ termSuffixesReader.reset(termSuffixes);
+
+ // docFreq, totalTermFreq
+ len = in.readVInt();
+ if (docFreqBytes.length < len) {
+ docFreqBytes = new byte[ArrayUtil.oversize(len, 1)];
+ }
+ //System.out.println(" freq bytes len=" + len);
+ in.readBytes(docFreqBytes, 0, len);
+ freqReader.reset(docFreqBytes);
+ metaDataUpto = 0;
+
+ state.termCount = 0;
+
+ postingsReader.readTermsBlock(in, fieldInfo, state);
+
+ blocksSinceSeek++;
+ indexIsCurrent &= (blocksSinceSeek < indexReader.getDivisor());
+ //System.out.println(" indexIsCurrent=" + indexIsCurrent);
+
+ return true;
+ }
+
+ private void decodeMetaData() throws IOException {
+ //System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termCount + " state=" + state);
+ if (!seekPending) {
+ // lazily catch up on metadata decode:
+ final int limit = state.termCount;
+ state.termCount = metaDataUpto;
+ while (metaDataUpto < limit) {
+ //System.out.println(" decode");
+ // TODO: we could make "tiers" of metadata, ie,
+ // decode docFreq/totalTF but don't decode postings
+ // metadata; this way caller could get
+ // docFreq/totalTF w/o paying decode cost for
+ // postings
+ state.docFreq = freqReader.readVInt();
+ if (!fieldInfo.omitTermFreqAndPositions) {
+ state.totalTermFreq = state.docFreq + freqReader.readVLong();
+ }
+ postingsReader.nextTerm(fieldInfo, state);
+ metaDataUpto++;
+ state.termCount++;
+ }
+ } else {
+ //System.out.println(" skip! seekPending");
+ }
+ }
+ }
+ }
+}
diff --git a/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java b/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
new file mode 100644
index 00000000000..c60b42506ed
--- /dev/null
+++ b/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
@@ -0,0 +1,316 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * 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 java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// TODO: currently we encode all terms between two indexed
+// terms as a block; but, we could decouple the two, ie
+// allow several blocks in between two indexed terms
+
+/**
+ * Writes terms dict, block-encoding (column stride) each
+ * term's metadata for each set of terms between two
+ * index terms.
+ *
+ * @lucene.experimental
+ */
+
+public class BlockTermsWriter extends FieldsConsumer {
+
+ final static String CODEC_NAME = "BLOCK_TERMS_DICT";
+
+ // Initial format
+ public static final int VERSION_START = 0;
+
+ public static final int VERSION_CURRENT = VERSION_START;
+
+ /** Extension of terms file */
+ static final String TERMS_EXTENSION = "tib";
+
+ protected final IndexOutput out;
+ final PostingsWriterBase postingsWriter;
+ final FieldInfos fieldInfos;
+ FieldInfo currentField;
+ private final TermsIndexWriterBase termsIndexWriter;
+ private final List fields = new ArrayList();
+ private final Comparator termComp;
+ private final String segment;
+
+ public BlockTermsWriter(
+ TermsIndexWriterBase termsIndexWriter,
+ SegmentWriteState state,
+ PostingsWriterBase postingsWriter,
+ Comparator termComp) throws IOException
+ {
+ final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
+ this.termsIndexWriter = termsIndexWriter;
+ this.termComp = termComp;
+ out = state.directory.createOutput(termsFileName);
+ fieldInfos = state.fieldInfos;
+ writeHeader(out);
+ currentField = null;
+ this.postingsWriter = postingsWriter;
+ segment = state.segmentName;
+
+ //System.out.println("BTW.init seg=" + state.segmentName);
+
+ postingsWriter.start(out); // have consumer write its format/header
+ }
+
+ protected void writeHeader(IndexOutput out) throws IOException {
+ CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
+
+ out.writeLong(0); // leave space for end index pointer
+ }
+
+ @Override
+ public TermsConsumer addField(FieldInfo field) throws IOException {
+ //System.out.println("\nBTW.addField seg=" + segment + " field=" + field.name);
+ assert currentField == null || currentField.name.compareTo(field.name) < 0;
+ currentField = field;
+ TermsIndexWriterBase.FieldWriter fieldIndexWriter = termsIndexWriter.addField(field, out.getFilePointer());
+ final TermsWriter terms = new TermsWriter(fieldIndexWriter, field, postingsWriter);
+ fields.add(terms);
+ return terms;
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ try {
+
+ int nonZeroCount = 0;
+ for(TermsWriter field : fields) {
+ if (field.numTerms > 0) {
+ nonZeroCount++;
+ }
+ }
+
+ final long dirStart = out.getFilePointer();
+
+ out.writeVInt(nonZeroCount);
+ for(TermsWriter field : fields) {
+ if (field.numTerms > 0) {
+ out.writeVInt(field.fieldInfo.number);
+ out.writeVLong(field.numTerms);
+ out.writeVLong(field.termsStartPointer);
+ if (!field.fieldInfo.omitTermFreqAndPositions) {
+ out.writeVLong(field.sumTotalTermFreq);
+ }
+ }
+ }
+ writeTrailer(dirStart);
+ } finally {
+ try {
+ out.close();
+ } finally {
+ try {
+ postingsWriter.close();
+ } finally {
+ termsIndexWriter.close();
+ }
+ }
+ }
+ }
+
+ protected void writeTrailer(long dirStart) throws IOException {
+ // TODO Auto-generated method stub
+ out.seek(CodecUtil.headerLength(CODEC_NAME));
+ out.writeLong(dirStart);
+ }
+
+ private static class TermEntry {
+ public final BytesRef term = new BytesRef();
+ public TermStats stats;
+ }
+
+ class TermsWriter extends TermsConsumer {
+ private final FieldInfo fieldInfo;
+ private final PostingsWriterBase postingsWriter;
+ private final long termsStartPointer;
+ private long numTerms;
+ private final TermsIndexWriterBase.FieldWriter fieldIndexWriter;
+ long sumTotalTermFreq;
+ private final BytesRef lastTerm = new BytesRef();
+
+ private TermEntry[] pendingTerms;
+
+ private int pendingCount;
+
+ TermsWriter(
+ TermsIndexWriterBase.FieldWriter fieldIndexWriter,
+ FieldInfo fieldInfo,
+ PostingsWriterBase postingsWriter)
+ {
+ this.fieldInfo = fieldInfo;
+ this.fieldIndexWriter = fieldIndexWriter;
+ pendingTerms = new TermEntry[32];
+ for(int i=0;i getComparator() {
+ return termComp;
+ }
+
+ @Override
+ public PostingsConsumer startTerm(BytesRef text) throws IOException {
+ //System.out.println("BTW.startTerm seg=" + segment + " term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text);
+ postingsWriter.startTerm();
+ return postingsWriter;
+ }
+
+ private final BytesRef lastPrevTerm = new BytesRef();
+
+ @Override
+ public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+
+ assert stats.docFreq > 0;
+ //System.out.println("BTW.finishTerm seg=" + segment + " term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " df=" + stats.docFreq);
+
+ final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
+
+ if (isIndexTerm) {
+ if (pendingCount > 0) {
+ // Instead of writing each term, live, we gather terms
+ // in RAM in a pending buffer, and then write the
+ // entire block in between index terms:
+ flushBlock();
+ }
+ fieldIndexWriter.add(text, stats, out.getFilePointer());
+ }
+
+ if (pendingTerms.length == pendingCount) {
+ final TermEntry[] newArray = new TermEntry[ArrayUtil.oversize(pendingCount+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+ System.arraycopy(pendingTerms, 0, newArray, 0, pendingCount);
+ for(int i=pendingCount;i 0) {
+ flushBlock();
+ }
+ // EOF marker:
+ out.writeVInt(0);
+
+ this.sumTotalTermFreq = sumTotalTermFreq;
+ fieldIndexWriter.finish(out.getFilePointer());
+ }
+
+ private int sharedPrefix(BytesRef term1, BytesRef term2) {
+ assert term1.offset == 0;
+ assert term2.offset == 0;
+ int pos1 = 0;
+ int pos1End = pos1 + Math.min(term1.length, term2.length);
+ int pos2 = 0;
+ while(pos1 < pos1End) {
+ if (term1.bytes[pos1] != term2.bytes[pos2]) {
+ return pos1;
+ }
+ pos1++;
+ pos2++;
+ }
+ return pos1;
+ }
+
+ private final RAMOutputStream bytesWriter = new RAMOutputStream();
+
+ private void flushBlock() throws IOException {
+ //System.out.println("BTW.flushBlock pendingCount=" + pendingCount);
+
+ // First pass: compute common prefix for all terms
+ // in the block, against term before first term in
+ // this block:
+ int commonPrefix = sharedPrefix(lastPrevTerm, pendingTerms[0].term);
+ for(int termCount=1;termCount attClass, String key, Object value);
+
+}
diff --git a/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java b/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java
new file mode 100644
index 00000000000..b8e9a0df7e2
--- /dev/null
+++ b/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java
@@ -0,0 +1,46 @@
+package org.apache.lucene.analysis.tokenattributes;
+
+/**
+ * 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._TestUtil;
+import org.apache.lucene.util.LuceneTestCase;
+
+import java.util.Collections;
+import java.util.HashMap;
+
+public class TestSimpleAttributeImpl extends LuceneTestCase {
+
+ // this checks using reflection API if the defaults are correct
+ public void testAttributes() {
+ _TestUtil.assertAttributeReflection(new PositionIncrementAttributeImpl(),
+ Collections.singletonMap(PositionIncrementAttribute.class.getName()+"#positionIncrement", 1));
+ _TestUtil.assertAttributeReflection(new FlagsAttributeImpl(),
+ Collections.singletonMap(FlagsAttribute.class.getName()+"#flags", 0));
+ _TestUtil.assertAttributeReflection(new TypeAttributeImpl(),
+ Collections.singletonMap(TypeAttribute.class.getName()+"#type", TypeAttribute.DEFAULT_TYPE));
+ _TestUtil.assertAttributeReflection(new PayloadAttributeImpl(),
+ Collections.singletonMap(PayloadAttribute.class.getName()+"#payload", null));
+ _TestUtil.assertAttributeReflection(new KeywordAttributeImpl(),
+ Collections.singletonMap(KeywordAttribute.class.getName()+"#keyword", false));
+ _TestUtil.assertAttributeReflection(new OffsetAttributeImpl(), new HashMap() {{
+ put(OffsetAttribute.class.getName()+"#startOffset", 0);
+ put(OffsetAttribute.class.getName()+"#endOffset", 0);
+ }});
+ }
+
+}
diff --git a/solr/contrib/analysis-extras/CHANGES.txt b/solr/contrib/analysis-extras/CHANGES.txt
new file mode 100644
index 00000000000..e512decaa99
--- /dev/null
+++ b/solr/contrib/analysis-extras/CHANGES.txt
@@ -0,0 +1,30 @@
+ Apache Solr - Analysis Extras
+ Release Notes
+
+Introduction
+------------
+The analysis-extras plugin provides additional analyzers that rely
+upon large dependencies/dictionaries.
+
+It includes integration with ICU for multilingual support, and
+analyzers for Chinese and Polish.
+
+
+$Id$
+================== Release 4.0-dev ==================
+
+(No Changes)
+
+================== Release 3.1-dev ==================
+
+* SOLR-2210: Add icu-based tokenizer and filters to contrib/analysis-extras (rmuir)
+
+* SOLR-1336: Add SmartChinese (word segmentation for Simplified Chinese)
+ tokenizer and filters to contrib/analysis-extras (rmuir)
+
+* SOLR-2211,LUCENE-2763: Added UAX29URLEmailTokenizerFactory, which implements
+ UAX#29, a unicode algorithm with good results for most languages, as well as
+ URL and E-mail tokenization according to the relevant RFCs.
+ (Tom Burton-West via rmuir)
+
+* SOLR-2237: Added StempelPolishStemFilterFactory to contrib/analysis-extras (rmuir)
diff --git a/solr/src/java/org/apache/solr/search/SolrSortField.java b/solr/src/java/org/apache/solr/search/SolrSortField.java
new file mode 100644
index 00000000000..8b21e4357bd
--- /dev/null
+++ b/solr/src/java/org/apache/solr/search/SolrSortField.java
@@ -0,0 +1,31 @@
+/**
+ * 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.
+ */
+
+
+package org.apache.solr.search;
+
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SortField;
+
+import java.io.IOException;
+
+/**@lucene.internal
+ *
+ */
+public interface SolrSortField {
+ public SortField weight(IndexSearcher searcher) throws IOException;
+}
diff --git a/solr/src/test/org/apache/solr/response/TestPHPSerializedResponseWriter.java b/solr/src/test/org/apache/solr/response/TestPHPSerializedResponseWriter.java
new file mode 100644
index 00000000000..d67e1fb8cb8
--- /dev/null
+++ b/solr/src/test/org/apache/solr/response/TestPHPSerializedResponseWriter.java
@@ -0,0 +1,107 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.response;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.response.PHPSerializedResponseWriter;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.QueryResponseWriter;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Basic PHPS tests based on JSONWriterTest
+ *
+ */
+public class TestPHPSerializedResponseWriter extends SolrTestCaseJ4 {
+ @BeforeClass
+ public static void beforeClass() throws Exception {
+ initCore("solrconfig.xml","schema.xml");
+ }
+
+ @Test
+ public void testSimple() throws IOException {
+ SolrQueryRequest req = req("dummy");
+ SolrQueryResponse rsp = new SolrQueryResponse();
+ QueryResponseWriter w = new PHPSerializedResponseWriter();
+
+ StringWriter buf = new StringWriter();
+ rsp.add("data1", "hello");
+ rsp.add("data2", 42);
+ rsp.add("data3", true);
+ w.write(buf, req, rsp);
+ assertEquals("a:3:{s:5:\"data1\";s:5:\"hello\";s:5:\"data2\";i:42;s:5:\"data3\";b:1;}",
+ buf.toString());
+ req.close();
+ }
+
+
+ @Test
+ public void testSolrDocuments() throws IOException {
+ SolrQueryRequest req = req("q","*:*");
+ SolrQueryResponse rsp = new SolrQueryResponse();
+ QueryResponseWriter w = new PHPSerializedResponseWriter();
+ StringWriter buf = new StringWriter();
+
+ SolrDocument d = new SolrDocument();
+
+ SolrDocument d1 = d;
+ d.addField("id","1");
+ d.addField("data1","hello");
+ d.addField("data2",42);
+ d.addField("data3",true);
+
+ // multivalued fields:
+
+ // extremely odd edge case: value is a map
+
+ // we use LinkedHashMap because we are doing a string comparison
+ // later and we need predictible ordering
+ LinkedHashMap nl = new LinkedHashMap();
+ nl.put("data4.1", "hashmap");
+ nl.put("data4.2", "hello");
+ d.addField("data4",nl);
+ // array value
+ d.addField("data5",Arrays.asList("data5.1", "data5.2", "data5.3"));
+
+ // adding one more document to test array indexes
+ d = new SolrDocument();
+ SolrDocument d2 = d;
+ d.addField("id","2");
+
+ SolrDocumentList sdl = new SolrDocumentList();
+ sdl.add(d1);
+ sdl.add(d2);
+ rsp.add("response", sdl);
+
+ w.write(buf, req, rsp);
+ assertEquals("a:1:{s:8:\"response\";a:3:{s:8:\"numFound\";i:0;s:5:\"start\";i:0;s:4:\"docs\";a:2:{i:0;a:6:{s:2:\"id\";s:1:\"1\";s:5:\"data1\";s:5:\"hello\";s:5:\"data2\";i:42;s:5:\"data3\";b:1;s:5:\"data4\";a:2:{s:7:\"data4.1\";s:7:\"hashmap\";s:7:\"data4.2\";s:5:\"hello\";}s:5:\"data5\";a:3:{i:0;s:7:\"data5.1\";i:1;s:7:\"data5.2\";i:2;s:7:\"data5.3\";}}i:1;a:1:{s:2:\"id\";s:1:\"2\";}}}}",
+ buf.toString());
+ req.close();
+ }
+
+}