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/28 04:32:09 UTC

[3/5] Most of Lucene.Net.Codecs/SimpleText ported - still a little bit of work around DecimalFormat conversion

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/07150be0/src/Lucene.Net.Codecs/SimpleText/SimpleTextDocValuesWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/SimpleText/SimpleTextDocValuesWriter.cs b/src/Lucene.Net.Codecs/SimpleText/SimpleTextDocValuesWriter.cs
index 1afcd00..81abb86 100644
--- a/src/Lucene.Net.Codecs/SimpleText/SimpleTextDocValuesWriter.cs
+++ b/src/Lucene.Net.Codecs/SimpleText/SimpleTextDocValuesWriter.cs
@@ -1,485 +1,466 @@
-using System;
-using System.Diagnostics;
-using System.Collections.Generic;
-using System.Text;
+/*
+ * 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
 {
 
-	/*
-	 * 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.
-	 */
-
-
-	using FieldInfo = Index.FieldInfo;
-	using IndexFileNames = Index.IndexFileNames;
-	using SegmentWriteState = Index.SegmentWriteState;
-	using DocValuesType = Index.FieldInfo.DocValuesType;
-	using IndexOutput = Store.IndexOutput;
-	using BytesRef = Util.BytesRef;
-	using IOUtils = Util.IOUtils;
-
-	public class SimpleTextDocValuesWriter : DocValuesConsumer
-	{
-	  internal static readonly BytesRef END = new BytesRef("END");
-	  internal static readonly BytesRef FIELD = new BytesRef("field ");
-	  internal static readonly BytesRef TYPE = new BytesRef("  type ");
-	  // used for numerics
-	  internal static readonly BytesRef MINVALUE = new BytesRef("  minvalue ");
-	  internal static readonly BytesRef PATTERN = new BytesRef("  pattern ");
-	  // used for bytes
-	  internal static readonly BytesRef LENGTH = new BytesRef("length ");
-	  internal static readonly BytesRef MAXLENGTH = new BytesRef("  maxlength ");
-	  // used for sorted bytes
-	  internal static readonly BytesRef NUMVALUES = new BytesRef("  numvalues ");
-	  internal static readonly BytesRef ORDPATTERN = new BytesRef("  ordpattern ");
-
-	  internal IndexOutput data;
-	  internal readonly BytesRef scratch = new BytesRef();
-	  internal readonly int numDocs;
-	  private readonly HashSet<string> fieldsSeen = new HashSet<string>(); // for asserting
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: public SimpleTextDocValuesWriter(index.SegmentWriteState state, String ext) throws java.io.IOException
-	  public SimpleTextDocValuesWriter(SegmentWriteState state, string ext)
-	  {
-		// System.out.println("WRITE: " + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext) + " " + state.segmentInfo.getDocCount() + " docs");
-		data = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
-		numDocs = state.segmentInfo.DocCount;
-	  }
-
-	  // for asserting
-	  private bool fieldSeen(string field)
-	  {
-		Debug.Assert(!fieldsSeen.Contains(field), "field \"" + field + "\" was added more than once during flush");
-		fieldsSeen.Add(field);
-		return true;
-	  }
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void addNumericField(index.FieldInfo field, Iterable<Number> values) throws java.io.IOException
-	  public override void addNumericField(FieldInfo field, IEnumerable<Number> values)
-	  {
-		Debug.Assert(fieldSeen(field.name));
-		assert(field.DocValuesType == FieldInfo.DocValuesType.NUMERIC || field.NormType == FieldInfo.DocValuesType.NUMERIC);
-		writeFieldEntry(field, FieldInfo.DocValuesType.NUMERIC);
-
-		// first pass to find min/max
-		long minValue = long.MaxValue;
-		long maxValue = long.MinValue;
-		foreach (Number n in values)
-		{
-		  long v = n == null ? 0 : (long)n;
-		  minValue = Math.Min(minValue, v);
-		  maxValue = Math.Max(maxValue, v);
-		}
-
-		// write our minimum value to the .dat, all entries are deltas from that
-		SimpleTextUtil.write(data, MINVALUE);
-		SimpleTextUtil.write(data, Convert.ToString(minValue), scratch);
-		SimpleTextUtil.WriteNewline(data);
-
-		// build up our fixed-width "simple text packed ints"
-		// format
-		System.Numerics.BigInteger maxBig = System.Numerics.BigInteger.valueOf(maxValue);
-		System.Numerics.BigInteger minBig = System.Numerics.BigInteger.valueOf(minValue);
-		System.Numerics.BigInteger diffBig = maxBig - minBig;
-		int maxBytesPerValue = diffBig.ToString().Length;
-		StringBuilder sb = new StringBuilder();
-		for (int i = 0; i < maxBytesPerValue; i++)
-		{
-		  sb.Append('0');
-		}
-
-		// write our pattern to the .dat
-		SimpleTextUtil.write(data, PATTERN);
-		SimpleTextUtil.write(data, sb.ToString(), scratch);
-		SimpleTextUtil.WriteNewline(data);
+    using System;
+    using System.Diagnostics;
+    using System.Collections.Generic;
+    using System.Text;
+
+    using FieldInfo = Index.FieldInfo;
+    using IndexFileNames = Index.IndexFileNames;
+    using SegmentWriteState = Index.SegmentWriteState;
+    using DocValuesType = Index.FieldInfo.DocValuesType_e;
+    using IndexOutput = Store.IndexOutput;
+    using BytesRef = Util.BytesRef;
+    using IOUtils = Util.IOUtils;
+
+    public class SimpleTextDocValuesWriter : DocValuesConsumer
+    {
+        internal static readonly BytesRef END = new BytesRef("END");
+        internal static readonly BytesRef FIELD = new BytesRef("field ");
+        internal static readonly BytesRef TYPE = new BytesRef("  type ");
+        
+        // used for numerics
+        internal static readonly BytesRef MINVALUE = new BytesRef("  minvalue ");
+        internal static readonly BytesRef PATTERN = new BytesRef("  pattern ");
+        
+        // used for bytes
+        internal static readonly BytesRef LENGTH = new BytesRef("length ");
+        internal static readonly BytesRef MAXLENGTH = new BytesRef("  maxlength ");
+        
+        // used for sorted bytes
+        internal static readonly BytesRef NUMVALUES = new BytesRef("  numvalues ");
+        internal static readonly BytesRef ORDPATTERN = new BytesRef("  ordpattern ");
+
+        internal IndexOutput data;
+        internal readonly BytesRef scratch = new BytesRef();
+        internal readonly int numDocs;
+        private readonly HashSet<string> _fieldsSeen = new HashSet<string>(); // for asserting
+
+        public SimpleTextDocValuesWriter(SegmentWriteState state, string ext)
+        {
+            data = state.Directory.CreateOutput(
+                    IndexFileNames.SegmentFileName(state.SegmentInfo.Name, state.SegmentSuffix, ext), state.Context);
+            numDocs = state.SegmentInfo.DocCount;
+        }
+
+        public override void AddNumericField(FieldInfo field, IEnumerable<long> values)
+        {
+            Debug.Assert(FieldSeen(field.Name));
+            Debug.Assert(field.DocValuesType == FieldInfo.DocValuesType_e.NUMERIC ||
+                         field.NormType == FieldInfo.DocValuesType_e.NUMERIC);
+            WriteFieldEntry(field, FieldInfo.DocValuesType_e.NUMERIC);
+
+            // first pass to find min/max
+            var minValue = long.MaxValue;
+            var maxValue = long.MinValue;
+            foreach (var n in values)
+            {
+                var v = n;
+                minValue = Math.Min(minValue, v);
+                maxValue = Math.Max(maxValue, v);
+            }
+
+            // write our minimum value to the .dat, all entries are deltas from that
+            SimpleTextUtil.Write(data, MINVALUE);
+            SimpleTextUtil.Write(data, Convert.ToString(minValue), scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            // build up our fixed-width "simple text packed ints"
+            // format
+            System.Numerics.BigInteger maxBig = System.Numerics.BigInteger.valueOf(maxValue);
+            System.Numerics.BigInteger minBig = System.Numerics.BigInteger.valueOf(minValue);
+            System.Numerics.BigInteger diffBig = maxBig - minBig;
+            int maxBytesPerValue = diffBig.ToString().Length;
+            StringBuilder sb = new StringBuilder();
+            for (int i = 0; i < maxBytesPerValue; i++)
+            {
+                sb.Append('0');
+            }
+
+            // write our pattern to the .dat
+            SimpleTextUtil.Write(data, PATTERN);
+            SimpleTextUtil.Write(data, sb.ToString(), scratch);
+            SimpleTextUtil.WriteNewline(data);
 
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
 //ORIGINAL LINE: final String patternString = sb.toString();
-		string patternString = sb.ToString();
+            string patternString = sb.ToString();
 
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
 //ORIGINAL LINE: final java.text.DecimalFormat encoder = new java.text.DecimalFormat(patternString, new java.text.DecimalFormatSymbols(java.util.Locale.ROOT));
-		DecimalFormat encoder = new DecimalFormat(patternString, new DecimalFormatSymbols(Locale.ROOT));
-
-		int numDocsWritten = 0;
-
-		// second pass to write the values
-		foreach (Number n in values)
-		{
-		  long value = n == null ? 0 : (long)n;
-		  Debug.Assert(value >= minValue);
-		  Number delta = System.Numerics.BigInteger.valueOf(value) - System.Numerics.BigInteger.valueOf(minValue);
-		  string s = encoder.format(delta);
-		  Debug.Assert(s.Length == patternString.Length);
-		  SimpleTextUtil.write(data, s, scratch);
-		  SimpleTextUtil.WriteNewline(data);
-		  if (n == null)
-		  {
-			SimpleTextUtil.write(data, "F", scratch);
-		  }
-		  else
-		  {
-			SimpleTextUtil.write(data, "T", scratch);
-		  }
-		  SimpleTextUtil.WriteNewline(data);
-		  numDocsWritten++;
-		  Debug.Assert(numDocsWritten <= numDocs);
-		}
-
-		Debug.Assert(numDocs == numDocsWritten, "numDocs=" + numDocs + " numDocsWritten=" + numDocsWritten);
-	  }
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void addBinaryField(index.FieldInfo field, Iterable<util.BytesRef> values) throws java.io.IOException
-	  public override void addBinaryField(FieldInfo field, IEnumerable<BytesRef> values)
-	  {
-		Debug.Assert(fieldSeen(field.name));
-		Debug.Assert(field.DocValuesType == FieldInfo.DocValuesType.BINARY);
-		int maxLength = 0;
-		foreach (BytesRef value in values)
-		{
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final int length = value == null ? 0 : value.length;
-		  int length = value == null ? 0 : value.length;
-		  maxLength = Math.Max(maxLength, length);
-		}
-		writeFieldEntry(field, FieldInfo.DocValuesType.BINARY);
-
-		// write maxLength
-		SimpleTextUtil.write(data, MAXLENGTH);
-		SimpleTextUtil.write(data, Convert.ToString(maxLength), scratch);
-		SimpleTextUtil.WriteNewline(data);
-
-		int maxBytesLength = Convert.ToString(maxLength).Length;
-		StringBuilder sb = new StringBuilder();
-		for (int i = 0; i < maxBytesLength; i++)
-		{
-		  sb.Append('0');
-		}
-		// write our pattern for encoding lengths
-		SimpleTextUtil.write(data, PATTERN);
-		SimpleTextUtil.write(data, sb.ToString(), scratch);
-		SimpleTextUtil.WriteNewline(data);
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final java.text.DecimalFormat encoder = new java.text.DecimalFormat(sb.toString(), new java.text.DecimalFormatSymbols(java.util.Locale.ROOT));
-		DecimalFormat encoder = new DecimalFormat(sb.ToString(), new DecimalFormatSymbols(Locale.ROOT));
-
-		int numDocsWritten = 0;
-		foreach (BytesRef value in values)
-		{
-		  // write length
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final int length = value == null ? 0 : value.length;
-		  int length = value == null ? 0 : value.length;
-		  SimpleTextUtil.write(data, LENGTH);
-		  SimpleTextUtil.write(data, encoder.format(length), scratch);
-		  SimpleTextUtil.WriteNewline(data);
-
-		  // write bytes -- don't use SimpleText.write
-		  // because it escapes:
-		  if (value != null)
-		  {
-			data.writeBytes(value.bytes, value.offset, value.length);
-		  }
-
-		  // pad to fit
-		  for (int i = length; i < maxLength; i++)
-		  {
-			data.writeByte((sbyte)' ');
-		  }
-		  SimpleTextUtil.WriteNewline(data);
-		  if (value == null)
-		  {
-			SimpleTextUtil.write(data, "F", scratch);
-		  }
-		  else
-		  {
-			SimpleTextUtil.write(data, "T", scratch);
-		  }
-		  SimpleTextUtil.WriteNewline(data);
-		  numDocsWritten++;
-		}
-
-		Debug.Assert(numDocs == numDocsWritten);
-	  }
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void addSortedField(index.FieldInfo field, Iterable<util.BytesRef> values, Iterable<Number> docToOrd) throws java.io.IOException
-	  public override void addSortedField(FieldInfo field, IEnumerable<BytesRef> values, IEnumerable<Number> docToOrd)
-	  {
-		Debug.Assert(fieldSeen(field.name));
-		Debug.Assert(field.DocValuesType == FieldInfo.DocValuesType.SORTED);
-		writeFieldEntry(field, FieldInfo.DocValuesType.SORTED);
-
-		int valueCount = 0;
-		int maxLength = -1;
-		foreach (BytesRef value in values)
-		{
-		  maxLength = Math.Max(maxLength, value.length);
-		  valueCount++;
-		}
-
-		// write numValues
-		SimpleTextUtil.write(data, NUMVALUES);
-		SimpleTextUtil.write(data, Convert.ToString(valueCount), scratch);
-		SimpleTextUtil.WriteNewline(data);
-
-		// write maxLength
-		SimpleTextUtil.write(data, MAXLENGTH);
-		SimpleTextUtil.write(data, Convert.ToString(maxLength), scratch);
-		SimpleTextUtil.WriteNewline(data);
-
-		int maxBytesLength = Convert.ToString(maxLength).Length;
-		StringBuilder sb = new StringBuilder();
-		for (int i = 0; i < maxBytesLength; i++)
-		{
-		  sb.Append('0');
-		}
-
-		// write our pattern for encoding lengths
-		SimpleTextUtil.write(data, PATTERN);
-		SimpleTextUtil.write(data, sb.ToString(), scratch);
-		SimpleTextUtil.WriteNewline(data);
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final java.text.DecimalFormat encoder = new java.text.DecimalFormat(sb.toString(), new java.text.DecimalFormatSymbols(java.util.Locale.ROOT));
-		DecimalFormat encoder = new DecimalFormat(sb.ToString(), new DecimalFormatSymbols(Locale.ROOT));
-
-		int maxOrdBytes = Convert.ToString(valueCount + 1L).Length;
-		sb.Length = 0;
-		for (int i = 0; i < maxOrdBytes; i++)
-		{
-		  sb.Append('0');
-		}
-
-		// write our pattern for ords
-		SimpleTextUtil.write(data, ORDPATTERN);
-		SimpleTextUtil.write(data, sb.ToString(), scratch);
-		SimpleTextUtil.WriteNewline(data);
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final java.text.DecimalFormat ordEncoder = new java.text.DecimalFormat(sb.toString(), new java.text.DecimalFormatSymbols(java.util.Locale.ROOT));
-		DecimalFormat ordEncoder = new DecimalFormat(sb.ToString(), new DecimalFormatSymbols(Locale.ROOT));
-
-		// for asserts:
-		int valuesSeen = 0;
-
-		foreach (BytesRef value in values)
-		{
-		  // write length
-		  SimpleTextUtil.write(data, LENGTH);
-		  SimpleTextUtil.write(data, encoder.format(value.length), scratch);
-		  SimpleTextUtil.WriteNewline(data);
-
-		  // write bytes -- don't use SimpleText.write
-		  // because it escapes:
-		  data.writeBytes(value.bytes, value.offset, value.length);
-
-		  // pad to fit
-		  for (int i = value.length; i < maxLength; i++)
-		  {
-			data.writeByte((sbyte)' ');
-		  }
-		  SimpleTextUtil.WriteNewline(data);
-		  valuesSeen++;
-		  Debug.Assert(valuesSeen <= valueCount);
-		}
-
-		Debug.Assert(valuesSeen == valueCount);
-
-		foreach (Number ord in docToOrd)
-		{
-		  SimpleTextUtil.write(data, ordEncoder.format((long)ord + 1), scratch);
-		  SimpleTextUtil.WriteNewline(data);
-		}
-	  }
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void addSortedSetField(index.FieldInfo field, Iterable<util.BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws java.io.IOException
-	  public override void addSortedSetField(FieldInfo field, IEnumerable<BytesRef> values, IEnumerable<Number> docToOrdCount, IEnumerable<Number> ords)
-	  {
-		Debug.Assert(fieldSeen(field.name));
-		Debug.Assert(field.DocValuesType == FieldInfo.DocValuesType.SORTED_SET);
-		writeFieldEntry(field, FieldInfo.DocValuesType.SORTED_SET);
-
-		long valueCount = 0;
-		int maxLength = 0;
-		foreach (BytesRef value in values)
-		{
-		  maxLength = Math.Max(maxLength, value.length);
-		  valueCount++;
-		}
-
-		// write numValues
-		SimpleTextUtil.write(data, NUMVALUES);
-		SimpleTextUtil.write(data, Convert.ToString(valueCount), scratch);
-		SimpleTextUtil.WriteNewline(data);
-
-		// write maxLength
-		SimpleTextUtil.write(data, MAXLENGTH);
-		SimpleTextUtil.write(data, Convert.ToString(maxLength), scratch);
-		SimpleTextUtil.WriteNewline(data);
-
-		int maxBytesLength = Convert.ToString(maxLength).Length;
-		StringBuilder sb = new StringBuilder();
-		for (int i = 0; i < maxBytesLength; i++)
-		{
-		  sb.Append('0');
-		}
-
-		// write our pattern for encoding lengths
-		SimpleTextUtil.write(data, PATTERN);
-		SimpleTextUtil.write(data, sb.ToString(), scratch);
-		SimpleTextUtil.WriteNewline(data);
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final java.text.DecimalFormat encoder = new java.text.DecimalFormat(sb.toString(), new java.text.DecimalFormatSymbols(java.util.Locale.ROOT));
-		DecimalFormat encoder = new DecimalFormat(sb.ToString(), new DecimalFormatSymbols(Locale.ROOT));
-
-		// compute ord pattern: this is funny, we encode all values for all docs to find the maximum length
-		int maxOrdListLength = 0;
-		StringBuilder sb2 = new StringBuilder();
-		IEnumerator<Number> ordStream = ords.GetEnumerator();
-		foreach (Number n in docToOrdCount)
-		{
-		  sb2.Length = 0;
-		  int count = (int)n;
-		  for (int i = 0; i < count; i++)
-		  {
-//JAVA TO C# CONVERTER TODO TASK: Java iterators are only converted within the context of 'while' and 'for' loops:
-			long ord = (long)ordStream.next();
-			if (sb2.Length > 0)
-			{
-			  sb2.Append(",");
-			}
-			sb2.Append(Convert.ToString(ord));
-		  }
-		  maxOrdListLength = Math.Max(maxOrdListLength, sb2.Length);
-		}
-
-		sb2.Length = 0;
-		for (int i = 0; i < maxOrdListLength; i++)
-		{
-		  sb2.Append('X');
-		}
-
-		// write our pattern for ord lists
-		SimpleTextUtil.write(data, ORDPATTERN);
-		SimpleTextUtil.write(data, sb2.ToString(), scratch);
-		SimpleTextUtil.WriteNewline(data);
-
-		// for asserts:
-		long valuesSeen = 0;
-
-		foreach (BytesRef value in values)
-		{
-		  // write length
-		  SimpleTextUtil.write(data, LENGTH);
-		  SimpleTextUtil.write(data, encoder.format(value.length), scratch);
-		  SimpleTextUtil.WriteNewline(data);
-
-		  // write bytes -- don't use SimpleText.write
-		  // because it escapes:
-		  data.writeBytes(value.bytes, value.offset, value.length);
-
-		  // pad to fit
-		  for (int i = value.length; i < maxLength; i++)
-		  {
-			data.writeByte((sbyte)' ');
-		  }
-		  SimpleTextUtil.WriteNewline(data);
-		  valuesSeen++;
-		  Debug.Assert(valuesSeen <= valueCount);
-		}
-
-		Debug.Assert(valuesSeen == valueCount);
-
-		ordStream = ords.GetEnumerator();
-
-		// write the ords for each doc comma-separated
-		foreach (Number n in docToOrdCount)
-		{
-		  sb2.Length = 0;
-		  int count = (int)n;
-		  for (int i = 0; i < count; i++)
-		  {
-//JAVA TO C# CONVERTER TODO TASK: Java iterators are only converted within the context of 'while' and 'for' loops:
-			long ord = (long)ordStream.next();
-			if (sb2.Length > 0)
-			{
-			  sb2.Append(",");
-			}
-			sb2.Append(Convert.ToString(ord));
-		  }
-		  // now pad to fit: these are numbers so spaces work well. reader calls trim()
-		  int numPadding = maxOrdListLength - sb2.Length;
-		  for (int i = 0; i < numPadding; i++)
-		  {
-			sb2.Append(' ');
-		  }
-		  SimpleTextUtil.write(data, sb2.ToString(), scratch);
-		  SimpleTextUtil.WriteNewline(data);
-		}
-	  }
-
-	  /// <summary>
-	  /// write the header for this field </summary>
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: private void writeFieldEntry(index.FieldInfo field, index.FieldInfo.DocValuesType type) throws java.io.IOException
-	  private void writeFieldEntry(FieldInfo field, FieldInfo.DocValuesType type)
-	  {
-		SimpleTextUtil.write(data, FIELD);
-		SimpleTextUtil.write(data, field.name, scratch);
-		SimpleTextUtil.WriteNewline(data);
-
-		SimpleTextUtil.write(data, TYPE);
-		SimpleTextUtil.write(data, type.ToString(), scratch);
-		SimpleTextUtil.WriteNewline(data);
-	  }
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void close() throws java.io.IOException
-	  public override void close()
-	  {
-		if (data != null)
-		{
-		  bool success = false;
-		  try
-		  {
-			Debug.Assert(fieldsSeen.Count > 0);
-			// TODO: sheisty to do this here?
-			SimpleTextUtil.write(data, END);
-			SimpleTextUtil.WriteNewline(data);
-			SimpleTextUtil.WriteChecksum(data, scratch);
-			success = true;
-		  }
-		  finally
-		  {
-			if (success)
-			{
-			  IOUtils.close(data);
-			}
-			else
-			{
-			  IOUtils.closeWhileHandlingException(data);
-			}
-			data = null;
-		  }
-		}
-	  }
-	}
-
+            DecimalFormat encoder = new DecimalFormat(patternString, new DecimalFormatSymbols(Locale.ROOT));
+
+            int numDocsWritten = 0;
+
+            // second pass to write the values
+            foreach (Number n in values)
+            {
+                long value = n == null ? 0 : (long) n;
+                Debug.Assert(value >= minValue);
+                Number delta = System.Numerics.BigInteger.valueOf(value) - System.Numerics.BigInteger.valueOf(minValue);
+                string s = encoder.format(delta);
+                Debug.Assert(s.Length == patternString.Length);
+                SimpleTextUtil.Write(data, s, scratch);
+                SimpleTextUtil.WriteNewline(data);
+                if (n == null)
+                {
+                    SimpleTextUtil.Write(data, "F", scratch);
+                }
+                else
+                {
+                    SimpleTextUtil.Write(data, "T", scratch);
+                }
+                SimpleTextUtil.WriteNewline(data);
+                numDocsWritten++;
+                Debug.Assert(numDocsWritten <= numDocs);
+            }
+
+            Debug.Assert(numDocs == numDocsWritten, "numDocs=" + numDocs + " numDocsWritten=" + numDocsWritten);
+        }
+
+        public override void AddBinaryField(FieldInfo field, IEnumerable<BytesRef> values)
+        {
+            Debug.Assert(FieldSeen(field.Name));
+            Debug.Assert(field.DocValuesType == FieldInfo.DocValuesType_e.BINARY);
+
+            int maxLength = 0;
+            foreach (BytesRef value in values)
+            {
+                int length = value == null ? 0 : value.Length;
+                maxLength = Math.Max(maxLength, length);
+            }
+            WriteFieldEntry(field, FieldInfo.DocValuesType_e.BINARY);
+
+            // write maxLength
+            SimpleTextUtil.Write(data, MAXLENGTH);
+            SimpleTextUtil.Write(data, Convert.ToString(maxLength), scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            var maxBytesLength = Convert.ToString(maxLength).Length;
+            var sb = new StringBuilder();
+            for (var i = 0; i < maxBytesLength; i++)
+            {
+                sb.Append('0');
+            }
+            // write our pattern for encoding lengths
+            SimpleTextUtil.Write(data, PATTERN);
+            SimpleTextUtil.Write(data, sb.ToString(), scratch);
+            SimpleTextUtil.WriteNewline(data);
+            
+            DecimalFormat encoder = new DecimalFormat(sb.ToString(), new DecimalFormatSymbols(Locale.ROOT));
+
+            int numDocsWritten = 0;
+            foreach (BytesRef value in values)
+            {
+                int length = value == null ? 0 : value.Length;
+                SimpleTextUtil.Write(data, LENGTH);
+                SimpleTextUtil.Write(data, encoder.format(length), scratch);
+                SimpleTextUtil.WriteNewline(data);
+
+                // write bytes -- don't use SimpleText.Write
+                // because it escapes:
+                if (value != null)
+                {
+                    data.WriteBytes(value.bytes, value.offset, value.length);
+                }
+
+                // pad to fit
+                for (int i = length; i < maxLength; i++)
+                {
+                    data.WriteByte((sbyte) ' ');
+                }
+                SimpleTextUtil.WriteNewline(data);
+                if (value == null)
+                {
+                    SimpleTextUtil.Write(data, "F", scratch);
+                }
+                else
+                {
+                    SimpleTextUtil.Write(data, "T", scratch);
+                }
+                SimpleTextUtil.WriteNewline(data);
+                numDocsWritten++;
+            }
+
+            Debug.Assert(numDocs == numDocsWritten);
+        }
+
+        public override void AddSortedField(FieldInfo field, IEnumerable<BytesRef> values, IEnumerable<long> docToOrd)
+        {
+            Debug.Assert(FieldSeen(field.Name));
+            Debug.Assert(field.DocValuesType == FieldInfo.DocValuesType_e.SORTED);
+            WriteFieldEntry(field, FieldInfo.DocValuesType_e.SORTED);
+
+            int valueCount = 0;
+            int maxLength = -1;
+            foreach (BytesRef value in values)
+            {
+                maxLength = Math.Max(maxLength, value.Length);
+                valueCount++;
+            }
+
+            // write numValues
+            SimpleTextUtil.Write(data, NUMVALUES);
+            SimpleTextUtil.Write(data, Convert.ToString(valueCount), scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            // write maxLength
+            SimpleTextUtil.Write(data, MAXLENGTH);
+            SimpleTextUtil.Write(data, Convert.ToString(maxLength), scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            int maxBytesLength = Convert.ToString(maxLength).Length;
+            StringBuilder sb = new StringBuilder();
+            for (int i = 0; i < maxBytesLength; i++)
+            {
+                sb.Append('0');
+            }
+
+            // write our pattern for encoding lengths
+            SimpleTextUtil.Write(data, PATTERN);
+            SimpleTextUtil.Write(data, sb.ToString(), scratch);
+            SimpleTextUtil.WriteNewline(data);
+            
+            DecimalFormat encoder = new DecimalFormat(sb.ToString(), new DecimalFormatSymbols(Locale.ROOT));
+
+            int maxOrdBytes = Convert.ToString(valueCount + 1L).Length;
+            sb.Length = 0;
+            for (int i = 0; i < maxOrdBytes; i++)
+            {
+                sb.Append('0');
+            }
+
+            // write our pattern for ords
+            SimpleTextUtil.Write(data, ORDPATTERN);
+            SimpleTextUtil.Write(data, sb.ToString(), scratch);
+            SimpleTextUtil.WriteNewline(data);
+            
+            DecimalFormat ordEncoder = new DecimalFormat(sb.ToString(), new DecimalFormatSymbols(Locale.ROOT));
+
+            // for asserts:
+            int valuesSeen = 0;
+
+            foreach (BytesRef value in values)
+            {
+                // write length
+                SimpleTextUtil.Write(data, LENGTH);
+                SimpleTextUtil.Write(data, encoder.format(value.Length), scratch);
+                SimpleTextUtil.WriteNewline(data);
+
+                // write bytes -- don't use SimpleText.Write
+                // because it escapes:
+                data.WriteBytes(value.Bytes, value.Offset, value.Length);
+
+                // pad to fit
+                for (int i = value.Length; i < maxLength; i++)
+                {
+                    data.WriteByte((sbyte) ' ');
+                }
+                SimpleTextUtil.WriteNewline(data);
+                valuesSeen++;
+                Debug.Assert(valuesSeen <= valueCount);
+            }
+
+            Debug.Assert(valuesSeen == valueCount);
+
+            foreach (var ord in docToOrd)
+            {
+                SimpleTextUtil.Write(data, ordEncoder.format(ord + 1), scratch);
+                SimpleTextUtil.WriteNewline(data);
+            }
+        }
+
+        public override void AddSortedSetField(FieldInfo field, IEnumerable<BytesRef> values,
+            IEnumerable<long> docToOrdCount, IEnumerable<long> ords)
+        {
+            Debug.Assert(FieldSeen(field.Name));
+            Debug.Assert(field.DocValuesType == FieldInfo.DocValuesType_e.SORTED_SET);
+            WriteFieldEntry(field, FieldInfo.DocValuesType_e.SORTED_SET);
+
+            long valueCount = 0;
+            int maxLength = 0;
+            foreach (BytesRef value in values)
+            {
+                maxLength = Math.Max(maxLength, value.Length);
+                valueCount++;
+            }
+
+            // write numValues
+            SimpleTextUtil.Write(data, NUMVALUES);
+            SimpleTextUtil.Write(data, Convert.ToString(valueCount), scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            // write maxLength
+            SimpleTextUtil.Write(data, MAXLENGTH);
+            SimpleTextUtil.Write(data, Convert.ToString(maxLength), scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            int maxBytesLength = Convert.ToString(maxLength).Length;
+            StringBuilder sb = new StringBuilder();
+            for (int i = 0; i < maxBytesLength; i++)
+            {
+                sb.Append('0');
+            }
+
+            // write our pattern for encoding lengths
+            SimpleTextUtil.Write(data, PATTERN);
+            SimpleTextUtil.Write(data, sb.ToString(), scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            DecimalFormat encoder = new DecimalFormat(sb.ToString(), new DecimalFormatSymbols(Locale.ROOT));
+
+            // compute ord pattern: this is funny, we encode all values for all docs to find the maximum length
+            var maxOrdListLength = 0;
+            var sb2 = new StringBuilder();
+            var ordStream = ords.GetEnumerator();
+            foreach (var n in docToOrdCount)
+            {
+                sb2.Length = 0;
+                int count = (int) n;
+                for (int i = 0; i < count; i++)
+                {
+                    long ord = (long) ordStream.next();
+                    if (sb2.Length > 0)
+                    {
+                        sb2.Append(",");
+                    }
+                    sb2.Append(Convert.ToString(ord));
+                }
+                maxOrdListLength = Math.Max(maxOrdListLength, sb2.Length);
+            }
+
+            sb2.Length = 0;
+            for (int i = 0; i < maxOrdListLength; i++)
+            {
+                sb2.Append('X');
+            }
+
+            // write our pattern for ord lists
+            SimpleTextUtil.Write(data, ORDPATTERN);
+            SimpleTextUtil.Write(data, sb2.ToString(), scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            // for asserts:
+            long valuesSeen = 0;
+
+            foreach (BytesRef value in values)
+            {
+                // write length
+                SimpleTextUtil.Write(data, LENGTH);
+                SimpleTextUtil.Write(data, encoder.format(value.Length), scratch);
+                SimpleTextUtil.WriteNewline(data);
+
+                // write bytes -- don't use SimpleText.Write
+                // because it escapes:
+                data.WriteBytes(value.Bytes, value.Offset, value.Length);
+
+                // pad to fit
+                for (int i = value.Length; i < maxLength; i++)
+                {
+                    data.WriteByte((sbyte) ' ');
+                }
+                SimpleTextUtil.WriteNewline(data);
+                valuesSeen++;
+                Debug.Assert(valuesSeen <= valueCount);
+            }
+
+            Debug.Assert(valuesSeen == valueCount);
+
+            ordStream = ords.GetEnumerator();
+
+            // write the ords for each doc comma-separated
+            foreach (var n in docToOrdCount)
+            {
+                sb2.Length = 0;
+                int count = (int) n;
+                for (int i = 0; i < count; i++)
+                {
+                    long ord = (long) ordStream.Next();
+                    if (sb2.Length > 0)
+                    {
+                        sb2.Append(",");
+                    }
+                    sb2.Append(Convert.ToString(ord));
+                }
+                // now pad to fit: these are numbers so spaces work well. reader calls trim()
+                int numPadding = maxOrdListLength - sb2.Length;
+                for (int i = 0; i < numPadding; i++)
+                {
+                    sb2.Append(' ');
+                }
+                SimpleTextUtil.Write(data, sb2.ToString(), scratch);
+                SimpleTextUtil.WriteNewline(data);
+            }
+        }
+
+        protected override void Dispose(bool disposing)
+        {
+            if (data == null || disposing) return;
+            var success = false;
+            try
+            {
+                Debug.Assert(_fieldsSeen.Count > 0);
+                // TODO: sheisty to do this here?
+                SimpleTextUtil.Write(data, END);
+                SimpleTextUtil.WriteNewline(data);
+                SimpleTextUtil.WriteChecksum(data, scratch);
+                success = true;
+            }
+            finally
+            {
+                if (success)
+                {
+                    IOUtils.Close(data);
+                }
+                else
+                {
+                    IOUtils.CloseWhileHandlingException(data);
+                }
+                data = null;
+            }
+        }
+
+        /// <summary>Write the header for this field </summary>
+        private void WriteFieldEntry(FieldInfo field, FieldInfo.DocValuesType_e type)
+        {
+            SimpleTextUtil.Write(data, FIELD);
+            SimpleTextUtil.Write(data, field.Name, scratch);
+            SimpleTextUtil.WriteNewline(data);
+
+            SimpleTextUtil.Write(data, TYPE);
+            SimpleTextUtil.Write(data, type.ToString(), scratch);
+            SimpleTextUtil.WriteNewline(data);
+        }
+
+        /// <summary>
+        /// For Asserting
+        /// </summary>
+        /// <param name="field"></param>
+        /// <returns></returns>
+        private bool FieldSeen(string field)
+        {
+            Debug.Assert(!_fieldsSeen.Contains(field), "field \"" + field + "\" was added more than once during flush");
+            _fieldsSeen.Add(field);
+            return true;
+        }
+
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/07150be0/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosReader.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosReader.cs b/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosReader.cs
index 1dd0ed2..fff24f0 100644
--- a/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosReader.cs
+++ b/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosReader.cs
@@ -1,190 +1,172 @@
-using System;
-using System.Diagnostics;
-using System.Collections.Generic;
+/*
+ * 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.
+ */
+
+
+using System.Collections.ObjectModel;
 
 namespace Lucene.Net.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.
-	 */
-
-
-	using FieldInfo = index.FieldInfo;
-	using DocValuesType = index.FieldInfo.DocValuesType;
-	using FieldInfos = index.FieldInfos;
-	using IndexFileNames = index.IndexFileNames;
-	using IndexOptions = index.FieldInfo.IndexOptions;
-	using ChecksumIndexInput = store.ChecksumIndexInput;
-	using Directory = store.Directory;
-	using IOContext = store.IOContext;
-	using BytesRef = util.BytesRef;
-	using IOUtils = util.IOUtils;
-	using StringHelper = util.StringHelper;
-
-//JAVA TO C# CONVERTER TODO TASK: This Java 'import static' statement cannot be converted to .NET:
-	import static Lucene.Net.Codecs.SimpleText.SimpleTextFieldInfosWriter.*;
-
-	/// <summary>
-	/// reads plaintext field infos files
-	/// <para>
-	/// <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
-	/// @lucene.experimental
-	/// </para>
-	/// </summary>
-	public class SimpleTextFieldInfosReader : FieldInfosReader
-	{
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public index.FieldInfos read(store.Directory directory, String segmentName, String segmentSuffix, store.IOContext iocontext) throws java.io.IOException
-	  public override FieldInfos read(Directory directory, string segmentName, string segmentSuffix, IOContext iocontext)
-	  {
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final String fileName = index.IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
-		string fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
-		ChecksumIndexInput input = directory.openChecksumInput(fileName, iocontext);
-		BytesRef scratch = new BytesRef();
-
-		bool success = false;
-		try
-		{
-
-		  SimpleTextUtil.ReadLine(input, scratch);
-		  Debug.Assert(StringHelper.StartsWith(scratch, NUMFIELDS));
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final int size = Integer.parseInt(readString(NUMFIELDS.length, scratch));
-		  int size = Convert.ToInt32(readString(NUMFIELDS.length, scratch));
-		  FieldInfo[] infos = new FieldInfo[size];
-
-		  for (int i = 0; i < size; i++)
-		  {
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, NAME));
-			string name = readString(NAME.length, scratch);
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, NUMBER));
-			int fieldNumber = Convert.ToInt32(readString(NUMBER.length, scratch));
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, ISINDEXED));
-			bool isIndexed = Convert.ToBoolean(readString(ISINDEXED.length, scratch));
-
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final index.FieldInfo.IndexOptions indexOptions;
-			FieldInfo.IndexOptions indexOptions;
-			if (isIndexed)
-			{
-			  SimpleTextUtil.ReadLine(input, scratch);
-			  Debug.Assert(StringHelper.StartsWith(scratch, INDEXOPTIONS));
-			  indexOptions = FieldInfo.IndexOptions.valueOf(readString(INDEXOPTIONS.length, scratch));
-			}
-			else
-			{
-			  indexOptions = null;
-			}
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, STORETV));
-			bool storeTermVector = Convert.ToBoolean(readString(STORETV.length, scratch));
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, PAYLOADS));
-			bool storePayloads = Convert.ToBoolean(readString(PAYLOADS.length, scratch));
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, NORMS));
-			bool omitNorms = !Convert.ToBoolean(readString(NORMS.length, scratch));
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, NORMS_TYPE));
-			string nrmType = readString(NORMS_TYPE.length, scratch);
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final index.FieldInfo.DocValuesType normsType = docValuesType(nrmType);
-			FieldInfo.DocValuesType normsType = docValuesType(nrmType);
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, DOCVALUES));
-			string dvType = readString(DOCVALUES.length, scratch);
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final index.FieldInfo.DocValuesType docValuesType = docValuesType(dvType);
-			FieldInfo.DocValuesType docValuesType = docValuesType(dvType);
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, DOCVALUES_GEN));
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final long dvGen = Long.parseLong(readString(DOCVALUES_GEN.length, scratch));
-			long dvGen = Convert.ToInt64(readString(DOCVALUES_GEN.length, scratch));
-
-			SimpleTextUtil.ReadLine(input, scratch);
-			Debug.Assert(StringHelper.StartsWith(scratch, NUM_ATTS));
-			int numAtts = Convert.ToInt32(readString(NUM_ATTS.length, scratch));
-			IDictionary<string, string> atts = new Dictionary<string, string>();
-
-			for (int j = 0; j < numAtts; j++)
-			{
-			  SimpleTextUtil.ReadLine(input, scratch);
-			  Debug.Assert(StringHelper.StartsWith(scratch, ATT_KEY));
-			  string key = readString(ATT_KEY.length, scratch);
-
-			  SimpleTextUtil.ReadLine(input, scratch);
-			  Debug.Assert(StringHelper.StartsWith(scratch, ATT_VALUE));
-			  string value = readString(ATT_VALUE.length, scratch);
-			  atts[key] = value;
-			}
-
-			infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(atts));
-			infos[i].DocValuesGen = dvGen;
-		  }
-
-		  SimpleTextUtil.CheckFooter(input);
-
-		  FieldInfos fieldInfos = new FieldInfos(infos);
-		  success = true;
-		  return fieldInfos;
-		}
-		finally
-		{
-		  if (success)
-		  {
-			input.close();
-		  }
-		  else
-		  {
-			IOUtils.closeWhileHandlingException(input);
-		  }
-		}
-	  }
-
-	  public virtual FieldInfo.DocValuesType docValuesType(string dvType)
-	  {
-		if ("false".Equals(dvType))
-		{
-		  return null;
-		}
-		else
-		{
-		  return FieldInfo.DocValuesType.valueOf(dvType);
-		}
-	  }
-
-	  private string readString(int offset, BytesRef scratch)
-	  {
-		return new string(scratch.bytes, scratch.offset + offset, scratch.length - offset, StandardCharsets.UTF_8);
-	  }
-	}
-
+    using System;
+    using System.Diagnostics;
+    using System.Collections.Generic;
+    using Support;
+
+    using FieldInfo = Index.FieldInfo;
+    using DocValuesType = Index.FieldInfo.DocValuesType_e;
+    using FieldInfos = Index.FieldInfos;
+    using IndexFileNames = Index.IndexFileNames;
+    using Directory = Store.Directory;
+    using IOContext = Store.IOContext;
+    using BytesRef = Util.BytesRef;
+    using IOUtils = Util.IOUtils;
+    using StringHelper = Util.StringHelper;
+
+    /// <summary>
+    /// reads plaintext field infos files
+    /// <para>
+    /// <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+    /// @lucene.experimental
+    /// </para>
+    /// </summary>
+    public class SimpleTextFieldInfosReader : FieldInfosReader
+    {
+        public override FieldInfos Read(Directory directory, string segmentName, string segmentSuffix,
+            IOContext iocontext)
+        {
+            var fileName = IndexFileNames.SegmentFileName(segmentName, segmentSuffix,
+                SimpleTextFieldInfosWriter.FIELD_INFOS_EXTENSION);
+            var input = directory.OpenChecksumInput(fileName, iocontext);
+            var scratch = new BytesRef();
+
+            var success = false;
+            try
+            {
+
+                SimpleTextUtil.ReadLine(input, scratch);
+                Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.NUMFIELDS));
+                var size = Convert.ToInt32(ReadString(SimpleTextFieldInfosWriter.NUMFIELDS.Length, scratch));
+                var infos = new FieldInfo[size];
+
+                for (var i = 0; i < size; i++)
+                {
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.NAME));
+                    string name = ReadString(SimpleTextFieldInfosWriter.NAME.Length, scratch);
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.NUMBER));
+                    int fieldNumber = Convert.ToInt32(ReadString(SimpleTextFieldInfosWriter.NUMBER.Length, scratch));
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.ISINDEXED));
+                    bool isIndexed = Convert.ToBoolean(ReadString(SimpleTextFieldInfosWriter.ISINDEXED.Length, scratch));
+
+                    FieldInfo.IndexOptions? indexOptions;
+                    if (isIndexed)
+                    {
+                        SimpleTextUtil.ReadLine(input, scratch);
+                        Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.INDEXOPTIONS));
+                        indexOptions = (FieldInfo.IndexOptions)Enum.Parse(typeof(FieldInfo.IndexOptions), ReadString(SimpleTextFieldInfosWriter.INDEXOPTIONS.Length,
+                                scratch));
+                    }
+                    else
+                    {
+                        indexOptions = null;
+                    }
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.STORETV));
+                    bool storeTermVector =
+                        Convert.ToBoolean(ReadString(SimpleTextFieldInfosWriter.STORETV.Length, scratch));
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.PAYLOADS));
+                    bool storePayloads =
+                        Convert.ToBoolean(ReadString(SimpleTextFieldInfosWriter.PAYLOADS.Length, scratch));
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.NORMS));
+                    bool omitNorms = !Convert.ToBoolean(ReadString(SimpleTextFieldInfosWriter.NORMS.Length, scratch));
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.NORMS_TYPE));
+                    string nrmType = ReadString(SimpleTextFieldInfosWriter.NORMS_TYPE.Length, scratch);
+                    FieldInfo.DocValuesType_e? normsType = DocValuesType(nrmType);
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.DOCVALUES));
+                    string dvType = ReadString(SimpleTextFieldInfosWriter.DOCVALUES.Length, scratch);
+                    FieldInfo.DocValuesType_e? docValuesType = DocValuesType(dvType);
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.DOCVALUES_GEN));
+                    long dvGen = Convert.ToInt64(ReadString(SimpleTextFieldInfosWriter.DOCVALUES_GEN.Length, scratch));
+
+                    SimpleTextUtil.ReadLine(input, scratch);
+                    Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.NUM_ATTS));
+                    int numAtts = Convert.ToInt32(ReadString(SimpleTextFieldInfosWriter.NUM_ATTS.Length, scratch));
+                    IDictionary<string, string> atts = new Dictionary<string, string>();
+
+                    for (int j = 0; j < numAtts; j++)
+                    {
+                        SimpleTextUtil.ReadLine(input, scratch);
+                        Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.ATT_KEY));
+                        string key = ReadString(SimpleTextFieldInfosWriter.ATT_KEY.Length, scratch);
+
+                        SimpleTextUtil.ReadLine(input, scratch);
+                        Debug.Assert(StringHelper.StartsWith(scratch, SimpleTextFieldInfosWriter.ATT_VALUE));
+                        string value = ReadString(SimpleTextFieldInfosWriter.ATT_VALUE.Length, scratch);
+                        atts[key] = value;
+                    }
+
+                    infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads,
+                        indexOptions, docValuesType, normsType, new ReadOnlyDictionary<string,string>(atts))
+                    {
+                        DocValuesGen = dvGen
+                    };
+                }
+
+                SimpleTextUtil.CheckFooter(input);
+
+                var fieldInfos = new FieldInfos(infos);
+                success = true;
+                return fieldInfos;
+            }
+            finally
+            {
+                if (success)
+                {
+                    input.Dispose();
+                }
+                else
+                {
+                    IOUtils.CloseWhileHandlingException(input);
+                }
+            }
+        }
+
+        public virtual FieldInfo.DocValuesType_e? DocValuesType(string dvType)
+        {
+            return "false".Equals(dvType) ? null : (FieldInfo.DocValuesType_e?)Enum.Parse(typeof(FieldInfo.DocValuesType_e), dvType);
+        }
+
+        private static string ReadString(int offset, BytesRef scratch)
+        {
+            return scratch.Bytes.SubList(scratch.Offset + offset, scratch.Length + offset).ToString();
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/07150be0/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosWriter.cs b/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosWriter.cs
index d90d428..355d360 100644
--- a/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosWriter.cs
+++ b/src/Lucene.Net.Codecs/SimpleText/SimpleTextFieldInfosWriter.cs
@@ -1,171 +1,165 @@
-using System;
-using System.Diagnostics;
-using System.Collections.Generic;
+/*
+ * 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
 {
 
-	/*
-	 * 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.
-	 */
-
-	using FieldInfo = index.FieldInfo;
-	using DocValuesType = index.FieldInfo.DocValuesType;
-	using FieldInfos = index.FieldInfos;
-	using IndexFileNames = index.IndexFileNames;
-	using IndexOptions = index.FieldInfo.IndexOptions;
-	using Directory = store.Directory;
-	using IOContext = store.IOContext;
-	using IndexOutput = store.IndexOutput;
-	using BytesRef = util.BytesRef;
-	using IOUtils = util.IOUtils;
-
-	/// <summary>
-	/// writes plaintext field infos files
-	/// <para>
-	/// <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
-	/// @lucene.experimental
-	/// </para>
-	/// </summary>
-	public class SimpleTextFieldInfosWriter : FieldInfosWriter
-	{
-
-	  /// <summary>
-	  /// Extension of field infos </summary>
-	  internal const string FIELD_INFOS_EXTENSION = "inf";
-
-	  internal static readonly BytesRef NUMFIELDS = new BytesRef("number of fields ");
-	  internal static readonly BytesRef NAME = new BytesRef("  name ");
-	  internal static readonly BytesRef NUMBER = new BytesRef("  number ");
-	  internal static readonly BytesRef ISINDEXED = new BytesRef("  indexed ");
-	  internal static readonly BytesRef STORETV = new BytesRef("  term vectors ");
-	  internal static readonly BytesRef STORETVPOS = new BytesRef("  term vector positions ");
-	  internal static readonly BytesRef STORETVOFF = new BytesRef("  term vector offsets ");
-	  internal static readonly BytesRef PAYLOADS = new BytesRef("  payloads ");
-	  internal static readonly BytesRef NORMS = new BytesRef("  norms ");
-	  internal static readonly BytesRef NORMS_TYPE = new BytesRef("  norms type ");
-	  internal static readonly BytesRef DOCVALUES = new BytesRef("  doc values ");
-	  internal static readonly BytesRef DOCVALUES_GEN = new BytesRef("  doc values gen ");
-	  internal static readonly BytesRef INDEXOPTIONS = new BytesRef("  index options ");
-	  internal static readonly BytesRef NUM_ATTS = new BytesRef("  attributes ");
-	  internal static readonly BytesRef ATT_KEY = new BytesRef("    key ");
-	  internal static readonly BytesRef ATT_VALUE = new BytesRef("    value ");
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void write(store.Directory directory, String segmentName, String segmentSuffix, index.FieldInfos infos, store.IOContext context) throws java.io.IOException
-	  public override void write(Directory directory, string segmentName, string segmentSuffix, FieldInfos infos, IOContext context)
-	  {
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final String fileName = index.IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
-		string fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
-		IndexOutput @out = directory.createOutput(fileName, context);
-		BytesRef scratch = new BytesRef();
-		bool success = false;
-		try
-		{
-		  SimpleTextUtil.write(@out, NUMFIELDS);
-		  SimpleTextUtil.write(@out, Convert.ToString(infos.size()), scratch);
-		  SimpleTextUtil.WriteNewline(@out);
-
-		  foreach (FieldInfo fi in infos)
-		  {
-			SimpleTextUtil.write(@out, NAME);
-			SimpleTextUtil.write(@out, fi.name, scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			SimpleTextUtil.write(@out, NUMBER);
-			SimpleTextUtil.write(@out, Convert.ToString(fi.number), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			SimpleTextUtil.write(@out, ISINDEXED);
-			SimpleTextUtil.write(@out, Convert.ToString(fi.Indexed), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			if (fi.Indexed)
-			{
-			  Debug.Assert(fi.IndexOptions.compareTo(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads());
-			  SimpleTextUtil.write(@out, INDEXOPTIONS);
-			  SimpleTextUtil.write(@out, fi.IndexOptions.ToString(), scratch);
-			  SimpleTextUtil.WriteNewline(@out);
-			}
-
-			SimpleTextUtil.write(@out, STORETV);
-			SimpleTextUtil.write(@out, Convert.ToString(fi.hasVectors()), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			SimpleTextUtil.write(@out, PAYLOADS);
-			SimpleTextUtil.write(@out, Convert.ToString(fi.hasPayloads()), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			SimpleTextUtil.write(@out, NORMS);
-			SimpleTextUtil.write(@out, Convert.ToString(!fi.omitsNorms()), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			SimpleTextUtil.write(@out, NORMS_TYPE);
-			SimpleTextUtil.write(@out, getDocValuesType(fi.NormType), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			SimpleTextUtil.write(@out, DOCVALUES);
-			SimpleTextUtil.write(@out, getDocValuesType(fi.DocValuesType), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			SimpleTextUtil.write(@out, DOCVALUES_GEN);
-			SimpleTextUtil.write(@out, Convert.ToString(fi.DocValuesGen), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			IDictionary<string, string> atts = fi.attributes();
-			int numAtts = atts == null ? 0 : atts.Count;
-			SimpleTextUtil.write(@out, NUM_ATTS);
-			SimpleTextUtil.write(@out, Convert.ToString(numAtts), scratch);
-			SimpleTextUtil.WriteNewline(@out);
-
-			if (numAtts > 0)
-			{
-			  foreach (KeyValuePair<string, string> entry in atts.SetOfKeyValuePairs())
-			  {
-				SimpleTextUtil.write(@out, ATT_KEY);
-				SimpleTextUtil.write(@out, entry.Key, scratch);
-				SimpleTextUtil.WriteNewline(@out);
-
-				SimpleTextUtil.write(@out, ATT_VALUE);
-				SimpleTextUtil.write(@out, entry.Value, scratch);
-				SimpleTextUtil.WriteNewline(@out);
-			  }
-			}
-		  }
-		  SimpleTextUtil.WriteChecksum(@out, scratch);
-		  success = true;
-		}
-		finally
-		{
-		  if (success)
-		  {
-			@out.close();
-		  }
-		  else
-		  {
-			IOUtils.closeWhileHandlingException(@out);
-		  }
-		}
-	  }
-
-	  private static string getDocValuesType(FieldInfo.DocValuesType type)
-	  {
-		return type == null ? "false" : type.ToString();
-	  }
-	}
+    using System;
+    using System.Diagnostics;
+    using System.Collections.Generic;
+
+    using FieldInfo = Index.FieldInfo;
+    using DocValuesType = Index.FieldInfo.DocValuesType_e;
+    using FieldInfos = Index.FieldInfos;
+    using IndexFileNames = Index.IndexFileNames;
+    using Directory = Store.Directory;
+    using IOContext = Store.IOContext;
+    using BytesRef = Util.BytesRef;
+    using IOUtils = Util.IOUtils;
+
+    /// <summary>
+    /// writes plaintext field infos files
+    /// <para>
+    /// <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+    /// @lucene.experimental
+    /// </para>
+    /// </summary>
+    public class SimpleTextFieldInfosWriter : FieldInfosWriter
+    {
+
+        /// <summary>
+        /// Extension of field infos </summary>
+        internal const string FIELD_INFOS_EXTENSION = "inf";
+
+        internal static readonly BytesRef NUMFIELDS = new BytesRef("number of fields ");
+        internal static readonly BytesRef NAME = new BytesRef("  name ");
+        internal static readonly BytesRef NUMBER = new BytesRef("  number ");
+        internal static readonly BytesRef ISINDEXED = new BytesRef("  indexed ");
+        internal static readonly BytesRef STORETV = new BytesRef("  term vectors ");
+        internal static readonly BytesRef STORETVPOS = new BytesRef("  term vector positions ");
+        internal static readonly BytesRef STORETVOFF = new BytesRef("  term vector offsets ");
+        internal static readonly BytesRef PAYLOADS = new BytesRef("  payloads ");
+        internal static readonly BytesRef NORMS = new BytesRef("  norms ");
+        internal static readonly BytesRef NORMS_TYPE = new BytesRef("  norms type ");
+        internal static readonly BytesRef DOCVALUES = new BytesRef("  doc values ");
+        internal static readonly BytesRef DOCVALUES_GEN = new BytesRef("  doc values gen ");
+        internal static readonly BytesRef INDEXOPTIONS = new BytesRef("  index options ");
+        internal static readonly BytesRef NUM_ATTS = new BytesRef("  attributes ");
+        internal static readonly BytesRef ATT_KEY = new BytesRef("    key ");
+        internal static readonly BytesRef ATT_VALUE = new BytesRef("    value ");
+
+        public override void Write(Directory directory, string segmentName, string segmentSuffix, FieldInfos infos,
+            IOContext context)
+        {
+            var fileName = IndexFileNames.SegmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
+            var output = directory.CreateOutput(fileName, context);
+            var scratch = new BytesRef();
+            var success = false;
+
+            try
+            {
+                SimpleTextUtil.Write(output, NUMFIELDS);
+                SimpleTextUtil.Write(output, Convert.ToString(infos.Size()), scratch);
+                SimpleTextUtil.WriteNewline(output);
+
+                foreach (FieldInfo fi in infos)
+                {
+                    SimpleTextUtil.Write(output, NAME);
+                    SimpleTextUtil.Write(output, fi.Name, scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    SimpleTextUtil.Write(output, NUMBER);
+                    SimpleTextUtil.Write(output, Convert.ToString(fi.Number), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    SimpleTextUtil.Write(output, ISINDEXED);
+                    SimpleTextUtil.Write(output, Convert.ToString(fi.Indexed), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    if (fi.Indexed)
+                    {
+                        Debug.Assert(fi.FieldIndexOptions >= FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !fi.HasPayloads());
+                        SimpleTextUtil.Write(output, INDEXOPTIONS);
+                        SimpleTextUtil.Write(output, fi.FieldIndexOptions.ToString(), scratch);
+                        SimpleTextUtil.WriteNewline(output);
+                    }
+
+                    SimpleTextUtil.Write(output, STORETV);
+                    SimpleTextUtil.Write(output, Convert.ToString(fi.HasVectors()), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    SimpleTextUtil.Write(output, PAYLOADS);
+                    SimpleTextUtil.Write(output, Convert.ToString(fi.HasPayloads()), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    SimpleTextUtil.Write(output, NORMS);
+                    SimpleTextUtil.Write(output, Convert.ToString(!fi.OmitsNorms()), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    SimpleTextUtil.Write(output, NORMS_TYPE);
+                    SimpleTextUtil.Write(output, GetDocValuesType(fi.NormType), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    SimpleTextUtil.Write(output, DOCVALUES);
+                    SimpleTextUtil.Write(output, GetDocValuesType(fi.DocValuesType), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    SimpleTextUtil.Write(output, DOCVALUES_GEN);
+                    SimpleTextUtil.Write(output, Convert.ToString(fi.DocValuesGen), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    IDictionary<string, string> atts = fi.Attributes();
+                    int numAtts = atts == null ? 0 : atts.Count;
+                    SimpleTextUtil.Write(output, NUM_ATTS);
+                    SimpleTextUtil.Write(output, Convert.ToString(numAtts), scratch);
+                    SimpleTextUtil.WriteNewline(output);
+
+                    if (numAtts <= 0 || atts == null) continue;
+                    foreach (var entry in atts)
+                    {
+                        SimpleTextUtil.Write(output, ATT_KEY);
+                        SimpleTextUtil.Write(output, entry.Key, scratch);
+                        SimpleTextUtil.WriteNewline(output);
+
+                        SimpleTextUtil.Write(output, ATT_VALUE);
+                        SimpleTextUtil.Write(output, entry.Value, scratch);
+                        SimpleTextUtil.WriteNewline(output);
+                    }
+                }
+                SimpleTextUtil.WriteChecksum(output, scratch);
+                success = true;
+            }
+            finally
+            {
+                if (success)
+                {
+                    output.Dispose();
+                }
+                else
+                {
+                    IOUtils.CloseWhileHandlingException(output);
+                }
+            }
+        }
+
+        private static string GetDocValuesType(FieldInfo.DocValuesType_e? type)
+        {
+            return type.HasValue ? type.ToString() : "false";
+        }
+    }
 
 }
\ No newline at end of file