You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by pn...@apache.org on 2014/09/21 18:53:29 UTC

[3/7] Cleanup of codes, mostly SimpleText in this commit

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/cf1df6be/src/Lucene.Net.Codecs/SimpleText/SimpleTextStoredFieldsWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/SimpleText/SimpleTextStoredFieldsWriter.cs b/src/Lucene.Net.Codecs/SimpleText/SimpleTextStoredFieldsWriter.cs
index 5c5346d..79d943f 100644
--- a/src/Lucene.Net.Codecs/SimpleText/SimpleTextStoredFieldsWriter.cs
+++ b/src/Lucene.Net.Codecs/SimpleText/SimpleTextStoredFieldsWriter.cs
@@ -1,6 +1,4 @@
-package org.apache.lucene.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.
@@ -17,181 +15,226 @@ package org.apache.lucene.codecs.simpletext;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
-import org.apache.lucene.codecs.StoredFieldsWriter;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexableField;
-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 stored fields.
- * <p>
- * <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
- * @lucene.experimental
- */
-public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
-  private int numDocsWritten = 0;
-  private final Directory directory;
-  private final String segment;
-  private IndexOutput out;
-  
-  final static String FIELDS_EXTENSION = "fld";
-  
-  final static BytesRef TYPE_STRING = new BytesRef("string");
-  final static BytesRef TYPE_BINARY = new BytesRef("binary");
-  final static BytesRef TYPE_INT    = new BytesRef("int");
-  final static BytesRef TYPE_LONG   = new BytesRef("long");
-  final static BytesRef TYPE_FLOAT  = new BytesRef("float");
-  final static BytesRef TYPE_DOUBLE = new BytesRef("double");
-
-  final static BytesRef END      = new BytesRef("END");
-  final static BytesRef DOC      = new BytesRef("doc ");
-  final static BytesRef NUM      = new BytesRef("  numfields ");
-  final static BytesRef FIELD    = new BytesRef("  field ");
-  final static BytesRef NAME     = new BytesRef("    name ");
-  final static BytesRef TYPE     = new BytesRef("    type ");
-  final static BytesRef VALUE    = new BytesRef("    value ");
-  
-  private final BytesRef scratch = new BytesRef();
-  
-  public SimpleTextStoredFieldsWriter(Directory directory, String segment, IOContext context)  {
-    this.directory = directory;
-    this.segment = segment;
-    bool success = false;
-    try {
-      out = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
-      success = true;
-    } finally {
-      if (!success) {
-        abort();
-      }
-    }
-  }
-
-  @Override
-  public void startDocument(int numStoredFields)  {
-    write(DOC);
-    write(Integer.toString(numDocsWritten));
-    newLine();
-    
-    write(NUM);
-    write(Integer.toString(numStoredFields));
-    newLine();
-    
-    numDocsWritten++;
-  }
-
-  @Override
-  public void writeField(FieldInfo info, IndexableField field)  {
-    write(FIELD);
-    write(Integer.toString(info.number));
-    newLine();
-    
-    write(NAME);
-    write(field.name());
-    newLine();
-    
-    write(TYPE);
-    final Number n = field.numericValue();
-
-    if (n != null) {
-      if (n instanceof Byte || n instanceof Short || n instanceof Integer) {
-        write(TYPE_INT);
-        newLine();
-          
-        write(VALUE);
-        write(Integer.toString(n.intValue()));
-        newLine();
-      } else if (n instanceof Long) {
-        write(TYPE_LONG);
-        newLine();
-
-        write(VALUE);
-        write(Long.toString(n.longValue()));
-        newLine();
-      } else if (n instanceof Float) {
-        write(TYPE_FLOAT);
-        newLine();
-          
-        write(VALUE);
-        write(Float.toString(n.floatValue()));
-        newLine();
-      } else if (n instanceof Double) {
-        write(TYPE_DOUBLE);
-        newLine();
-          
-        write(VALUE);
-        write(Double.toString(n.doubleValue()));
-        newLine();
-      } else {
-        throw new IllegalArgumentException("cannot store numeric type " + n.getClass());
-      }
-    } else { 
-      BytesRef bytes = field.binaryValue();
-      if (bytes != null) {
-        write(TYPE_BINARY);
-        newLine();
-        
-        write(VALUE);
-        write(bytes);
-        newLine();
-      } else if (field.stringValue() == null) {
-        throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
-      } else {
-        write(TYPE_STRING);
-        newLine();
-        
-        write(VALUE);
-        write(field.stringValue());
-        newLine();
-      }
-    }
-  }
-
-  @Override
-  public void abort() {
-    try {
-      close();
-    } catch (Throwable ignored) {}
-    IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
-  }
-
-  @Override
-  public void finish(FieldInfos fis, int numDocs)  {
-    if (numDocsWritten != numDocs) {
-      throw new RuntimeException("mergeFields produced an invalid result: docCount is " + numDocs 
-          + " but only saw " + numDocsWritten + " file=" + out.toString() + "; now aborting this merge to prevent index corruption");
-    }
-    write(END);
-    newLine();
-    SimpleTextUtil.writeChecksum(out, scratch);
-  }
-
-  @Override
-  public void close()  {
-    try {
-      IOUtils.close(out);
-    } finally {
-      out = null;
+namespace Lucene.Net.Codecs.SimpleText
+{
+
+    using System;
+
+	using FieldInfo = Index.FieldInfo;
+	using FieldInfos = Index.FieldInfos;
+	using IndexFileNames = Index.IndexFileNames;
+	using IndexableField = Index.IndexableField;
+	using Directory = Store.Directory;
+	using IOContext = Store.IOContext;
+	using IndexOutput = Store.IndexOutput;
+	using BytesRef = Util.BytesRef;
+	using IOUtils = Util.IOUtils;
+
+    /// <summary>
+    /// Writes plain-text stored fields.
+    /// <para>
+    /// <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+    /// @lucene.experimental
+    /// </para>
+    /// </summary>
+    public class SimpleTextStoredFieldsWriter : StoredFieldsWriter
+    {
+        private int _numDocsWritten;
+        private readonly Directory _directory;
+        private readonly string _segment;
+        private IndexOutput _output;
+
+        internal const string FIELDS_EXTENSION = "fld";
+
+        internal static readonly BytesRef TYPE_STRING = new BytesRef("string");
+        internal static readonly BytesRef TYPE_BINARY = new BytesRef("binary");
+        internal static readonly BytesRef TYPE_INT = new BytesRef("int");
+        internal static readonly BytesRef TYPE_LONG = new BytesRef("long");
+        internal static readonly BytesRef TYPE_FLOAT = new BytesRef("float");
+        internal static readonly BytesRef TYPE_DOUBLE = new BytesRef("double");
+
+        internal static readonly BytesRef END = new BytesRef("END");
+        internal static readonly BytesRef DOC = new BytesRef("doc ");
+        internal static readonly BytesRef NUM = new BytesRef("  numfields ");
+        internal static readonly BytesRef FIELD = new BytesRef("  field ");
+        internal static readonly BytesRef NAME = new BytesRef("    name ");
+        internal static readonly BytesRef TYPE = new BytesRef("    type ");
+        internal static readonly BytesRef VALUE = new BytesRef("    value ");
+
+        private readonly BytesRef _scratch = new BytesRef();
+
+        public SimpleTextStoredFieldsWriter(Directory directory, string segment, IOContext context)
+        {
+            _directory = directory;
+            _segment = segment;
+            var success = false;
+            try
+            {
+                _output = directory.CreateOutput(IndexFileNames.SegmentFileName(segment, "", FIELDS_EXTENSION), context);
+                success = true;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    Abort();
+                }
+            }
+        }
+
+        public override void StartDocument(int numStoredFields)
+	    {
+	        Write(DOC);
+	        Write(Convert.ToString(_numDocsWritten));
+	        NewLine();
+
+	        Write(NUM);
+	        Write(Convert.ToString(numStoredFields));
+	        NewLine();
+
+	        _numDocsWritten++;
+	    }
+
+        public override void WriteField(FieldInfo info, IndexableField field)
+        {
+            Write(FIELD);
+            Write(Convert.ToString(info.Number));
+            NewLine();
+
+            Write(NAME);
+            Write(field.Name());
+            NewLine();
+
+            Write(TYPE);
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final Number n = field.numericValue();
+            Number n = field.NumericValue;
+
+            if (n != null)
+            {
+                if (n is sbyte? || n is short? || n is int?)
+                {
+                    Write(TYPE_INT);
+                    NewLine();
+
+                    Write(VALUE);
+                    Write(Convert.ToString((int) n));
+                    NewLine();
+                }
+                else if (n is long?)
+                {
+                    Write(TYPE_LONG);
+                    NewLine();
+
+                    Write(VALUE);
+                    Write(Convert.ToString((long) n));
+                    NewLine();
+                }
+                else if (n is float?)
+                {
+                    Write(TYPE_FLOAT);
+                    NewLine();
+
+                    Write(VALUE);
+                    Write(Convert.ToString((float) n));
+                    NewLine();
+                }
+                else if (n is double?)
+                {
+                    Write(TYPE_DOUBLE);
+                    NewLine();
+
+                    Write(VALUE);
+                    Write(Convert.ToString((double) n));
+                    NewLine();
+                }
+                else
+                {
+                    throw new System.ArgumentException("cannot store numeric type " + n.GetType());
+                }
+            }
+            else
+            {
+                BytesRef bytes = field.binaryValue();
+                if (bytes != null)
+                {
+                    Write(TYPE_BINARY);
+                    NewLine();
+
+                    Write(VALUE);
+                    Write(bytes);
+                    NewLine();
+                }
+                else if (field.stringValue() == null)
+                {
+                    throw new ArgumentException("field " + field.Name() +
+                                                       " is stored but does not have binaryValue, stringValue nor numericValue");
+                }
+                else
+                {
+                    Write(TYPE_STRING);
+                    NewLine();
+
+                    Write(VALUE);
+                    Write(field.StringValue());
+                    NewLine();
+                }
+            }
+        }
+
+        public override sealed void Abort()
+	    {
+	        try
+	        {
+	            Dispose();
+	        }
+	        finally
+	        {
+	            IOUtils.DeleteFilesIgnoringExceptions(_directory,
+	                IndexFileNames.SegmentFileName(_segment, "", FIELDS_EXTENSION));
+	        }
+	    }
+
+	    public override void Finish(FieldInfos fis, int numDocs)
+	    {
+	        if (_numDocsWritten != numDocs)
+	        {
+	            throw new Exception("mergeFields produced an invalid result: docCount is " + numDocs + " but only saw " +
+	                                _numDocsWritten + " file=" + _output +
+	                                "; now aborting this merge to prevent index corruption");
+	        }
+	        Write(END);
+	        NewLine();
+	        SimpleTextUtil.WriteChecksum(_output, _scratch);
+	    }
+
+	    protected override void Dispose(bool disposing)
+	    {
+	        if (disposing) return;
+	        try
+	        {
+	            IOUtils.Close(_output);
+	        }
+	        finally
+	        {
+	            _output = null;
+	        }
+	    }
+
+        private void Write(string s)
+        {
+            SimpleTextUtil.Write(_output, s, _scratch);
+        }
+
+        private void Write(BytesRef bytes)
+        {
+            SimpleTextUtil.Write(_output, bytes);
+        }
+
+        private void NewLine()
+        {
+            SimpleTextUtil.WriteNewline(_output);
+        }
     }
-  }
-  
-  private void write(String s)  {
-    SimpleTextUtil.write(out, s, scratch);
-  }
-  
-  private void write(BytesRef bytes)  {
-    SimpleTextUtil.write(out, bytes);
-  }
-  
-  private void newLine()  {
-    SimpleTextUtil.writeNewline(out);
-  }
-}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/cf1df6be/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsFormat.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsFormat.cs b/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsFormat.cs
index 4b332e5..d9829ab 100644
--- a/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsFormat.cs
+++ b/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsFormat.cs
@@ -1,6 +1,4 @@
-package org.apache.lucene.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.
@@ -17,31 +15,32 @@ package org.apache.lucene.codecs.simpletext;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
-import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.TermVectorsReader;
-import org.apache.lucene.codecs.TermVectorsWriter;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
+namespace Lucene.Net.Codecs.SimpleText
+{
+    using FieldInfos = Index.FieldInfos;
+    using SegmentInfo = Index.SegmentInfo;
+    using Directory = Store.Directory;
+    using IOContext = 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 {
+    /// <summary>
+    /// plain text term vectors format.
+    /// <para>
+    /// <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+    /// @lucene.experimental
+    /// </para>
+    /// </summary>
+    public class SimpleTextTermVectorsFormat : TermVectorsFormat
+    {
+        public override TermVectorsReader VectorsReader(Directory directory, SegmentInfo segmentInfo,
+            FieldInfos fieldInfos, IOContext context)
+        {
+            return new SimpleTextTermVectorsReader(directory, segmentInfo, context);
+        }
 
-  @Override
-  public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context)  {
-    return new SimpleTextTermVectorsReader(directory, segmentInfo, context);
-  }
 
-  @Override
-  public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context)  {
-    return new SimpleTextTermVectorsWriter(directory, segmentInfo.name, context);
-  }
-}
+        public override TermVectorsWriter VectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context)
+        {
+            return new SimpleTextTermVectorsWriter(directory, segmentInfo.Name, context);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/cf1df6be/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsReader.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsReader.cs b/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsReader.cs
index 634e4ae..09e4a30 100644
--- a/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsReader.cs
+++ b/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsReader.cs
@@ -1,558 +1,617 @@
-package org.apache.lucene.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.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.lucene.codecs.TermVectorsReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.Fields;
-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.store.AlreadyClosedException;
-import org.apache.lucene.store.BufferedChecksumIndexInput;
-import org.apache.lucene.store.ChecksumIndexInput;
-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.StringHelper;
-import org.apache.lucene.util.UnicodeUtil;
-import static org.apache.lucene.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 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, IOContext context)  {
-    bool success = false;
-    try {
-      in = directory.openInput(IndexFileNames.segmentFileName(si.name, "", VECTORS_EXTENSION), context);
-      success = true;
-    } finally {
-      if (!success) {
-        try {
-          close();
-        } catch (Throwable t) {} // ensure we throw our original exception
-      }
-    }
-    readIndex(si.getDocCount());
-  }
-  
-  // used by clone
-  SimpleTextTermVectorsReader(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(int maxDoc)  {
-    ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
-    offsets = new long[maxDoc];
-    int upto = 0;
-    while (!scratch.equals(END)) {
-      SimpleTextUtil.readLine(input, scratch);
-      if (StringHelper.startsWith(scratch, DOC)) {
-        offsets[upto] = input.getFilePointer();
-        upto++;
-      }
-    }
-    SimpleTextUtil.checkFooter(input);
-    Debug.Assert( upto == offsets.length;
-  }
-  
-  @Override
-  public Fields get(int doc)  {
-    SortedMap<String,SimpleTVTerms> fields = new TreeMap<>();
-    in.seek(offsets[doc]);
-    readLine();
-    Debug.Assert( StringHelper.startsWith(scratch, NUMFIELDS);
-    int numFields = parseIntAt(NUMFIELDS.length);
-    if (numFields == 0) {
-      return null; // no vectors for this doc
-    }
-    for (int i = 0; i < numFields; i++) {
-      readLine();
-      Debug.Assert( StringHelper.startsWith(scratch, FIELD);
-      // skip fieldNumber:
-      parseIntAt(FIELD.length);
-      
-      readLine();
-      Debug.Assert( StringHelper.startsWith(scratch, FIELDNAME);
-      String fieldName = readString(FIELDNAME.length, scratch);
-      
-      readLine();
-      Debug.Assert( StringHelper.startsWith(scratch, FIELDPOSITIONS);
-      bool positions = bool.parsebool(readString(FIELDPOSITIONS.length, scratch));
-      
-      readLine();
-      Debug.Assert( StringHelper.startsWith(scratch, FIELDOFFSETS);
-      bool offsets = bool.parsebool(readString(FIELDOFFSETS.length, scratch));
-      
-      readLine();
-      Debug.Assert( StringHelper.startsWith(scratch, FIELDPAYLOADS);
-      bool payloads = bool.parsebool(readString(FIELDPAYLOADS.length, scratch));
-      
-      readLine();
-      Debug.Assert( StringHelper.startsWith(scratch, FIELDTERMCOUNT);
-      int termCount = parseIntAt(FIELDTERMCOUNT.length);
-      
-      SimpleTVTerms terms = new SimpleTVTerms(offsets, positions, payloads);
-      fields.put(fieldName, terms);
-      
-      for (int j = 0; j < termCount; j++) {
-        readLine();
-        Debug.Assert( StringHelper.startsWith(scratch, 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();
-        Debug.Assert( StringHelper.startsWith(scratch, TERMFREQ);
-        postings.freq = parseIntAt(TERMFREQ.length);
-        
-        if (positions || offsets) {
-          if (positions) {
-            postings.positions = new int[postings.freq];
-            if (payloads) {
-              postings.payloads = new BytesRef[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();
-              Debug.Assert( StringHelper.startsWith(scratch, POSITION);
-              postings.positions[k] = parseIntAt(POSITION.length);
-              if (payloads) {
-                readLine();
-                Debug.Assert( StringHelper.startsWith(scratch, PAYLOAD);
-                if (scratch.length - PAYLOAD.length == 0) {
-                  postings.payloads[k] = null;
-                } else {
-                  byte payloadBytes[] = new byte[scratch.length - PAYLOAD.length];
-                  System.arraycopy(scratch.bytes, scratch.offset+PAYLOAD.length, payloadBytes, 0, payloadBytes.length);
-                  postings.payloads[k] = new BytesRef(payloadBytes);
+/*
+* 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.
+*/
+
+namespace Lucene.Net.Codecs.SimpleText
+{
+    
+    using System;
+    using System.Diagnostics;
+    using System.Collections.Generic;
+
+	using DocsAndPositionsEnum = Index.DocsAndPositionsEnum;
+	using DocsEnum = Index.DocsEnum;
+	using Fields = Index.Fields;
+	using IndexFileNames = Index.IndexFileNames;
+	using SegmentInfo = Index.SegmentInfo;
+	using Terms = Index.Terms;
+	using TermsEnum = Index.TermsEnum;
+	using AlreadyClosedException = Store.AlreadyClosedException;
+	using BufferedChecksumIndexInput = Store.BufferedChecksumIndexInput;
+	using ChecksumIndexInput = Store.ChecksumIndexInput;
+	using Directory = Store.Directory;
+	using IOContext = Store.IOContext;
+	using IndexInput = Store.IndexInput;
+	using ArrayUtil = Util.ArrayUtil;
+	using Bits = Util.Bits;
+	using BytesRef = Util.BytesRef;
+	using CharsRef = Util.CharsRef;
+	using IOUtils = Util.IOUtils;
+	using StringHelper = Util.StringHelper;
+	using UnicodeUtil = Util.UnicodeUtil;
+
+    //JAVA TO C# CONVERTER TODO TASK: This Java 'import static' statement cannot be converted to .NET:
+	//import static Lucene.Net.Codecs.SimpleText.SimpleTextTermVectorsWriter.*;
+
+    /// <summary>
+    /// Reads plain-text term vectors.
+    /// <para>
+    /// <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+    /// @lucene.experimental
+    /// </para>
+    /// </summary>
+    public class SimpleTextTermVectorsReader : TermVectorsReader
+    {
+        private long[] _offsets; // docid -> offset in .vec file
+        private IndexInput _input;
+        private readonly BytesRef _scratch = new BytesRef();
+        private readonly CharsRef _scratchUtf16 = new CharsRef();
+
+        public SimpleTextTermVectorsReader(Directory directory, SegmentInfo si, IOContext context)
+        {
+            bool success = false;
+            try
+            {
+                _input = directory.OpenInput(IndexFileNames.SegmentFileName(si.Name, "", VECTORS_EXTENSION), context);
+                success = true;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    try
+                    {
+                        Dispose();
+                    } // ensure we throw our original exception
+                    catch (Exception)
+                    {
+                    }
                 }
-              }
-            }
-            
-            if (offsets) {
-              readLine();
-              Debug.Assert( StringHelper.startsWith(scratch, STARTOFFSET);
-              postings.startOffsets[k] = parseIntAt(STARTOFFSET.length);
-              
-              readLine();
-              Debug.Assert( StringHelper.startsWith(scratch, ENDOFFSET);
-              postings.endOffsets[k] = parseIntAt(ENDOFFSET.length);
-            }
-          }
+            }
+            ReadIndex(si.DocCount);
         }
-      }
-    }
-    return new SimpleTVFields(fields);
-  }
 
-  @Override
-  public TermVectorsReader clone() {
-    if (in == null) {
-      throw new AlreadyClosedException("this TermVectorsReader is closed");
-    }
-    return new SimpleTextTermVectorsReader(offsets, in.clone());
-  }
-  
-  @Override
-  public void close()  {
-    try {
-      IOUtils.close(in); 
-    } finally {
-      in = null;
-      offsets = null;
-    }
-  }
-
-  private void readLine()  {
-    SimpleTextUtil.readLine(in, scratch);
-  }
-  
-  private int parseIntAt(int offset) {
-    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) {
-      this.fields = fields;
-    }
+        // used by clone
+        internal SimpleTextTermVectorsReader(long[] offsets, IndexInput @in)
+        {
+            this._offsets = offsets;
+            _input = @in;
+        }
 
-    @Override
-    public Iterator<String> iterator() {
-      return Collections.unmodifiableSet(fields.keySet()).iterator();
-    }
+        // 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(int maxDoc)
+        {
+            ChecksumIndexInput input = new BufferedChecksumIndexInput(_input);
+            _offsets = new long[maxDoc];
+            int upto = 0;
+            while (!_scratch.Equals(END))
+            {
+                SimpleTextUtil.ReadLine(input, _scratch);
+                if (StringHelper.StartsWith(_scratch, DOC))
+                {
+                    _offsets[upto] = input.FilePointer;
+                    upto++;
+                }
+            }
+            SimpleTextUtil.CheckFooter(input);
+            Debug.Assert(upto == _offsets.Length);
+        }
 
-    @Override
-    public Terms terms(String field)  {
-      return fields.get(field);
-    }
+        public override Fields Get(int doc)
+        {
+            SortedMap<string, SimpleTVTerms> fields = new SortedDictionary<string, SimpleTVTerms>();
+            _input.Seek(_offsets[doc]);
+            ReadLine();
+            Debug.Assert(StringHelper.StartsWith(_scratch, NUMFIELDS));
+            int numFields = ParseIntAt(NUMFIELDS.length);
+            if (numFields == 0)
+            {
+                return null; // no vectors for this doc
+            }
+            for (int i = 0; i < numFields; i++)
+            {
+                ReadLine();
+                Debug.Assert(StringHelper.StartsWith(_scratch, FIELD));
+                // skip fieldNumber:
+                ParseIntAt(FIELD.length);
+
+                ReadLine();
+                Debug.Assert(StringHelper.StartsWith(_scratch, FIELDNAME));
+                string fieldName = ReadString(FIELDNAME.length, _scratch);
+
+                ReadLine();
+                Debug.Assert(StringHelper.StartsWith(_scratch, FIELDPOSITIONS));
+                bool positions = Convert.ToBoolean(ReadString(FIELDPOSITIONS.length, _scratch));
+
+                ReadLine();
+                Debug.Assert(StringHelper.StartsWith(_scratch, FIELDOFFSETS));
+                bool offsets = Convert.ToBoolean(ReadString(FIELDOFFSETS.length, _scratch));
+
+                ReadLine();
+                Debug.Assert(StringHelper.StartsWith(_scratch, FIELDPAYLOADS));
+                bool payloads = Convert.ToBoolean(ReadString(FIELDPAYLOADS.length, _scratch));
+
+                ReadLine();
+                Debug.Assert(StringHelper.StartsWith(_scratch, FIELDTERMCOUNT));
+                int termCount = ParseIntAt(FIELDTERMCOUNT.length);
+
+                SimpleTVTerms terms = new SimpleTVTerms(offsets, positions, payloads);
+                fields.put(fieldName, terms);
+
+                for (int j = 0; j < termCount; j++)
+                {
+                    ReadLine();
+                    Debug.Assert(StringHelper.StartsWith(_scratch, TERMTEXT));
+                    BytesRef term = new BytesRef();
+                    int termLength = _scratch.length - TERMTEXT.length;
+                    term.grow(termLength);
+                    term.length = termLength;
+                    Array.Copy(_scratch.bytes, _scratch.offset + TERMTEXT.length, term.bytes, term.offset, termLength);
+
+                    SimpleTVPostings postings = new SimpleTVPostings();
+                    terms.TERMS.put(term, postings);
+
+                    ReadLine();
+                    Debug.Assert(StringHelper.StartsWith(_scratch, TERMFREQ));
+                    postings.FREQ = ParseIntAt(TERMFREQ.length);
+
+                    if (positions || offsets)
+                    {
+                        if (positions)
+                        {
+                            postings.POSITIONS = new int[postings.FREQ];
+                            if (payloads)
+                            {
+                                postings.PAYLOADS = new BytesRef[postings.FREQ];
+                            }
+                        }
+
+                        if (offsets)
+                        {
+                            postings.START_OFFSETS = new int[postings.FREQ];
+                            postings.END_OFFSETS = new int[postings.FREQ];
+                        }
+
+                        for (int k = 0; k < postings.FREQ; k++)
+                        {
+                            if (positions)
+                            {
+                                ReadLine();
+                                Debug.Assert(StringHelper.StartsWith(_scratch, POSITION));
+                                postings.POSITIONS[k] = ParseIntAt(POSITION.length);
+                                if (payloads)
+                                {
+                                    ReadLine();
+                                    Debug.Assert(StringHelper.StartsWith(_scratch, PAYLOAD));
+                                    if (_scratch.length - PAYLOAD.length == 0)
+                                    {
+                                        postings.PAYLOADS[k] = null;
+                                    }
+                                    else
+                                    {
+                                        sbyte[] payloadBytes = new sbyte[_scratch.length - PAYLOAD.length];
+                                        Array.Copy(_scratch.bytes, _scratch.offset + PAYLOAD.length, payloadBytes, 0,
+                                            payloadBytes.Length);
+                                        postings.PAYLOADS[k] = new BytesRef(payloadBytes);
+                                    }
+                                }
+                            }
+
+                            if (offsets)
+                            {
+                                ReadLine();
+                                Debug.Assert(StringHelper.StartsWith(_scratch, STARTOFFSET));
+                                postings.START_OFFSETS[k] = ParseIntAt(STARTOFFSET.length);
+
+                                ReadLine();
+                                Debug.Assert(StringHelper.StartsWith(_scratch, ENDOFFSET));
+                                postings.END_OFFSETS[k] = ParseIntAt(ENDOFFSET.length);
+                            }
+                        }
+                    }
+                }
+            }
+            return new SimpleTVFields(this, fields);
+        }
 
-    @Override
-    public int size() {
-      return fields.size();
-    }
-  }
-  
-  private static class SimpleTVTerms extends Terms {
-    final SortedMap<BytesRef,SimpleTVPostings> terms;
-    final bool hasOffsets;
-    final bool hasPositions;
-    final bool hasPayloads;
-    
-    SimpleTVTerms(bool hasOffsets, bool hasPositions, bool hasPayloads) {
-      this.hasOffsets = hasOffsets;
-      this.hasPositions = hasPositions;
-      this.hasPayloads = hasPayloads;
-      terms = new TreeMap<>();
-    }
-    
-    @Override
-    public TermsEnum iterator(TermsEnum reuse)  {
-      // TODO: reuse
-      return new SimpleTVTermsEnum(terms);
-    }
+        public override TermVectorsReader Clone()
+        {
+            if (_input == null)
+            {
+                throw new AlreadyClosedException("this TermVectorsReader is closed");
+            }
+            return new SimpleTextTermVectorsReader(_offsets, _input.Clone());
+        }
 
-    @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
+        protected override void Dispose(bool disposing)
+        {
+            if (disposing) return;
 
-    @Override
-    public long size()  {
-      return terms.size();
-    }
+            try
+            {
+                IOUtils.Close(_input);
+            }
+            finally
+            {
+                _input = null;
+                _offsets = null;
+            }
+        }
 
-    @Override
-    public long getSumTotalTermFreq()  {
-      return -1;
-    }
+        private void ReadLine()
+        {
+            SimpleTextUtil.ReadLine(_input, _scratch);
+        }
 
-    @Override
-    public long getSumDocFreq()  {
-      return terms.size();
-    }
+        private int ParseIntAt(int offset)
+        {
+            UnicodeUtil.UTF8toUTF16(_scratch.Bytes, _scratch.Offset + offset, _scratch.Length - offset, _scratchUtf16);
+            return ArrayUtil.ParseInt(_scratchUtf16.Chars, 0, _scratchUtf16.length);
+        }
 
-    @Override
-    public int getDocCount()  {
-      return 1;
-    }
+        private string ReadString(int offset, BytesRef scratch)
+        {
+            UnicodeUtil.UTF8toUTF16(scratch.Bytes, scratch.Offset + offset, scratch.Length - offset, _scratchUtf16);
+            return _scratchUtf16.ToString();
+        }
 
-    @Override
-    public bool hasFreqs() {
-      return true;
-    }
+        public override long RamBytesUsed()
+        {
+            return 0;
+        }
 
-    @Override
-    public bool hasOffsets() {
-      return hasOffsets;
-    }
+        public override void CheckIntegrity()
+        {
+        }
 
-    @Override
-    public bool hasPositions() {
-      return hasPositions;
-    }
-    
-    @Override
-    public bool hasPayloads() {
-      return hasPayloads;
-    }
-  }
-  
-  private static class SimpleTVPostings {
-    private int freq;
-    private int positions[];
-    private int startOffsets[];
-    private int endOffsets[];
-    private BytesRef payloads[];
-  }
-  
-  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)  {
-      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)  {
-      throw new UnsupportedOperationException();
-    }
+        private class SimpleTVFields : Fields
+        {
+            private readonly SimpleTextTermVectorsReader _outerInstance;
+            private readonly SortedDictionary<string, SimpleTVTerms> _fields;
 
-    @Override
-    public BytesRef next()  {
-      if (!iterator.hasNext()) {
-        return null;
-      } else {
-        current = iterator.next();
-        return current.getKey();
-      }
-    }
+            internal SimpleTVFields(SimpleTextTermVectorsReader outerInstance, SortedDictionary<string, SimpleTVTerms> fields)
+            {
+                _outerInstance = outerInstance;
+                _fields = fields;
+            }
 
-    @Override
-    public BytesRef term()  {
-      return current.getKey();
-    }
+            public override IEnumerator<string> GetEnumerator()
+            {
+                return _fields.Keys.GetEnumerator();
+            }
 
-    @Override
-    public long ord()  {
-      throw new UnsupportedOperationException();
-    }
+            public override Terms Terms(string field)
+            {
+                return _fields[field];
+            }
 
-    @Override
-    public int docFreq()  {
-      return 1;
-    }
+            public override int Size()
+            {
+                return _fields.Count;
+            }
+        }
 
-    @Override
-    public long totalTermFreq()  {
-      return current.getValue().freq;
-    }
+        private class SimpleTVTerms : Terms
+        {
+            internal readonly SortedDictionary<BytesRef, SimpleTVPostings> TERMS;
+            private readonly bool _hasOffsetsRenamed;
+            private readonly bool _hasPositionsRenamed;
+            private readonly bool _hasPayloadsRenamed;
+
+            internal SimpleTVTerms(bool hasOffsets, bool hasPositions, bool hasPayloads)
+            {
+                _hasOffsetsRenamed = hasOffsets;
+                _hasPositionsRenamed = hasPositions;
+                _hasPayloadsRenamed = hasPayloads;
+                TERMS = new SortedDictionary<BytesRef, SimpleTVPostings>();
+            }
 
-    @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags)  {
-      // TODO: reuse
-      SimpleTVDocsEnum e = new SimpleTVDocsEnum();
-      e.reset(liveDocs, (flags & DocsEnum.FLAG_FREQS) == 0 ? 1 : current.getValue().freq);
-      return e;
-    }
+            public override TermsEnum Iterator(TermsEnum reuse)
+            {
+                // TODO: reuse
+                return new SimpleTVTermsEnum(TERMS);
+            }
 
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags)  {
-      SimpleTVPostings postings = current.getValue();
-      if (postings.positions == null && postings.startOffsets == null) {
-        return null;
-      }
-      // TODO: reuse
-      SimpleTVDocsAndPositionsEnum e = new SimpleTVDocsAndPositionsEnum();
-      e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads);
-      return e;
-    }
+            public override IComparer<BytesRef> Comparator
+            {
+                get { return BytesRef.UTF8SortedAsUnicodeComparer; }
+            }
 
-    @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
-  }
-  
-  // note: these two enum classes are exactly like the Default impl...
-  private static class SimpleTVDocsEnum extends DocsEnum {
-    private bool didNext;
-    private int doc = -1;
-    private int freq;
-    private Bits liveDocs;
-
-    @Override
-    public int freq()  {
-      Debug.Assert( freq != -1;
-      return freq;
-    }
+            public override long Size()
+            {
+                return TERMS.Count;
+            }
 
-    @Override
-    public int docID() {
-      return doc;
-    }
+            public override long SumTotalTermFreq
+            {
+                get { return -1; }
+            }
 
-    @Override
-    public int nextDoc() {
-      if (!didNext && (liveDocs == null || liveDocs.get(0))) {
-        didNext = true;
-        return (doc = 0);
-      } else {
-        return (doc = NO_MORE_DOCS);
-      }
-    }
+            public override long SumDocFreq
+            {
+                get { return TERMS.Count; }
+            }
 
-    @Override
-    public int advance(int target)  {
-      return slowAdvance(target);
-    }
+            public override int DocCount
+            {
+                get { return 1; }
+            }
 
-    public void reset(Bits liveDocs, int freq) {
-      this.liveDocs = liveDocs;
-      this.freq = freq;
-      this.doc = -1;
-      didNext = false;
-    }
-    
-    @Override
-    public long cost() {
-      return 1;
-    }
-  }
-  
-  private static class SimpleTVDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    private bool didNext;
-    private int doc = -1;
-    private int nextPos;
-    private Bits liveDocs;
-    private int[] positions;
-    private BytesRef[] payloads;
-    private int[] startOffsets;
-    private int[] endOffsets;
-
-    @Override
-    public int freq()  {
-      if (positions != null) {
-        return positions.length;
-      } else {
-        Debug.Assert( startOffsets != null;
-        return startOffsets.length;
-      }
-    }
+            public override bool HasFreqs()
+            {
+                return true;
+            }
 
-    @Override
-    public int docID() {
-      return doc;
-    }
+            public override bool HasOffsets()
+            {
+                return _hasOffsetsRenamed;
+            }
 
-    @Override
-    public int nextDoc() {
-      if (!didNext && (liveDocs == null || liveDocs.get(0))) {
-        didNext = true;
-        return (doc = 0);
-      } else {
-        return (doc = NO_MORE_DOCS);
-      }
-    }
+            public override bool HasPositions()
+            {
+                return _hasPositionsRenamed;
+            }
 
-    @Override
-    public int advance(int target)  {
-      return slowAdvance(target);
-    }
+            public override bool HasPayloads()
+            {
+                return _hasPayloadsRenamed;
+            }
+        }
 
-    public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets, BytesRef payloads[]) {
-      this.liveDocs = liveDocs;
-      this.positions = positions;
-      this.startOffsets = startOffsets;
-      this.endOffsets = endOffsets;
-      this.payloads = payloads;
-      this.doc = -1;
-      didNext = false;
-      nextPos = 0;
-    }
+        private class SimpleTVPostings
+        {
+            internal int FREQ;
+            internal int[] POSITIONS;
+            internal int[] START_OFFSETS;
+            internal int[] END_OFFSETS;
+            internal BytesRef[] PAYLOADS;
+        }
 
-    @Override
-    public BytesRef getPayload() {
-      return payloads == null ? null : payloads[nextPos-1];
-    }
+        private class SimpleTVTermsEnum : TermsEnum
+        {
+            internal SortedMap<BytesRef, SimpleTVPostings> terms;
+            private IEnumerator<KeyValuePair<BytesRef, SimpleTVPostings>> _iterator;
+            private KeyValuePair<BytesRef, SimpleTVPostings> _current;
 
-    @Override
-    public int nextPosition() {
-      Debug.Assert( (positions != null && nextPos < positions.length) ||
-        startOffsets != null && nextPos < startOffsets.length;
-      if (positions != null) {
-        return positions[nextPos++];
-      } else {
-        nextPos++;
-        return -1;
-      }
-    }
+            internal SimpleTVTermsEnum(SortedMap<BytesRef, SimpleTVPostings> terms)
+            {
+                this.terms = terms;
+                this._iterator = terms.EntrySet().GetEnumerator();
+            }
 
-    @Override
-    public int startOffset() {
-      if (startOffsets == null) {
-        return -1;
-      } else {
-        return startOffsets[nextPos-1];
-      }
-    }
+            public override SeekStatus SeekCeil(BytesRef text)
+            {
+                _iterator = terms.TailMap(text).entrySet().GetEnumerator();
+                //JAVA TO C# CONVERTER TODO TASK: Java iterators are only converted within the context of 'while' and 'for' loops:
+                if (!_iterator.HasNext())
+                {
+                    return SeekStatus.END;
+                }
+                else
+                {
+                    return Next().Equals(text) ? SeekStatus.FOUND : SeekStatus.NOT_FOUND;
+                }
+            }
 
-    @Override
-    public int endOffset() {
-      if (endOffsets == null) {
-        return -1;
-      } else {
-        return endOffsets[nextPos-1];
-      }
-    }
-    
-    @Override
-    public long cost() {
-      return 1;
-    }
-  }
+            public override void SeekExact(long ord)
+            {
+                throw new NotSupportedException();
+            }
+
+            public override BytesRef Next()
+            {
+                //JAVA TO C# CONVERTER TODO TASK: Java iterators are only converted within the context of 'while' and 'for' loops:
+                if (!_iterator.HasNext())
+                {
+                    return null;
+                }
+                else
+                {
+                    //JAVA TO C# CONVERTER TODO TASK: Java iterators are only converted within the context of 'while' and 'for' loops:
+                    _current = _iterator.Next();
+                    return _current.Key;
+                }
+            }
+
+            public override BytesRef Term()
+            {
+                return _current.Key;
+            }
+
+            public override long Ord()
+            {
+                throw new NotSupportedException();
+            }
+
+            public override int DocFreq()
+            {
+                return 1;
+            }
+
+            public override long TotalTermFreq()
+            {
+                return _current.Value.FREQ;
+            }
+
+            public override DocsEnum Docs(Bits liveDocs, DocsEnum reuse, int flags)
+            {
+                // TODO: reuse
+                var e = new SimpleTVDocsEnum();
+                e.Reset(liveDocs, (flags & DocsEnum.FLAG_FREQS) == 0 ? 1 : _current.Value.FREQ);
+                return e;
+            }
+
+            public override DocsAndPositionsEnum DocsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags)
+            {
+                SimpleTVPostings postings = _current.Value;
+                if (postings.POSITIONS == null && postings.START_OFFSETS == null)
+                    return null;
+
+                // TODO: reuse
+                var e = new SimpleTVDocsAndPositionsEnum();
+                e.Reset(liveDocs, postings.POSITIONS, postings.START_OFFSETS, postings.END_OFFSETS, postings.PAYLOADS);
+                return e;
+            }
+
+            public override IComparer<BytesRef> Comparator
+            {
+                get { return BytesRef.UTF8SortedAsUnicodeComparer; }
+            }
+        }
 
-  @Override
-  public long ramBytesUsed() {
-    return 0;
-  }
+        // note: these two enum classes are exactly like the Default impl...
+        private sealed class SimpleTVDocsEnum : DocsEnum
+        {
+            private bool _didNext;
+            private int _doc = -1;
+            private int _freqRenamed;
+            private Bits _liveDocs;
+
+            public override int Freq()
+            {
+                Debug.Assert(_freqRenamed != -1);
+                return _freqRenamed;
+            }
+
+            public override int DocID()
+            {
+                return _doc;
+            }
+
+            public override int NextDoc()
+            {
+                if (_didNext || (_liveDocs != null && !_liveDocs.Get(0))) return (_doc = NO_MORE_DOCS);
+                _didNext = true;
+                return (_doc = 0);
+            }
+
+            public override int Advance(int target)
+            {
+                return SlowAdvance(target);
+            }
+
+            public void Reset(Bits liveDocs, int freq)
+            {
+                _liveDocs = liveDocs;
+                _freqRenamed = freq;
+                _doc = -1;
+                _didNext = false;
+            }
+
+            public override long Cost()
+            {
+                return 1;
+            }
+        }
+
+        private sealed class SimpleTVDocsAndPositionsEnum : DocsAndPositionsEnum
+        {
+            private bool _didNext;
+            private int _doc = -1;
+            private int _nextPos;
+            private Bits _liveDocs;
+            private int[] _positions;
+            private BytesRef[] _payloads;
+            private int[] _startOffsets;
+            private int[] _endOffsets;
+
+            public override int Freq()
+            {
+                if (_positions != null)
+                    return _positions.Length;
+
+                Debug.Assert(_startOffsets != null);
+                return _startOffsets.Length;
+            }
+
+            public override int DocID()
+            {
+                return _doc;
+            }
+
+            public override int NextDoc()
+            {
+                if (_didNext || (_liveDocs != null && !_liveDocs.Get(0))) return (_doc = NO_MORE_DOCS);
+                _didNext = true;
+                return (_doc = 0);
+            }
+
+            public override int Advance(int target)
+            {
+                return SlowAdvance(target);
+            }
+
+            public void Reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets,
+                BytesRef[] payloads)
+            {
+                _liveDocs = liveDocs;
+                _positions = positions;
+                _startOffsets = startOffsets;
+                _endOffsets = endOffsets;
+                _payloads = payloads;
+                _doc = -1;
+                _didNext = false;
+                _nextPos = 0;
+            }
+
+            public override BytesRef Payload
+            {
+                get { return _payloads == null ? null : _payloads[_nextPos - 1]; }
+            }
+
+            public override int NextPosition()
+            {
+                Debug.Assert((_positions != null && _nextPos < _positions.Length) ||
+                             _startOffsets != null && _nextPos < _startOffsets.Length);
+                if (_positions != null)
+                {
+                    return _positions[_nextPos++];
+                }
+
+                _nextPos++;
+                return -1;
+            }
+
+            public override int StartOffset()
+            {
+                if (_startOffsets == null)
+                    return -1;
+
+                return _startOffsets[_nextPos - 1];
+            }
+
+            public override int EndOffset()
+            {
+                if (_endOffsets == null)
+                {
+                    return -1;
+                }
+
+                return _endOffsets[_nextPos - 1];
+            }
+
+            public override long Cost()
+            {
+                return 1;
+            }
+        }
+
+    }
 
-  @Override
-  public void checkIntegrity()  {}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/cf1df6be/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsWriter.cs b/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsWriter.cs
index 5e497cc..67cc4e2 100644
--- a/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsWriter.cs
+++ b/src/Lucene.Net.Codecs/SimpleText/SimpleTextTermVectorsWriter.cs
@@ -1,6 +1,4 @@
-package org.apache.lucene.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.
@@ -17,193 +15,224 @@ package org.apache.lucene.codecs.simpletext;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Comparator;
-
-import org.apache.lucene.codecs.TermVectorsWriter;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
-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 FIELDPAYLOADS      = new BytesRef("    payloads  ");
-  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 PAYLOAD            = new BytesRef("        payload ");
-  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 bool offsets;
-  private bool positions;
-  private bool payloads;
-
-  public SimpleTextTermVectorsWriter(Directory directory, String segment, IOContext context)  {
-    this.directory = directory;
-    this.segment = segment;
-    bool success = false;
-    try {
-      out = directory.createOutput(IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION), context);
-      success = true;
-    } finally {
-      if (!success) {
-        abort();
-      }
-    }
-  }
-  
-  @Override
-  public void startDocument(int numVectorFields)  {
-    write(DOC);
-    write(Integer.toString(numDocsWritten));
-    newLine();
-    
-    write(NUMFIELDS);
-    write(Integer.toString(numVectorFields));
-    newLine();
-    numDocsWritten++;
-  }
-
-  @Override
-  public void startField(FieldInfo info, int numTerms, bool positions, bool offsets, bool payloads)  {  
-    write(FIELD);
-    write(Integer.toString(info.number));
-    newLine();
-    
-    write(FIELDNAME);
-    write(info.name);
-    newLine();
-    
-    write(FIELDPOSITIONS);
-    write(bool.toString(positions));
-    newLine();
-    
-    write(FIELDOFFSETS);
-    write(bool.toString(offsets));
-    newLine();
-    
-    write(FIELDPAYLOADS);
-    write(bool.toString(payloads));
-    newLine();
-    
-    write(FIELDTERMCOUNT);
-    write(Integer.toString(numTerms));
-    newLine();
-    
-    this.positions = positions;
-    this.offsets = offsets;
-    this.payloads = payloads;
-  }
-
-  @Override
-  public void startTerm(BytesRef term, int freq)  {
-    write(TERMTEXT);
-    write(term);
-    newLine();
-    
-    write(TERMFREQ);
-    write(Integer.toString(freq));
-    newLine();
-  }
-
-  @Override
-  public void addPosition(int position, int startOffset, int endOffset, BytesRef payload)  {
-    Debug.Assert( positions || offsets;
-    
-    if (positions) {
-      write(POSITION);
-      write(Integer.toString(position));
-      newLine();
-      
-      if (payloads) {
-        write(PAYLOAD);
-        if (payload != null) {
-          Debug.Assert( payload.length > 0;
-          write(payload);
+namespace Lucene.Net.Codecs.SimpleText
+{
+
+    using System;
+    using System.Diagnostics;
+    using System.Collections.Generic;
+    using FieldInfo = Index.FieldInfo;
+    using FieldInfos = Index.FieldInfos;
+    using IndexFileNames = Index.IndexFileNames;
+    using Directory = Store.Directory;
+    using IOContext = Store.IOContext;
+    using IndexOutput = Store.IndexOutput;
+    using BytesRef = Util.BytesRef;
+    using IOUtils = Util.IOUtils;
+
+    /// <summary>
+    /// Writes plain-text term vectors.
+    /// <para>
+    /// <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+    /// @lucene.experimental
+    /// </para>
+    /// </summary>
+    public class SimpleTextTermVectorsWriter : TermVectorsWriter
+    {
+
+        internal static readonly BytesRef END = new BytesRef("END");
+        internal static readonly BytesRef DOC = new BytesRef("doc ");
+        internal static readonly BytesRef NUMFIELDS = new BytesRef("  numfields ");
+        internal static readonly BytesRef FIELD = new BytesRef("  field ");
+        internal static readonly BytesRef FIELDNAME = new BytesRef("    name ");
+        internal static readonly BytesRef FIELDPOSITIONS = new BytesRef("    positions ");
+        internal static readonly BytesRef FIELDOFFSETS = new BytesRef("    offsets   ");
+        internal static readonly BytesRef FIELDPAYLOADS = new BytesRef("    payloads  ");
+        internal static readonly BytesRef FIELDTERMCOUNT = new BytesRef("    numterms ");
+        internal static readonly BytesRef TERMTEXT = new BytesRef("    term ");
+        internal static readonly BytesRef TERMFREQ = new BytesRef("      freq ");
+        internal static readonly BytesRef POSITION = new BytesRef("      position ");
+        internal static readonly BytesRef PAYLOAD = new BytesRef("        payload ");
+        internal static readonly BytesRef STARTOFFSET = new BytesRef("        startoffset ");
+        internal static readonly BytesRef ENDOFFSET = new BytesRef("        endoffset ");
+
+        internal const string VECTORS_EXTENSION = "vec";
+
+        private readonly Directory directory;
+        private readonly string segment;
+        private IndexOutput _output;
+        private int numDocsWritten = 0;
+        private readonly BytesRef scratch = new BytesRef();
+        private bool offsets;
+        private bool positions;
+        private bool payloads;
+
+        public SimpleTextTermVectorsWriter(Directory directory, string segment, IOContext context)
+        {
+            this.directory = directory;
+            this.segment = segment;
+            bool success = false;
+            try
+            {
+                _output = directory.CreateOutput(IndexFileNames.SegmentFileName(segment, "", VECTORS_EXTENSION), context);
+                success = true;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    Abort();
+                }
+            }
+        }
+
+        public override void StartDocument(int numVectorFields)
+        {
+            Write(DOC);
+            Write(Convert.ToString(numDocsWritten));
+            NewLine();
+
+            Write(NUMFIELDS);
+            Write(Convert.ToString(numVectorFields));
+            NewLine();
+            numDocsWritten++;
+        }
+
+        public override void StartField(FieldInfo info, int numTerms, bool positions, bool offsets, bool payloads)
+        {
+            Write(FIELD);
+            Write(Convert.ToString(info.Number));
+            NewLine();
+
+            Write(FIELDNAME);
+            Write(info.Name);
+            NewLine();
+
+            Write(FIELDPOSITIONS);
+            Write(Convert.ToString(positions));
+            NewLine();
+
+            Write(FIELDOFFSETS);
+            Write(Convert.ToString(offsets));
+            NewLine();
+
+            Write(FIELDPAYLOADS);
+            Write(Convert.ToString(payloads));
+            NewLine();
+
+            Write(FIELDTERMCOUNT);
+            Write(Convert.ToString(numTerms));
+            NewLine();
+
+            this.positions = positions;
+            this.offsets = offsets;
+            this.payloads = payloads;
+        }
+
+        public override void StartTerm(BytesRef term, int freq)
+        {
+            Write(TERMTEXT);
+            Write(term);
+            NewLine();
+
+            Write(TERMFREQ);
+            Write(Convert.ToString(freq));
+            NewLine();
+        }
+
+        public override void AddPosition(int position, int startOffset, int endOffset, BytesRef payload)
+        {
+            Debug.Assert(positions || offsets);
+
+            if (positions)
+            {
+                Write(POSITION);
+                Write(Convert.ToString(position));
+                NewLine();
+
+                if (payloads)
+                {
+                    Write(PAYLOAD);
+                    if (payload != null)
+                    {
+                        Debug.Assert(payload.Length > 0);
+                        Write(payload);
+                    }
+                    NewLine();
+                }
+            }
+
+            if (offsets)
+            {
+                Write(STARTOFFSET);
+                Write(Convert.ToString(startOffset));
+                NewLine();
+
+                Write(ENDOFFSET);
+                Write(Convert.ToString(endOffset));
+                NewLine();
+            }
+        }
+
+        public override void Abort()
+        {
+            try
+            {
+                Dispose();
+            }
+            finally
+            {
+
+                IOUtils.DeleteFilesIgnoringExceptions(directory,
+                    IndexFileNames.SegmentFileName(segment, "", VECTORS_EXTENSION));
+            }
+        }
+
+        public override void Finish(FieldInfos fis, int numDocs)
+        {
+            if (numDocsWritten != numDocs)
+            {
+                throw new Exception("mergeVectors produced an invalid result: mergedDocs is " + numDocs +
+                                    " but vec numDocs is " + numDocsWritten + " file=" + _output.ToString() +
+                                    "; now aborting this merge to prevent index corruption");
+            }
+            Write(END);
+            NewLine();
+            SimpleTextUtil.WriteChecksum(_output, scratch);
+        }
+
+        protected override void Dispose(bool disposing)
+        {
+            if (disposing) return;
+
+            try
+            {
+                IOUtils.Close(_output);
+            }
+            finally
+            {
+                _output = null;
+            }
+        }
+
+        public override IComparer<BytesRef> Comparator
+        {
+            get { return BytesRef.UTF8SortedAsUnicodeComparer; }
+        }
+
+        private void Write(string s)
+        {
+            SimpleTextUtil.Write(_output, s, scratch);
+        }
+
+        private void Write(BytesRef bytes)
+        {
+            SimpleTextUtil.Write(_output, bytes);
+        }
+
+        private void NewLine()
+        {
+            SimpleTextUtil.WriteNewline(_output);
         }
-        newLine();
-      }
-    }
-    
-    if (offsets) {
-      write(STARTOFFSET);
-      write(Integer.toString(startOffset));
-      newLine();
-      
-      write(ENDOFFSET);
-      write(Integer.toString(endOffset));
-      newLine();
-    }
-  }
-
-  @Override
-  public void abort() {
-    try {
-      close();
-    } catch (Throwable ignored) {}
-    IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION));
-  }
-
-  @Override
-  public void finish(FieldInfos fis, int numDocs)  {
-    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();
-    SimpleTextUtil.writeChecksum(out, scratch);
-  }
-  
-  @Override
-  public void close()  {
-    try {
-      IOUtils.close(out);
-    } finally {
-      out = null;
     }
-  }
-  
-  @Override
-  public Comparator<BytesRef> getComparator()  {
-    return BytesRef.getUTF8SortedAsUnicodeComparator();
-  }
-  
-  private void write(String s)  {
-    SimpleTextUtil.write(out, s, scratch);
-  }
-  
-  private void write(BytesRef bytes)  {
-    SimpleTextUtil.write(out, bytes);
-  }
-  
-  private void newLine()  {
-    SimpleTextUtil.writeNewline(out);
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/cf1df6be/src/Lucene.Net.Codecs/SimpleText/SimpleTextUtil.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/SimpleText/SimpleTextUtil.cs b/src/Lucene.Net.Codecs/SimpleText/SimpleTextUtil.cs
index edb4da7..447b1bb 100644
--- a/src/Lucene.Net.Codecs/SimpleText/SimpleTextUtil.cs
+++ b/src/Lucene.Net.Codecs/SimpleText/SimpleTextUtil.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -18,28 +18,32 @@
 namespace Lucene.Net.Codecs.SimpleText
 {
 
-    using System;
-    using Lucene.Net.Index;
-    using Lucene.Net.Store;
-    using Lucene.Net.Util;
+    using CorruptIndexException = Lucene.Net.Index.CorruptIndexException;
+    using ChecksumIndexInput = Store.ChecksumIndexInput;
+    using DataInput = Store.DataInput;
+    using DataOutput = Store.DataOutput;
+    using IndexOutput = Store.IndexOutput;
+    using BytesRef = Util.BytesRef;
+    using StringHelper = Util.StringHelper;
+    using UnicodeUtil = Util.UnicodeUtil;
 
-    public static class SimpleTextUtil
+    internal class SimpleTextUtil
     {
-        public const byte NEWLINE = 10;
-        public const byte ESCAPE = 92;
-        public static BytesRef CHECKSUM = new BytesRef("checksum ");
+        public const sbyte NEWLINE = 10;
+        public const sbyte ESCAPE = 92;
+        internal static readonly BytesRef CHECKSUM = new BytesRef("checksum ");
 
-        public static void Write(DataOutput output, String s, BytesRef scratch)
+        public static void Write(DataOutput output, string s, BytesRef scratch)
         {
-            UnicodeUtil.UTF16toUTF8(s, 0, s.Length, scratch);
+            UnicodeUtil.UTF16toUTF8(s.ToCharArray(), 0, s.Length, scratch);
             Write(output, scratch);
         }
 
         public static void Write(DataOutput output, BytesRef b)
         {
-            for (int i = 0; i < b.Length; i++)
+            for (var i = 0; i < b.Length; i++)
             {
-                sbyte bx = b.Bytes[b.Offset + i];
+                var bx = b.Bytes[b.Offset + i];
                 if (bx == NEWLINE || bx == ESCAPE)
                 {
                     output.WriteByte(ESCAPE);
@@ -55,17 +59,17 @@ namespace Lucene.Net.Codecs.SimpleText
 
         public static void ReadLine(DataInput input, BytesRef scratch)
         {
-            int upto = 0;
+            var upto = 0;
             while (true)
             {
-                byte b = input.ReadByte();
+                var b = input.ReadSByte();
                 if (scratch.Bytes.Length == upto)
                 {
                     scratch.Grow(1 + upto);
                 }
                 if (b == ESCAPE)
                 {
-                    scratch.Bytes[upto++] = input.ReadByte();
+                    scratch.Bytes[upto++] = input.ReadSByte();
                 }
                 else
                 {
@@ -88,29 +92,29 @@ namespace Lucene.Net.Codecs.SimpleText
             // Pad with zeros so different checksum values use the
             // same number of bytes
             // (BaseIndexFileFormatTestCase.testMergeStability cares):
-            String checksum = String.Format(Locale.ROOT, "%020d", output.Checksum);
-            SimpleTextUtil.Write(output, CHECKSUM);
-            SimpleTextUtil.Write(output, checksum, scratch);
-            SimpleTextUtil.WriteNewline(output);
+            var checksum = string.Format("{0:D}", output.Checksum);
+            Write(output, CHECKSUM);
+            Write(output, checksum, scratch);
+            WriteNewline(output);
         }
 
         public static void CheckFooter(ChecksumIndexInput input)
         {
-            BytesRef scratch = new BytesRef();
-            String expectedChecksum = String.Format(Locale.ROOT, "%020d", input.Checksum);
-            SimpleTextUtil.ReadLine(input, scratch);
+            var scratch = new BytesRef();
+            var expectedChecksum = string.Format("{0:D}", input.Checksum);
+            ReadLine(input, scratch);
+
             if (StringHelper.StartsWith(scratch, CHECKSUM) == false)
             {
                 throw new CorruptIndexException("SimpleText failure: expected checksum line but got " +
                                                 scratch.Utf8ToString() + " (resource=" + input + ")");
             }
-            String actualChecksum =
-                new BytesRef(scratch.Bytes, CHECKSUM.Length, scratch.Length - CHECKSUM.Length).Utf8ToString();
+            var actualChecksum =
+                (new BytesRef(scratch.Bytes, CHECKSUM.Length, scratch.Length - CHECKSUM.Length)).Utf8ToString();
             if (!expectedChecksum.Equals(actualChecksum))
             {
                 throw new CorruptIndexException("SimpleText checksum failure: " + actualChecksum + " != " +
-                                                expectedChecksum +
-                                                " (resource=" + input + ")");
+                                                expectedChecksum + " (resource=" + input + ")");
             }
             if (input.Length() != input.FilePointer)
             {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/cf1df6be/src/Lucene.Net.Codecs/StringHelperClass.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/StringHelperClass.cs b/src/Lucene.Net.Codecs/StringHelperClass.cs
new file mode 100644
index 0000000..a9ba97a
--- /dev/null
+++ b/src/Lucene.Net.Codecs/StringHelperClass.cs
@@ -0,0 +1,119 @@
+/*
+* 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.
+*/
+
+//-------------------------------------------------------------------------------------------
+//	Copyright © 2007 - 2014 Tangible Software Solutions Inc.
+//	This class can be used by anyone provided that the copyright notice remains intact.
+//
+//	This class is used to convert some aspects of the Java String class.
+//-------------------------------------------------------------------------------------------
+
+namespace Lucene.Net.Codes
+{
+
+    internal static class StringHelperClass
+    {
+        //----------------------------------------------------------------------------------
+        //	This method replaces the Java String.substring method when 'start' is a
+        //	method call or calculated value to ensure that 'start' is obtained just once.
+        //----------------------------------------------------------------------------------
+        internal static string SubstringSpecial(this string self, int start, int end)
+        {
+            return self.Substring(start, end - start);
+        }
+
+        //------------------------------------------------------------------------------------
+        //	This method is used to replace calls to the 2-arg Java String.startsWith method.
+        //------------------------------------------------------------------------------------
+        internal static bool StartsWith(this string self, string prefix, int toffset)
+        {
+            return self.IndexOf(prefix, toffset, System.StringComparison.Ordinal) == toffset;
+        }
+
+        //------------------------------------------------------------------------------
+        //	This method is used to replace most calls to the Java String.split method.
+        //------------------------------------------------------------------------------
+        internal static string[] Split(this string self, string regexDelimiter, bool trimTrailingEmptyStrings)
+        {
+            string[] splitArray = System.Text.RegularExpressions.Regex.Split(self, regexDelimiter);
+
+            if (trimTrailingEmptyStrings)
+            {
+                if (splitArray.Length > 1)
+                {
+                    for (int i = splitArray.Length; i > 0; i--)
+                    {
+                        if (splitArray[i - 1].Length > 0)
+                        {
+                            if (i < splitArray.Length)
+                                System.Array.Resize(ref splitArray, i);
+
+                            break;
+                        }
+                    }
+                }
+            }
+
+            return splitArray;
+        }
+
+        #region These methods are used to replace calls to some Java String constructors.
+
+        internal static string NewString(sbyte[] bytes)
+        {
+            return NewString(bytes, 0, bytes.Length);
+        }
+
+        internal static string NewString(sbyte[] bytes, int index, int count)
+        {
+            return System.Text.Encoding.UTF8.GetString((byte[]) (object) bytes, index, count);
+        }
+
+        internal static string NewString(sbyte[] bytes, string encoding)
+        {
+            return NewString(bytes, 0, bytes.Length, encoding);
+        }
+
+        internal static string NewString(sbyte[] bytes, int index, int count, string encoding)
+        {
+            return System.Text.Encoding.GetEncoding(encoding).GetString((byte[]) (object) bytes, index, count);
+        }
+
+        #endregion
+
+        #region	These methods are used to replace calls to the Java String.getBytes methods.
+
+        internal static sbyte[] GetBytes(this string self)
+        {
+            return GetSBytesForEncoding(System.Text.Encoding.UTF8, self);
+        }
+
+        internal static sbyte[] GetBytes(this string self, string encoding)
+        {
+            return GetSBytesForEncoding(System.Text.Encoding.GetEncoding(encoding), self);
+        }
+
+        private static sbyte[] GetSBytesForEncoding(System.Text.Encoding encoding, string s)
+        {
+            sbyte[] sbytes = new sbyte[encoding.GetByteCount(s)];
+            encoding.GetBytes(s, 0, s.Length, (byte[]) (object) sbytes, 0);
+            return sbytes;
+        }
+
+        #endregion
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/cf1df6be/src/Lucene.Net.Core/Search/MultiTermQuery.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Search/MultiTermQuery.cs b/src/Lucene.Net.Core/Search/MultiTermQuery.cs
index c2bd033..8952e69 100644
--- a/src/Lucene.Net.Core/Search/MultiTermQuery.cs
+++ b/src/Lucene.Net.Core/Search/MultiTermQuery.cs
@@ -57,7 +57,7 @@ namespace Lucene.Net.Search
     /// a priority queue to only collect competitive terms
     /// and not hit this limitation.
     ///
-    /// Note that org.apache.lucene.queryparser.classic.QueryParser produces
+    /// Note that queryparser.classic.QueryParser produces
     /// MultiTermQueries using {@link
     /// #CONSTANT_SCORE_AUTO_REWRITE_DEFAULT} by default.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/cf1df6be/src/Lucene.Net.Core/Store/Directory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Store/Directory.cs b/src/Lucene.Net.Core/Store/Directory.cs
index a485343..8286867 100644
--- a/src/Lucene.Net.Core/Store/Directory.cs
+++ b/src/Lucene.Net.Core/Store/Directory.cs
@@ -27,16 +27,16 @@ namespace Lucene.Net.Store
     /// are created.  Once a file is created it may only be opened for read, or
     /// deleted.  Random access is permitted both when reading and writing.
     ///
-    /// <p> Java's i/o APIs not used directly, but rather all i/o is
+    /// Java's i/o APIs not used directly, but rather all i/o is
     /// through this API.  this permits things such as: <ul>
-    /// <li> implementation of RAM-based indices;
-    /// <li> implementation indices stored in a database, via JDBC;
-    /// <li> implementation of an index as a single file;
+    /// <li> implementation of RAM-based indices;</li>
+    /// <li> implementation indices stored in a database, via JDBC;</li>
+    /// <li> implementation of an index as a single file;</li>
     /// </ul>
     ///
     /// Directory locking is implemented by an instance of {@link
     /// LockFactory}, and can be changed for each Directory
-    /// instance using <seealso cref="#setLockFactory"/>.
+    /// instance using <seealso cref="setLockFactory"/>.
     ///
     /// </summary>
     public abstract class Directory : IDisposable