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 2011/11/15 18:05:56 UTC

svn commit: r1202305 - in /lucene/dev/branches/lucene2621/lucene/src: java/org/apache/lucene/index/codecs/simpletext/ test/org/apache/lucene/index/

Author: rmuir
Date: Tue Nov 15 17:05:56 2011
New Revision: 1202305

URL: http://svn.apache.org/viewvc?rev=1202305&view=rev
Log:
LUCENE-2621: SimpleText term vectors

Added:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsFormat.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsWriter.java   (with props)
Modified:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1202305&r1=1202304&r2=1202305&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Tue Nov 15 17:05:56 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.index.codecs.s
 
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
-import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
 import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
@@ -38,9 +37,7 @@ public final class SimpleTextCodec exten
   private final StoredFieldsFormat storedFields = new SimpleTextStoredFieldsFormat();
   private final SegmentInfosFormat segmentInfos = new SimpleTextSegmentInfosFormat();
   private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
-
-  // nocommit: need a plain-text impl
-  private final TermVectorsFormat vectorsFormat = new DefaultTermVectorsFormat();
+  private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
   // TODO: need a plain-text impl
   private final DocValuesFormat docValues = new DefaultDocValuesFormat();
   

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsFormat.java?rev=1202305&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsFormat.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsFormat.java Tue Nov 15 17:05:56 2011
@@ -0,0 +1,53 @@
+package org.apache.lucene.index.codecs.simpletext;
+
+/**
+ * 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.Set;
+
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.codecs.TermVectorsFormat;
+import org.apache.lucene.index.codecs.TermVectorsReader;
+import org.apache.lucene.index.codecs.TermVectorsWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/**
+ * plain text term vectors format.
+ * <p>
+ * <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+ * @lucene.experimental
+ */
+public class SimpleTextTermVectorsFormat extends TermVectorsFormat {
+
+  @Override
+  public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
+    return new SimpleTextTermVectorsReader(directory, segmentInfo, fieldInfos, context);
+  }
+
+  @Override
+  public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
+    return new SimpleTextTermVectorsWriter(directory, segment, context);
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+    SimpleTextTermVectorsReader.files(dir, info, files);
+  }
+}

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1202305&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java Tue Nov 15 17:05:56 2011
@@ -0,0 +1,526 @@
+package org.apache.lucene.index.codecs.simpletext;
+
+/**
+ * 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.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.FieldsEnum;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.codecs.TermVectorsReader;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.UnicodeUtil;
+
+import static org.apache.lucene.index.codecs.simpletext.SimpleTextTermVectorsWriter.*;
+
+/**
+ * Reads plain-text term vectors.
+ * <p>
+ * <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+ * @lucene.experimental
+ */
+public class SimpleTextTermVectorsReader extends TermVectorsReader {
+  private ArrayList<Long> offsets; /* docid -> offset in .vec file */
+  private IndexInput in;
+  private BytesRef scratch = new BytesRef();
+  private CharsRef scratchUTF16 = new CharsRef();
+  
+  public SimpleTextTermVectorsReader(Directory directory, SegmentInfo si, FieldInfos fieldInfos, IOContext context) throws IOException {
+    boolean success = false;
+    try {
+      in = directory.openInput(IndexFileNames.segmentFileName(si.name, "", VECTORS_EXTENSION), context);
+      success = true;
+    } finally {
+      if (!success) {
+        close();
+      }
+    }
+    readIndex();
+  }
+  
+  // used by clone
+  SimpleTextTermVectorsReader(ArrayList<Long> offsets, IndexInput in) {
+    this.offsets = offsets;
+    this.in = in;
+  }
+  
+  // we don't actually write a .tvx-like index, instead we read the 
+  // vectors file in entirety up-front and save the offsets 
+  // so we can seek to the data later.
+  private void readIndex() throws IOException {
+    offsets = new ArrayList<Long>();
+    while (!scratch.equals(END)) {
+      readLine();
+      if (scratch.startsWith(DOC)) {
+        offsets.add(in.getFilePointer());
+      }
+    }
+  }
+  
+  @Override
+  public Fields get(int doc) throws IOException {
+    // TestTV tests for this in testBadParams... but is this
+    // really guaranteed by the API?
+    if (doc < 0 || doc >= offsets.size()) {
+      throw new IllegalArgumentException("doc id out of range");
+    }
+
+    SortedMap<String,SimpleTVTerms> fields = new TreeMap<String,SimpleTVTerms>();
+    in.seek(offsets.get(doc));
+    readLine();
+    assert scratch.startsWith(NUMFIELDS);
+    int numFields = parseIntAt(NUMFIELDS.length);
+    if (numFields == 0) {
+      return null; // no vectors for this doc
+    }
+    for (int i = 0; i < numFields; i++) {
+      readLine();
+      assert scratch.startsWith(FIELD);
+      int fieldNumber = parseIntAt(FIELD.length);
+      
+      readLine();
+      assert scratch.startsWith(FIELDNAME);
+      String fieldName = readString(FIELDNAME.length, scratch);
+      
+      readLine();
+      assert scratch.startsWith(FIELDPOSITIONS);
+      boolean positions = Boolean.parseBoolean(readString(FIELDPOSITIONS.length, scratch));
+      
+      readLine();
+      assert scratch.startsWith(FIELDOFFSETS);
+      boolean offsets = Boolean.parseBoolean(readString(FIELDOFFSETS.length, scratch));
+      
+      readLine();
+      assert scratch.startsWith(FIELDTERMCOUNT);
+      int termCount = parseIntAt(FIELDTERMCOUNT.length);
+      
+      SimpleTVTerms terms = new SimpleTVTerms();
+      fields.put(fieldName, terms);
+      
+      for (int j = 0; j < termCount; j++) {
+        readLine();
+        assert scratch.startsWith(TERMTEXT);
+        BytesRef term = new BytesRef();
+        int termLength = scratch.length - TERMTEXT.length;
+        term.grow(termLength);
+        term.length = termLength;
+        System.arraycopy(scratch.bytes, scratch.offset+TERMTEXT.length, term.bytes, term.offset, termLength);
+        
+        SimpleTVPostings postings = new SimpleTVPostings();
+        terms.terms.put(term, postings);
+        
+        readLine();
+        assert scratch.startsWith(TERMFREQ);
+        postings.freq = parseIntAt(TERMFREQ.length);
+        
+        if (positions || offsets) {
+          if (positions) {
+            postings.positions = new int[postings.freq];
+          }
+        
+          if (offsets) {
+            postings.startOffsets = new int[postings.freq];
+            postings.endOffsets = new int[postings.freq];
+          }
+          
+          for (int k = 0; k < postings.freq; k++) {
+            if (positions) {
+              readLine();
+              assert scratch.startsWith(POSITION);
+              postings.positions[k] = parseIntAt(POSITION.length);
+            }
+            
+            if (offsets) {
+              readLine();
+              assert scratch.startsWith(STARTOFFSET);
+              postings.startOffsets[k] = parseIntAt(STARTOFFSET.length);
+              
+              readLine();
+              assert scratch.startsWith(ENDOFFSET);
+              postings.endOffsets[k] = parseIntAt(ENDOFFSET.length);
+            }
+          }
+        }
+      }
+    }
+    return new SimpleTVFields(fields);
+  }
+
+  @Override
+  public TermVectorsReader clone() {
+    if (in == null) {
+      throw new AlreadyClosedException("this TermVectorsReader is closed");
+    }
+    return new SimpleTextTermVectorsReader(offsets, (IndexInput) in.clone());
+  }
+  
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.close(in); 
+    } finally {
+      in = null;
+      offsets = null;
+    }
+  }
+  
+  public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+    if (info.getHasVectors()) {
+      files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_EXTENSION));
+    }
+  }
+  
+  private void readLine() throws IOException {
+    SimpleTextUtil.readLine(in, scratch);
+  }
+  
+  private int parseIntAt(int offset) throws IOException {
+    UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+offset, scratch.length-offset, scratchUTF16);
+    return ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
+  }
+  
+  private String readString(int offset, BytesRef scratch) {
+    UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+offset, scratch.length-offset, scratchUTF16);
+    return scratchUTF16.toString();
+  }
+  
+  private class SimpleTVFields extends Fields {
+    private final SortedMap<String,SimpleTVTerms> fields;
+    
+    SimpleTVFields(SortedMap<String,SimpleTVTerms> fields) throws IOException {
+      this.fields = fields;
+    }
+
+    @Override
+    public FieldsEnum iterator() throws IOException {
+      return new FieldsEnum() {
+        private Iterator<Map.Entry<String,SimpleTVTerms>> iterator = fields.entrySet().iterator();
+        private Map.Entry<String,SimpleTVTerms> current = null;
+        
+        @Override
+        public String next() throws IOException {
+          if (!iterator.hasNext()) {
+            return null;
+          } else {
+            current = iterator.next();
+            return current.getKey();
+          }
+        }
+
+        @Override
+        public Terms terms() throws IOException {
+          return current.getValue();
+        }
+      };
+    }
+
+    @Override
+    public Terms terms(String field) throws IOException {
+      return fields.get(field);
+    }
+
+    @Override
+    public int getUniqueFieldCount() throws IOException {
+      return fields.size();
+    }
+  }
+  
+  private static class SimpleTVTerms extends Terms {
+    final SortedMap<BytesRef,SimpleTVPostings> terms;
+    
+    SimpleTVTerms() {
+      terms = new TreeMap<BytesRef,SimpleTVPostings>();
+    }
+    
+    @Override
+    public TermsEnum iterator(TermsEnum reuse) throws IOException {
+      // TODO: reuse
+      return new SimpleTVTermsEnum(terms);
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() throws IOException {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public long getUniqueTermCount() throws IOException {
+      return terms.size();
+    }
+
+    @Override
+    public long getSumTotalTermFreq() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public long getSumDocFreq() throws IOException {
+      return terms.size();
+    }
+
+    @Override
+    public int getDocCount() throws IOException {
+      return 1;
+    }
+  }
+  
+  private static class SimpleTVPostings {
+    private int freq;
+    private int positions[];
+    private int startOffsets[];
+    private int endOffsets[];
+  }
+  
+  private static class SimpleTVTermsEnum extends TermsEnum {
+    SortedMap<BytesRef,SimpleTVPostings> terms;
+    Iterator<Map.Entry<BytesRef,SimpleTextTermVectorsReader.SimpleTVPostings>> iterator;
+    Map.Entry<BytesRef,SimpleTextTermVectorsReader.SimpleTVPostings> current;
+    
+    SimpleTVTermsEnum(SortedMap<BytesRef,SimpleTVPostings> terms) {
+      this.terms = terms;
+      this.iterator = terms.entrySet().iterator();
+    }
+    
+    @Override
+    public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
+      iterator = terms.tailMap(text).entrySet().iterator();
+      if (!iterator.hasNext()) {
+        return SeekStatus.END;
+      } else {
+        return next().equals(text) ? SeekStatus.FOUND : SeekStatus.NOT_FOUND;
+      }
+    }
+
+    @Override
+    public void seekExact(long ord) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BytesRef next() throws IOException {
+      if (!iterator.hasNext()) {
+        return null;
+      } else {
+        current = iterator.next();
+        return current.getKey();
+      }
+    }
+
+    @Override
+    public BytesRef term() throws IOException {
+      return current.getKey();
+    }
+
+    @Override
+    public long ord() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int docFreq() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public long totalTermFreq() throws IOException {
+      return current.getValue().freq;
+    }
+
+    @Override
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
+      // TODO: reuse
+      SimpleTVDocsEnum e = new SimpleTVDocsEnum();
+      e.reset(liveDocs, current.getValue().freq);
+      return e;
+    }
+
+    @Override
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
+      SimpleTVPostings postings = current.getValue();
+      if (postings.positions == null && postings.startOffsets == null) {
+        return null;
+      }
+      // TODO: reuse
+      SimpleTVDocsAndPositionsEnum e = new SimpleTVDocsAndPositionsEnum(postings.startOffsets != null);
+      e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets);
+      return e;
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() throws IOException {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+  }
+  
+  // note: these two enum classes are exactly like the Default impl...
+  private static class SimpleTVDocsEnum extends DocsEnum {
+    private boolean didNext;
+    private int freq;
+    private Bits liveDocs;
+
+    @Override
+    public int freq() {
+      return freq;
+    }
+
+    @Override
+    public int docID() {
+      return 0;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (!didNext && (liveDocs == null || liveDocs.get(0))) {
+        didNext = true;
+        return 0;
+      } else {
+        return NO_MORE_DOCS;
+      }
+    }
+
+    @Override
+    public int advance(int target) {
+      if (!didNext && target == 0) {
+        return nextDoc();
+      } else {
+        return NO_MORE_DOCS;
+      }
+    }
+
+    public void reset(Bits liveDocs, int freq) {
+      this.liveDocs = liveDocs;
+      this.freq = freq;
+      didNext = false;
+    }
+  }
+  
+  private static class SimpleTVDocsAndPositionsEnum extends DocsAndPositionsEnum {
+    private final OffsetAttribute offsetAtt;
+    private boolean didNext;
+    private int nextPos;
+    private Bits liveDocs;
+    private int[] positions;
+    private int[] startOffsets;
+    private int[] endOffsets;
+
+    public SimpleTVDocsAndPositionsEnum(boolean storeOffsets) {
+      if (storeOffsets) {
+        offsetAtt = attributes().addAttribute(OffsetAttribute.class);
+      } else {
+        offsetAtt = null;
+      }
+    }
+
+    public boolean canReuse(boolean storeOffsets) {
+      return storeOffsets == (offsetAtt != null);
+    }
+
+    @Override
+    public int freq() {
+      if (positions != null) {
+        return positions.length;
+      } else {
+        assert startOffsets != null;
+        return startOffsets.length;
+      }
+    }
+
+    @Override
+    public int docID() {
+      return 0;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (!didNext && (liveDocs == null || liveDocs.get(0))) {
+        didNext = true;
+        return 0;
+      } else {
+        return NO_MORE_DOCS;
+      }
+    }
+
+    @Override
+    public int advance(int target) {
+      if (!didNext && target == 0) {
+        return nextDoc();
+      } else {
+        return NO_MORE_DOCS;
+      }
+    }
+
+    public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets) {
+      this.liveDocs = liveDocs;
+      this.positions = positions;
+      this.startOffsets = startOffsets;
+      assert (offsetAtt != null) == (startOffsets != null);
+      this.endOffsets = endOffsets;
+      didNext = false;
+      nextPos = 0;
+    }
+
+    @Override
+    public BytesRef getPayload() {
+      return null;
+    }
+
+    @Override
+    public boolean hasPayload() {
+      return false;
+    }
+
+    @Override
+    public int nextPosition() {
+      assert (positions != null && nextPos < positions.length) ||
+        startOffsets != null && nextPos < startOffsets.length;
+
+      if (startOffsets != null) {
+        offsetAtt.setOffset(startOffsets[nextPos],
+                            endOffsets[nextPos]);
+      }
+      if (positions != null) {
+        return positions[nextPos++];
+      } else {
+        nextPos++;
+        return -1;
+      }
+    }
+  }
+}

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsWriter.java?rev=1202305&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsWriter.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsWriter.java Tue Nov 15 17:05:56 2011
@@ -0,0 +1,179 @@
+package org.apache.lucene.index.codecs.simpletext;
+
+/**
+ * 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.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.codecs.TermVectorsWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Writes plain-text term vectors.
+ * <p>
+ * <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+ * @lucene.experimental
+ */
+public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
+  
+  static final BytesRef END                = new BytesRef("END");
+  static final BytesRef DOC                = new BytesRef("doc ");
+  static final BytesRef NUMFIELDS          = new BytesRef("  numfields ");
+  static final BytesRef FIELD              = new BytesRef("  field ");
+  static final BytesRef FIELDNAME          = new BytesRef("    name ");
+  static final BytesRef FIELDPOSITIONS     = new BytesRef("    positions ");
+  static final BytesRef FIELDOFFSETS       = new BytesRef("    offsets   ");
+  static final BytesRef FIELDTERMCOUNT     = new BytesRef("    numterms ");
+  static final BytesRef TERMTEXT           = new BytesRef("    term ");
+  static final BytesRef TERMFREQ           = new BytesRef("      freq ");
+  static final BytesRef POSITION           = new BytesRef("      position ");
+  static final BytesRef STARTOFFSET        = new BytesRef("        startoffset ");
+  static final BytesRef ENDOFFSET          = new BytesRef("        endoffset ");
+
+  static final String VECTORS_EXTENSION = "vec";
+  
+  private final Directory directory;
+  private final String segment;
+  private IndexOutput out;
+  private int numDocsWritten = 0;
+  private final BytesRef scratch = new BytesRef();
+  private boolean offsets;
+  private boolean positions;
+
+  public SimpleTextTermVectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
+    this.directory = directory;
+    this.segment = segment;
+    out = directory.createOutput(IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION), context);
+  }
+  
+  @Override
+  public void startDocument(int numVectorFields) throws IOException {
+    write(DOC);
+    write(Integer.toString(numDocsWritten));
+    newLine();
+    
+    write(NUMFIELDS);
+    write(Integer.toString(numVectorFields));
+    newLine();
+    numDocsWritten++;
+  }
+
+  @Override
+  public void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets) throws IOException {  
+    write(FIELD);
+    write(Integer.toString(info.number));
+    newLine();
+    
+    write(FIELDNAME);
+    write(info.name);
+    newLine();
+    
+    write(FIELDPOSITIONS);
+    write(Boolean.toString(positions));
+    newLine();
+    
+    write(FIELDOFFSETS);
+    write(Boolean.toString(offsets));
+    newLine();
+    
+    write(FIELDTERMCOUNT);
+    write(Integer.toString(numTerms));
+    newLine();
+    
+    this.positions = positions;
+    this.offsets = offsets;
+  }
+
+  @Override
+  public void startTerm(BytesRef term, int freq) throws IOException {
+    write(TERMTEXT);
+    write(term);
+    newLine();
+    
+    write(TERMFREQ);
+    write(Integer.toString(freq));
+    newLine();
+  }
+
+  @Override
+  public void addPosition(int position, int startOffset, int endOffset) throws IOException {
+    assert positions || offsets;
+    
+    if (positions) {
+      write(POSITION);
+      write(Integer.toString(position));
+      newLine();
+    }
+    
+    if (offsets) {
+      write(STARTOFFSET);
+      write(Integer.toString(startOffset));
+      newLine();
+      
+      write(ENDOFFSET);
+      write(Integer.toString(endOffset));
+      newLine();
+    }
+  }
+
+  @Override
+  public void abort() {
+    try {
+      close();
+    } catch (IOException ignored) {}
+    
+    try {
+      directory.deleteFile(IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION));
+    } catch (IOException ignored) {}
+  }
+
+  @Override
+  public void finish(int numDocs) throws IOException {
+    if (numDocsWritten != numDocs) {
+      throw new RuntimeException("mergeVectors produced an invalid result: mergedDocs is " + numDocs + " but vec numDocs is " + numDocsWritten + " file=" + out.toString() + "; now aborting this merge to prevent index corruption");
+    }
+    write(END);
+    newLine();
+  }
+  
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.close(out);
+    } finally {
+      out = null;
+    }
+  }
+  
+  private void write(String s) throws IOException {
+    SimpleTextUtil.write(out, s, scratch);
+  }
+  
+  private void write(BytesRef bytes) throws IOException {
+    SimpleTextUtil.write(out, bytes);
+  }
+  
+  private void newLine() throws IOException {
+    SimpleTextUtil.writeNewline(out);
+  }
+}

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1202305&r1=1202304&r2=1202305&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Tue Nov 15 17:05:56 2011
@@ -1075,7 +1075,7 @@ public class TestIndexWriter extends Luc
     assertEquals(100, dpEnum.nextPosition());
 
     assertNotNull(termsEnum.next());
-    termsEnum.docsAndPositions(null, dpEnum);
+    dpEnum = termsEnum.docsAndPositions(null, dpEnum);
     assertNotNull(dpEnum);
     assertTrue(dpEnum.nextDoc() != DocsEnum.NO_MORE_DOCS);
     assertEquals(1, dpEnum.freq());

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java?rev=1202305&r1=1202304&r2=1202305&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java Tue Nov 15 17:05:56 2011
@@ -33,6 +33,7 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DefaultTermVectorsReader;
+import org.apache.lucene.index.codecs.TermVectorsReader;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -197,7 +198,7 @@ public class TestTermVectorsReader exten
   }
 
   public void testReader() throws IOException {
-    DefaultTermVectorsReader reader = new DefaultTermVectorsReader(dir, seg, fieldInfos, newIOContext(random));
+    TermVectorsReader reader = Codec.getDefault().termVectorsFormat().vectorsReader(dir, seg, fieldInfos, newIOContext(random));
     for (int j = 0; j < 5; j++) {
       Terms vector = reader.get(j).terms(testFields[0]);
       assertNotNull(vector);
@@ -216,7 +217,7 @@ public class TestTermVectorsReader exten
   }
 
   public void testPositionReader() throws IOException {
-    DefaultTermVectorsReader reader = new DefaultTermVectorsReader(dir, seg, fieldInfos, newIOContext(random));
+    TermVectorsReader reader = Codec.getDefault().termVectorsFormat().vectorsReader(dir, seg, fieldInfos, newIOContext(random));
     BytesRef[] terms;
     Terms vector = reader.get(0).terms(testFields[0]);
     assertNotNull(vector);
@@ -269,7 +270,7 @@ public class TestTermVectorsReader exten
   }
 
   public void testOffsetReader() throws IOException {
-    DefaultTermVectorsReader reader = new DefaultTermVectorsReader(dir, seg, fieldInfos, newIOContext(random));
+    TermVectorsReader reader = Codec.getDefault().termVectorsFormat().vectorsReader(dir, seg, fieldInfos, newIOContext(random));
     Terms vector = reader.get(0).terms(testFields[0]);
     assertNotNull(vector);
     TermsEnum termsEnum = vector.iterator(null);
@@ -311,9 +312,9 @@ public class TestTermVectorsReader exten
    * Make sure exceptions and bad params are handled appropriately
    */
   public void testBadParams() throws IOException {
-    DefaultTermVectorsReader reader = null;
+    TermVectorsReader reader = null;
     try {
-      reader = new DefaultTermVectorsReader(dir, seg, fieldInfos, newIOContext(random));
+      reader = Codec.getDefault().termVectorsFormat().vectorsReader(dir, seg, fieldInfos, newIOContext(random));
       //Bad document number, good field number
       reader.get(50);
       fail();
@@ -322,7 +323,7 @@ public class TestTermVectorsReader exten
     } finally {
       reader.close();
     }
-    reader = new DefaultTermVectorsReader(dir, seg, fieldInfos, newIOContext(random));
+    reader = Codec.getDefault().termVectorsFormat().vectorsReader(dir, seg, fieldInfos, newIOContext(random));
     //good document number, bad field
     Terms vector = reader.get(0).terms("f50");
     assertNull(vector);