You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by sy...@apache.org on 2014/09/19 16:19:42 UTC

[09/21] Moving ValueSource -> ValueSources to avoid name conflicts

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/MultiFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/MultiFloatFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/MultiFloatFunction.cs
new file mode 100644
index 0000000..a32c2c0
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/MultiFloatFunction.cs
@@ -0,0 +1,141 @@
+using System.Collections;
+using System.Text;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+	/*
+	 * 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.
+	 */
+    /// <summary>
+	/// Abstract <seealso cref="ValueSource"/> implementation which wraps multiple ValueSources
+	/// and applies an extendible float function to their values.
+	/// 
+	/// </summary>
+	public abstract class MultiFloatFunction : ValueSource
+	{
+	  protected internal readonly ValueSource[] sources;
+
+	  public MultiFloatFunction(ValueSource[] sources)
+	  {
+		this.sources = sources;
+	  }
+
+	  protected internal abstract string name();
+	  protected internal abstract float func(int doc, FunctionValues[] valsArr);
+
+	  public override string description()
+	  {
+		StringBuilder sb = new StringBuilder();
+		sb.Append(name()).Append('(');
+		bool firstTime = true;
+		foreach (ValueSource source in sources)
+		{
+		  if (firstTime)
+		  {
+			firstTime = false;
+		  }
+		  else
+		  {
+			sb.Append(',');
+		  }
+		  sb.Append((object) source);
+		}
+		sb.Append(')');
+		return sb.ToString();
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.queries.function.FunctionValues[] valsArr = new org.apache.lucene.queries.function.FunctionValues[sources.length];
+		FunctionValues[] valsArr = new FunctionValues[sources.Length];
+		for (int i = 0; i < sources.Length; i++)
+		{
+		  valsArr[i] = sources[i].getValues(context, readerContext);
+		}
+
+		return new FloatDocValuesAnonymousInnerClassHelper(this, this, valsArr);
+	  }
+
+	  private class FloatDocValuesAnonymousInnerClassHelper : FloatDocValues
+	  {
+		  private readonly MultiFloatFunction outerInstance;
+
+		  private FunctionValues[] valsArr;
+
+		  public FloatDocValuesAnonymousInnerClassHelper(MultiFloatFunction outerInstance, MultiFloatFunction this, FunctionValues[] valsArr) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.valsArr = valsArr;
+		  }
+
+		  public override float floatVal(int doc)
+		  {
+			return outerInstance.func(doc, valsArr);
+		  }
+		   public override string ToString(int doc)
+		   {
+			StringBuilder sb = new StringBuilder();
+			sb.Append(outerInstance.name()).Append('(');
+			bool firstTime = true;
+			foreach (FunctionValues vals in valsArr)
+			{
+			  if (firstTime)
+			  {
+				firstTime = false;
+			  }
+			  else
+			  {
+				sb.Append(',');
+			  }
+			  sb.Append(vals.ToString(doc));
+			}
+			sb.Append(')');
+			return sb.ToString();
+		   }
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		foreach (ValueSource source in sources)
+		{
+		  source.createWeight(context, searcher);
+		}
+	  }
+
+	  public override int GetHashCode()
+	  {
+		return Arrays.GetHashCode(sources) + name().GetHashCode();
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (this.GetType() != o.GetType())
+		{
+			return false;
+		}
+		MultiFloatFunction other = (MultiFloatFunction)o;
+		return this.name().Equals(other.name()) && Arrays.Equals(this.sources, other.sources);
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/MultiFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/MultiFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/MultiFunction.cs
new file mode 100644
index 0000000..91a13fb
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/MultiFunction.cs
@@ -0,0 +1,156 @@
+using System.Collections;
+using System.Collections.Generic;
+using System.Text;
+using org.apache.lucene.queries.function;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+	/*
+	 * 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.
+	 */
+    /// <summary>
+	/// Abstract parent class for <seealso cref="ValueSource"/> implementations that wrap multiple
+	/// ValueSources and apply their own logic.
+	/// </summary>
+	public abstract class MultiFunction : ValueSource
+	{
+	  protected internal readonly IList<ValueSource> sources;
+
+	  public MultiFunction(IList<ValueSource> sources)
+	  {
+		this.sources = sources;
+	  }
+
+	  protected internal abstract string name();
+
+	  public override string description()
+	  {
+		return description(name(), sources);
+	  }
+
+	  public static string description(string name, IList<ValueSource> sources)
+	  {
+		StringBuilder sb = new StringBuilder();
+		sb.Append(name).Append('(');
+		bool firstTime = true;
+		foreach (ValueSource source in sources)
+		{
+		  if (firstTime)
+		  {
+			firstTime = false;
+		  }
+		  else
+		  {
+			sb.Append(',');
+		  }
+		  sb.Append((object) source);
+		}
+		sb.Append(')');
+		return sb.ToString();
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: public static org.apache.lucene.queries.function.FunctionValues[] valsArr(java.util.List<org.apache.lucene.queries.function.ValueSource> sources, java.util.Map fcontext, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public static FunctionValues[] valsArr(IList<ValueSource> sources, IDictionary fcontext, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.queries.function.FunctionValues[] valsArr = new org.apache.lucene.queries.function.FunctionValues[sources.size()];
+		FunctionValues[] valsArr = new FunctionValues[sources.Count];
+		int i = 0;
+		foreach (ValueSource source in sources)
+		{
+		  valsArr[i++] = source.getValues(fcontext, readerContext);
+		}
+		return valsArr;
+	  }
+
+	  public class Values : FunctionValues
+	  {
+		  private readonly MultiFunction outerInstance;
+
+		internal readonly FunctionValues[] valsArr;
+
+		public Values(MultiFunction outerInstance, FunctionValues[] valsArr)
+		{
+			this.outerInstance = outerInstance;
+		  this.valsArr = valsArr;
+		}
+
+		public override string ToString(int doc)
+		{
+		  return MultiFunction.ToString(outerInstance.name(), valsArr, doc);
+		}
+
+		public override ValueFiller ValueFiller
+		{
+			get
+			{
+			  // TODO: need ValueSource.type() to determine correct type
+			  return base.ValueFiller;
+			}
+		}
+	  }
+
+
+	  public static string ToString(string name, FunctionValues[] valsArr, int doc)
+	  {
+		StringBuilder sb = new StringBuilder();
+		sb.Append(name).Append('(');
+		bool firstTime = true;
+		foreach (FunctionValues vals in valsArr)
+		{
+		  if (firstTime)
+		  {
+			firstTime = false;
+		  }
+		  else
+		  {
+			sb.Append(',');
+		  }
+		  sb.Append(vals.ToString(doc));
+		}
+		sb.Append(')');
+		return sb.ToString();
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		foreach (ValueSource source in sources)
+		{
+		  source.createWeight(context, searcher);
+		}
+	  }
+
+	  public override int GetHashCode()
+	  {
+		return sources.GetHashCode() + name().GetHashCode();
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (this.GetType() != o.GetType())
+		{
+			return false;
+		}
+		MultiFunction other = (MultiFunction)o;
+		return this.sources.Equals(other.sources);
+	  }
+	}
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/MultiValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/MultiValueSource.cs b/src/Lucene.Net.Queries/Function/ValueSources/MultiValueSource.cs
new file mode 100644
index 0000000..215d74e
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/MultiValueSource.cs
@@ -0,0 +1,32 @@
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+	/*
+	 * 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.
+	 */
+
+
+	/// <summary>
+	/// A <seealso cref="ValueSource"/> that abstractly represents <seealso cref="ValueSource"/>s for
+	/// poly fields, and other things.
+	/// 
+	/// </summary>
+	public abstract class MultiValueSource : ValueSource
+	{
+
+	  public abstract int dimension();
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/NormValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/NormValueSource.cs b/src/Lucene.Net.Queries/Function/ValueSources/NormValueSource.cs
new file mode 100644
index 0000000..babb01f
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/NormValueSource.cs
@@ -0,0 +1,118 @@
+/*
+ * 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;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// Function that returns <seealso cref="TFIDFSimilarity#decodeNormValue(long)"/>
+	/// for every document.
+	/// <para>
+	/// Note that the configured Similarity for the field must be
+	/// a subclass of <seealso cref="TFIDFSimilarity"/>
+	/// @lucene.internal 
+	/// </para>
+	/// </summary>
+	public class NormValueSource : ValueSource
+	{
+	  protected internal readonly string field;
+	  public NormValueSource(string field)
+	  {
+		this.field = field;
+	  }
+
+	  public virtual string name()
+	  {
+		return "norm";
+	  }
+
+	  public override string description()
+	  {
+		return name() + '(' + field + ')';
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		context["searcher"] = searcher;
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+		IndexSearcher searcher = (IndexSearcher)context["searcher"];
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.search.similarities.TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.getSimilarity(), field);
+		TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.Similarity, field);
+		if (similarity == null)
+		{
+		  throw new System.NotSupportedException("requires a TFIDFSimilarity (such as DefaultSimilarity)");
+		}
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.index.NumericDocValues norms = readerContext.reader().getNormValues(field);
+		NumericDocValues norms = readerContext.reader().getNormValues(field);
+
+		if (norms == null)
+		{
+		  return new ConstDoubleDocValues(0.0, this);
+		}
+
+		return new FloatDocValuesAnonymousInnerClassHelper(this, this, similarity, norms);
+	  }
+
+	  private class FloatDocValuesAnonymousInnerClassHelper : FloatDocValues
+	  {
+		  private readonly NormValueSource outerInstance;
+
+		  private TFIDFSimilarity similarity;
+		  private NumericDocValues norms;
+
+		  public FloatDocValuesAnonymousInnerClassHelper(NormValueSource outerInstance, NormValueSource this, TFIDFSimilarity similarity, NumericDocValues norms) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.similarity = similarity;
+			  this.norms = norms;
+		  }
+
+		  public override float floatVal(int doc)
+		  {
+			return similarity.decodeNormValue(norms.get(doc));
+		  }
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (this.GetType() != o.GetType())
+		{
+		  return false;
+		}
+		return this.field.Equals(((NormValueSource)o).field);
+	  }
+
+	  public override int GetHashCode()
+	  {
+		return this.GetType().GetHashCode() + field.GetHashCode();
+	  }
+	}
+
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/NumDocsValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/NumDocsValueSource.cs b/src/Lucene.Net.Queries/Function/ValueSources/NumDocsValueSource.cs
new file mode 100644
index 0000000..3a9014a
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/NumDocsValueSource.cs
@@ -0,0 +1,59 @@
+/*
+ * 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;
+using Lucene.Net.Index;
+using org.apache.lucene.queries.function;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// Returns the value of <seealso cref="IndexReader#numDocs()"/>
+	/// for every document. This is the number of documents
+	/// excluding deletions.
+	/// </summary>
+	public class NumDocsValueSource : ValueSource
+	{
+	  public virtual string name()
+	  {
+		return "numdocs";
+	  }
+
+	  public override string description()
+	  {
+		return name() + "()";
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+		// Searcher has no numdocs so we must use the reader instead
+		return new ConstIntDocValues(ReaderUtil.getTopLevelContext(readerContext).reader().numDocs(), this);
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		return this.GetType() == o.GetType();
+	  }
+
+	  public override int GetHashCode()
+	  {
+		return this.GetType().GetHashCode();
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/OrdFieldSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/OrdFieldSource.cs b/src/Lucene.Net.Queries/Function/ValueSources/OrdFieldSource.cs
new file mode 100644
index 0000000..4828d86
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/OrdFieldSource.cs
@@ -0,0 +1,162 @@
+/*
+ * 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;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// Obtains the ordinal of the field value from the default Lucene <seealso cref="org.apache.lucene.search.FieldCache"/> using getStringIndex().
+	/// <br>
+	/// The native lucene index order is used to assign an ordinal value for each field value.
+	/// <br>Field values (terms) are lexicographically ordered by unicode value, and numbered starting at 1.
+	/// <br>
+	/// Example:<br>
+	///  If there were only three field values: "apple","banana","pear"
+	/// <br>then ord("apple")=1, ord("banana")=2, ord("pear")=3
+	/// <para>
+	/// WARNING: ord() depends on the position in an index and can thus change when other documents are inserted or deleted,
+	///  or if a MultiSearcher is used.
+	/// <br>WARNING: as of Solr 1.4, ord() and rord() can cause excess memory use since they must use a FieldCache entry
+	/// at the top level reader, while sorting and function queries now use entries at the segment level.  Hence sorting
+	/// or using a different function query, in addition to ord()/rord() will double memory use.
+	/// 
+	/// </para>
+	/// </summary>
+
+	public class OrdFieldSource : ValueSource
+	{
+	  protected internal readonly string field;
+
+	  public OrdFieldSource(string field)
+	  {
+		this.field = field;
+	  }
+
+	  public override string description()
+	  {
+		return "ord(" + field + ')';
+	  }
+
+
+	  // TODO: this is trappy? perhaps this query instead should make you pass a slow reader yourself?
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final int off = readerContext.docBase;
+		int off = readerContext.docBase;
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.index.IndexReader topReader = org.apache.lucene.index.ReaderUtil.getTopLevelContext(readerContext).reader();
+		IndexReader topReader = ReaderUtil.getTopLevelContext(readerContext).reader();
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.index.AtomicReader r = org.apache.lucene.index.SlowCompositeReaderWrapper.wrap(topReader);
+		AtomicReader r = SlowCompositeReaderWrapper.wrap(topReader);
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.index.SortedDocValues sindex = org.apache.lucene.search.FieldCache.DEFAULT.getTermsIndex(r, field);
+		SortedDocValues sindex = FieldCache.DEFAULT.getTermsIndex(r, field);
+		return new IntDocValuesAnonymousInnerClassHelper(this, this, off, sindex);
+	  }
+
+	  private class IntDocValuesAnonymousInnerClassHelper : IntDocValues
+	  {
+		  private readonly OrdFieldSource outerInstance;
+
+		  private int off;
+		  private SortedDocValues sindex;
+
+		  public IntDocValuesAnonymousInnerClassHelper(OrdFieldSource outerInstance, OrdFieldSource this, int off, SortedDocValues sindex) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.off = off;
+			  this.sindex = sindex;
+		  }
+
+		  protected internal virtual string toTerm(string readableValue)
+		  {
+			return readableValue;
+		  }
+		  public override int intVal(int doc)
+		  {
+			return sindex.getOrd(doc + off);
+		  }
+		  public override int ordVal(int doc)
+		  {
+			return sindex.getOrd(doc + off);
+		  }
+		  public override int numOrd()
+		  {
+			return sindex.ValueCount;
+		  }
+
+		  public override bool exists(int doc)
+		  {
+			return sindex.getOrd(doc + off) != 0;
+		  }
+
+		  public override ValueFiller ValueFiller
+		  {
+			  get
+			  {
+				return new ValueFillerAnonymousInnerClassHelper(this);
+			  }
+		  }
+
+		  private class ValueFillerAnonymousInnerClassHelper : ValueFiller
+		  {
+			  private readonly IntDocValuesAnonymousInnerClassHelper outerInstance;
+
+			  public ValueFillerAnonymousInnerClassHelper(IntDocValuesAnonymousInnerClassHelper outerInstance)
+			  {
+				  this.outerInstance = outerInstance;
+				  mval = new MutableValueInt();
+			  }
+
+			  private readonly MutableValueInt mval;
+
+			  public override MutableValue Value
+			  {
+				  get
+				  {
+					return mval;
+				  }
+			  }
+
+			  public override void fillValue(int doc)
+			  {
+				mval.value = outerInstance.sindex.getOrd(doc);
+				mval.exists = mval.value != 0;
+			  }
+		  }
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		return o != null && o.GetType() == typeof(OrdFieldSource) && this.field.Equals(((OrdFieldSource)o).field);
+	  }
+
+	  private static readonly int hcode = typeof(OrdFieldSource).GetHashCode();
+	  public override int GetHashCode()
+	  {
+		return hcode + field.GetHashCode();
+	  }
+
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/PowFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/PowFloatFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/PowFloatFunction.cs
new file mode 100644
index 0000000..ec1be2a
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/PowFloatFunction.cs
@@ -0,0 +1,48 @@
+/*
+ * 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;
+using org.apache.lucene.queries.function;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+
+
+	/// <summary>
+	/// Function to raise the base "a" to the power "b"
+	/// </summary>
+	public class PowFloatFunction : DualFloatFunction
+	{
+	 /// <param name="a">  the base. </param>
+	 /// <param name="b">  the exponent. </param>
+	  public PowFloatFunction(ValueSource a, ValueSource b) : base(a,b)
+	  {
+	  }
+
+	  protected internal override string name()
+	  {
+		return "pow";
+	  }
+
+	  protected internal override float func(int doc, FunctionValues aVals, FunctionValues bVals)
+	  {
+		return (float)Math.Pow(aVals.floatVal(doc), bVals.floatVal(doc));
+	  }
+	}
+
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/ProductFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/ProductFloatFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/ProductFloatFunction.cs
new file mode 100644
index 0000000..85fb92b
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/ProductFloatFunction.cs
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.lucene.queries.function;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+
+
+	/// <summary>
+	/// <code>ProductFloatFunction</code> returns the product of it's components.
+	/// </summary>
+	public class ProductFloatFunction : MultiFloatFunction
+	{
+	  public ProductFloatFunction(ValueSource[] sources) : base(sources)
+	  {
+	  }
+
+	  protected internal override string name()
+	  {
+		return "product";
+	  }
+
+	  protected internal override float func(int doc, FunctionValues[] valsArr)
+	  {
+		float val = 1.0f;
+		foreach (FunctionValues vals in valsArr)
+		{
+		  val *= vals.floatVal(doc);
+		}
+		return val;
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/QueryValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/QueryValueSource.cs b/src/Lucene.Net.Queries/Function/ValueSources/QueryValueSource.cs
new file mode 100644
index 0000000..8ff11cb
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/QueryValueSource.cs
@@ -0,0 +1,319 @@
+/*
+ * 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;
+using System.Collections;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// <code>QueryValueSource</code> returns the relevance score of the query
+	/// </summary>
+	public class QueryValueSource : ValueSource
+	{
+	  internal readonly Query q;
+	  internal readonly float defVal;
+
+	  public QueryValueSource(Query q, float defVal)
+	  {
+		this.q = q;
+		this.defVal = defVal;
+	  }
+
+	  public virtual Query Query
+	  {
+		  get
+		  {
+			  return q;
+		  }
+	  }
+	  public virtual float DefaultValue
+	  {
+		  get
+		  {
+			  return defVal;
+		  }
+	  }
+
+	  public override string description()
+	  {
+		return "query(" + q + ",def=" + defVal + ")";
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map fcontext, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary fcontext, AtomicReaderContext readerContext)
+	  {
+		return new QueryDocValues(this, readerContext, fcontext);
+	  }
+
+	  public override int GetHashCode()
+	  {
+		return q.GetHashCode() * 29;
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (typeof(QueryValueSource) != o.GetType())
+		{
+			return false;
+		}
+		QueryValueSource other = (QueryValueSource)o;
+		return this.q.Equals(other.q) && this.defVal == other.defVal;
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		Weight w = searcher.createNormalizedWeight(q);
+		context[this] = w;
+	  }
+	}
+
+
+	internal class QueryDocValues : FloatDocValues
+	{
+	  internal readonly AtomicReaderContext readerContext;
+	  internal readonly Bits acceptDocs;
+	  internal readonly Weight weight;
+	  internal readonly float defVal;
+	  internal readonly IDictionary fcontext;
+	  internal readonly Query q;
+
+	  internal Scorer scorer;
+	  internal int scorerDoc; // the document the scorer is on
+	  internal bool noMatches = false;
+
+	  // the last document requested... start off with high value
+	  // to trigger a scorer reset on first access.
+	  internal int lastDocRequested = int.MaxValue;
+
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: public QueryDocValues(QueryValueSource vs, org.apache.lucene.index.AtomicReaderContext readerContext, java.util.Map fcontext) throws java.io.IOException
+	  public QueryDocValues(QueryValueSource vs, AtomicReaderContext readerContext, IDictionary fcontext) : base(vs)
+	  {
+
+		this.readerContext = readerContext;
+		this.acceptDocs = readerContext.reader().LiveDocs;
+		this.defVal = vs.defVal;
+		this.q = vs.q;
+		this.fcontext = fcontext;
+
+		Weight w = fcontext == null ? null : (Weight)fcontext[vs];
+		if (w == null)
+		{
+		  IndexSearcher weightSearcher;
+		  if (fcontext == null)
+		  {
+			weightSearcher = new IndexSearcher(ReaderUtil.getTopLevelContext(readerContext));
+		  }
+		  else
+		  {
+			weightSearcher = (IndexSearcher)fcontext["searcher"];
+			if (weightSearcher == null)
+			{
+			  weightSearcher = new IndexSearcher(ReaderUtil.getTopLevelContext(readerContext));
+			}
+		  }
+		  vs.createWeight(fcontext, weightSearcher);
+		  w = (Weight)fcontext[vs];
+		}
+		weight = w;
+	  }
+
+	  public override float floatVal(int doc)
+	  {
+		try
+		{
+		  if (doc < lastDocRequested)
+		  {
+			if (noMatches)
+			{
+				return defVal;
+			}
+			scorer = weight.scorer(readerContext, acceptDocs);
+			if (scorer == null)
+			{
+			  noMatches = true;
+			  return defVal;
+			}
+			scorerDoc = -1;
+		  }
+		  lastDocRequested = doc;
+
+		  if (scorerDoc < doc)
+		  {
+			scorerDoc = scorer.advance(doc);
+		  }
+
+		  if (scorerDoc > doc)
+		  {
+			// query doesn't match this document... either because we hit the
+			// end, or because the next doc is after this doc.
+			return defVal;
+		  }
+
+		  // a match!
+		  return scorer.score();
+		}
+		catch (IOException e)
+		{
+		  throw new Exception("caught exception in QueryDocVals(" + q + ") doc=" + doc, e);
+		}
+	  }
+
+	  public override bool exists(int doc)
+	  {
+		try
+		{
+		  if (doc < lastDocRequested)
+		  {
+			if (noMatches)
+			{
+				return false;
+			}
+			scorer = weight.scorer(readerContext, acceptDocs);
+			scorerDoc = -1;
+			if (scorer == null)
+			{
+			  noMatches = true;
+			  return false;
+			}
+		  }
+		  lastDocRequested = doc;
+
+		  if (scorerDoc < doc)
+		  {
+			scorerDoc = scorer.advance(doc);
+		  }
+
+		  if (scorerDoc > doc)
+		  {
+			// query doesn't match this document... either because we hit the
+			// end, or because the next doc is after this doc.
+			return false;
+		  }
+
+		  // a match!
+		  return true;
+		}
+		catch (IOException e)
+		{
+		  throw new Exception("caught exception in QueryDocVals(" + q + ") doc=" + doc, e);
+		}
+	  }
+
+	   public override object objectVal(int doc)
+	   {
+		 try
+		 {
+		   return exists(doc) ? scorer.score() : null;
+		 }
+		 catch (IOException e)
+		 {
+		   throw new Exception("caught exception in QueryDocVals(" + q + ") doc=" + doc, e);
+		 }
+	   }
+
+	  public override ValueFiller ValueFiller
+	  {
+		  get
+		  {
+			//
+			// TODO: if we want to support more than one value-filler or a value-filler in conjunction with
+			// the FunctionValues, then members like "scorer" should be per ValueFiller instance.
+			// Or we can say that the user should just instantiate multiple FunctionValues.
+			//
+			return new ValueFillerAnonymousInnerClassHelper(this);
+		  }
+	  }
+
+	  private class ValueFillerAnonymousInnerClassHelper : ValueFiller
+	  {
+		  private readonly QueryDocValues outerInstance;
+
+		  public ValueFillerAnonymousInnerClassHelper(QueryDocValues outerInstance)
+		  {
+			  this.outerInstance = outerInstance;
+			  mval = new MutableValueFloat();
+		  }
+
+		  private readonly MutableValueFloat mval;
+
+		  public override MutableValue Value
+		  {
+			  get
+			  {
+				return mval;
+			  }
+		  }
+
+		  public override void fillValue(int doc)
+		  {
+			try
+			{
+			  if (outerInstance.noMatches)
+			  {
+				mval.value = outerInstance.defVal;
+				mval.exists = false;
+				return;
+			  }
+			  outerInstance.scorer = outerInstance.weight.scorer(outerInstance.readerContext, outerInstance.acceptDocs);
+			  outerInstance.scorerDoc = -1;
+			  if (outerInstance.scorer == null)
+			  {
+				outerInstance.noMatches = true;
+				mval.value = outerInstance.defVal;
+				mval.exists = false;
+				return;
+			  }
+			  outerInstance.lastDocRequested = doc;
+
+			  if (outerInstance.scorerDoc < doc)
+			  {
+				outerInstance.scorerDoc = outerInstance.scorer.advance(doc);
+			  }
+
+			  if (outerInstance.scorerDoc > doc)
+			  {
+				// query doesn't match this document... either because we hit the
+				// end, or because the next doc is after this doc.
+				mval.value = outerInstance.defVal;
+				mval.exists = false;
+				return;
+			  }
+
+			  // a match!
+			  mval.value = outerInstance.scorer.score();
+			  mval.exists = true;
+			}
+			catch (IOException e)
+			{
+			  throw new Exception("caught exception in QueryDocVals(" + outerInstance.q + ") doc=" + doc, e);
+			}
+		  }
+	  }
+
+	  public override string ToString(int doc)
+	  {
+		return "query(" + q + ",def=" + defVal + ")=" + floatVal(doc);
+	  }
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/RangeMapFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/RangeMapFloatFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/RangeMapFloatFunction.cs
new file mode 100644
index 0000000..b5993a7
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/RangeMapFloatFunction.cs
@@ -0,0 +1,133 @@
+/*
+ * 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;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// <code>RangeMapFloatFunction</code> implements a map function over
+	/// another <seealso cref="ValueSource"/> whose values fall within min and max inclusive to target.
+	/// <br>
+	/// Normally Used as an argument to a <seealso cref="FunctionQuery"/>
+	/// 
+	/// 
+	/// </summary>
+	public class RangeMapFloatFunction : ValueSource
+	{
+	  protected internal readonly ValueSource source;
+	  protected internal readonly float min;
+	  protected internal readonly float max;
+	  protected internal readonly ValueSource target;
+	  protected internal readonly ValueSource defaultVal;
+
+	  public RangeMapFloatFunction(ValueSource source, float min, float max, float target, float? def) : this(source, min, max, (ValueSource) new ConstValueSource(target), (ValueSource) (def == null ? null : new ConstValueSource(def.Value)))
+	  {
+	  }
+
+	  public RangeMapFloatFunction(ValueSource source, float min, float max, ValueSource target, ValueSource def)
+	  {
+		this.source = source;
+		this.min = min;
+		this.max = max;
+		this.target = target;
+		this.defaultVal = def;
+	  }
+
+	  public override string description()
+	  {
+		return "map(" + source.description() + "," + min + "," + max + "," + target.description() + ")";
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.queries.function.FunctionValues vals = source.getValues(context, readerContext);
+		FunctionValues vals = source.getValues(context, readerContext);
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.queries.function.FunctionValues targets = target.getValues(context, readerContext);
+		FunctionValues targets = target.getValues(context, readerContext);
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.queries.function.FunctionValues defaults = (this.defaultVal == null) ? null : defaultVal.getValues(context, readerContext);
+		FunctionValues defaults = (this.defaultVal == null) ? null : defaultVal.getValues(context, readerContext);
+		return new FloatDocValuesAnonymousInnerClassHelper(this, this, vals, targets, defaults);
+	  }
+
+	  private class FloatDocValuesAnonymousInnerClassHelper : FloatDocValues
+	  {
+		  private readonly RangeMapFloatFunction outerInstance;
+
+		  private FunctionValues vals;
+		  private FunctionValues targets;
+		  private FunctionValues defaults;
+
+		  public FloatDocValuesAnonymousInnerClassHelper(RangeMapFloatFunction outerInstance, RangeMapFloatFunction this, FunctionValues vals, FunctionValues targets, FunctionValues defaults) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.vals = vals;
+			  this.targets = targets;
+			  this.defaults = defaults;
+		  }
+
+		  public override float floatVal(int doc)
+		  {
+			float val = vals.floatVal(doc);
+			return (val >= outerInstance.min && val <= outerInstance.max) ? targets.floatVal(doc) : (outerInstance.defaultVal == null ? val : defaults.floatVal(doc));
+		  }
+		  public override string ToString(int doc)
+		  {
+			return "map(" + vals.ToString(doc) + ",min=" + outerInstance.min + ",max=" + outerInstance.max + ",target=" + targets.ToString(doc) + ")";
+		  }
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		source.createWeight(context, searcher);
+	  }
+
+	  public override int GetHashCode()
+	  {
+		int h = source.GetHashCode();
+		h ^= (h << 10) | ((int)((uint)h >> 23));
+		h += float.floatToIntBits(min);
+		h ^= (h << 14) | ((int)((uint)h >> 19));
+		h += float.floatToIntBits(max);
+		h += target.GetHashCode();
+		if (defaultVal != null)
+		{
+		  h += defaultVal.GetHashCode();
+		}
+		return h;
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (typeof(RangeMapFloatFunction) != o.GetType())
+		{
+			return false;
+		}
+		RangeMapFloatFunction other = (RangeMapFloatFunction)o;
+		return this.min == other.min && this.max == other.max && this.target.Equals(other.target) && this.source.Equals(other.source) && (this.defaultVal == other.defaultVal || (this.defaultVal != null && this.defaultVal.Equals(other.defaultVal)));
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/ReciprocalFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/ReciprocalFloatFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/ReciprocalFloatFunction.cs
new file mode 100644
index 0000000..bcd0ebf
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/ReciprocalFloatFunction.cs
@@ -0,0 +1,125 @@
+/*
+ * 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;
+using System.Collections;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// <code>ReciprocalFloatFunction</code> implements a reciprocal function f(x) = a/(mx+b), based on
+	/// the float value of a field or function as exported by <seealso cref="ValueSource"/>.
+	/// <br>
+	/// 
+	/// When a and b are equal, and x>=0, this function has a maximum value of 1 that drops as x increases.
+	/// Increasing the value of a and b together results in a movement of the entire function to a flatter part of the curve.
+	/// <para>These properties make this an idea function for boosting more recent documents.
+	/// </para>
+	/// <para>Example:<code>  recip(ms(NOW,mydatefield),3.16e-11,1,1)</code>
+	/// </para>
+	/// <para>A multiplier of 3.16e-11 changes the units from milliseconds to years (since there are about 3.16e10 milliseconds
+	/// per year).  Thus, a very recent date will yield a value close to 1/(0+1) or 1,
+	/// a date a year in the past will get a multiplier of about 1/(1+1) or 1/2,
+	/// and date two years old will yield 1/(2+1) or 1/3.
+	/// 
+	/// </para>
+	/// </summary>
+	/// <seealso cref= org.apache.lucene.queries.function.FunctionQuery
+	/// 
+	///  </seealso>
+	public class ReciprocalFloatFunction : ValueSource
+	{
+	  protected internal readonly ValueSource source;
+	  protected internal readonly float m;
+	  protected internal readonly float a;
+	  protected internal readonly float b;
+
+	  /// <summary>
+	  ///  f(source) = a/(m*float(source)+b)
+	  /// </summary>
+	  public ReciprocalFloatFunction(ValueSource source, float m, float a, float b)
+	  {
+		this.source = source;
+		this.m = m;
+		this.a = a;
+		this.b = b;
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.queries.function.FunctionValues vals = source.getValues(context, readerContext);
+		FunctionValues vals = source.getValues(context, readerContext);
+		return new FloatDocValuesAnonymousInnerClassHelper(this, this, vals);
+	  }
+
+	  private class FloatDocValuesAnonymousInnerClassHelper : FloatDocValues
+	  {
+		  private readonly ReciprocalFloatFunction outerInstance;
+
+		  private FunctionValues vals;
+
+		  public FloatDocValuesAnonymousInnerClassHelper(ReciprocalFloatFunction outerInstance, ReciprocalFloatFunction this, FunctionValues vals) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.vals = vals;
+		  }
+
+		  public override float floatVal(int doc)
+		  {
+			return outerInstance.a / (outerInstance.m * vals.floatVal(doc) + outerInstance.b);
+		  }
+		  public override string ToString(int doc)
+		  {
+			return Convert.ToString(outerInstance.a) + "/(" + outerInstance.m + "*float(" + vals.ToString(doc) + ')' + '+' + outerInstance.b + ')';
+		  }
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		source.createWeight(context, searcher);
+	  }
+
+	  public override string description()
+	  {
+		return Convert.ToString(a) + "/(" + m + "*float(" + source.description() + ")" + "+" + b + ')';
+	  }
+
+	  public override int GetHashCode()
+	  {
+		int h = float.floatToIntBits(a) + float.floatToIntBits(m);
+		h ^= (h << 13) | ((int)((uint)h >> 20));
+		return h + (float.floatToIntBits(b)) + source.GetHashCode();
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (typeof(ReciprocalFloatFunction) != o.GetType())
+		{
+			return false;
+		}
+		ReciprocalFloatFunction other = (ReciprocalFloatFunction)o;
+		return this.m == other.m && this.a == other.a && this.b == other.b && this.source.Equals(other.source);
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/ReverseOrdFieldSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/ReverseOrdFieldSource.cs b/src/Lucene.Net.Queries/Function/ValueSources/ReverseOrdFieldSource.cs
new file mode 100644
index 0000000..6628ea0
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/ReverseOrdFieldSource.cs
@@ -0,0 +1,124 @@
+/*
+ * 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;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// Obtains the ordinal of the field value from the default Lucene <seealso cref="org.apache.lucene.search.FieldCache"/> using getTermsIndex()
+	/// and reverses the order.
+	/// <br>
+	/// The native lucene index order is used to assign an ordinal value for each field value.
+	/// <br>Field values (terms) are lexicographically ordered by unicode value, and numbered starting at 1.
+	/// <br>
+	/// Example of reverse ordinal (rord):<br>
+	///  If there were only three field values: "apple","banana","pear"
+	/// <br>then rord("apple")=3, rord("banana")=2, ord("pear")=1
+	/// <para>
+	///  WARNING: ord() depends on the position in an index and can thus change when other documents are inserted or deleted,
+	///  or if a MultiSearcher is used.
+	/// <br>
+	///  WARNING: as of Solr 1.4, ord() and rord() can cause excess memory use since they must use a FieldCache entry
+	/// at the top level reader, while sorting and function queries now use entries at the segment level.  Hence sorting
+	/// or using a different function query, in addition to ord()/rord() will double memory use.
+	/// 
+	/// 
+	/// </para>
+	/// </summary>
+
+	public class ReverseOrdFieldSource : ValueSource
+	{
+	  public readonly string field;
+
+	  public ReverseOrdFieldSource(string field)
+	  {
+		this.field = field;
+	  }
+
+	  public override string description()
+	  {
+		return "rord(" + field + ')';
+	  }
+
+	  // TODO: this is trappy? perhaps this query instead should make you pass a slow reader yourself?
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.index.IndexReader topReader = org.apache.lucene.index.ReaderUtil.getTopLevelContext(readerContext).reader();
+		IndexReader topReader = ReaderUtil.getTopLevelContext(readerContext).reader();
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.index.AtomicReader r = org.apache.lucene.index.SlowCompositeReaderWrapper.wrap(topReader);
+		AtomicReader r = SlowCompositeReaderWrapper.wrap(topReader);
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final int off = readerContext.docBase;
+		int off = readerContext.docBase;
+
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.index.SortedDocValues sindex = org.apache.lucene.search.FieldCache.DEFAULT.getTermsIndex(r, field);
+		SortedDocValues sindex = FieldCache.DEFAULT.getTermsIndex(r, field);
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final int end = sindex.getValueCount();
+		int end = sindex.ValueCount;
+
+		return new IntDocValuesAnonymousInnerClassHelper(this, this, off, sindex, end);
+	  }
+
+	  private class IntDocValuesAnonymousInnerClassHelper : IntDocValues
+	  {
+		  private readonly ReverseOrdFieldSource outerInstance;
+
+		  private int off;
+		  private SortedDocValues sindex;
+		  private int end;
+
+		  public IntDocValuesAnonymousInnerClassHelper(ReverseOrdFieldSource outerInstance, ReverseOrdFieldSource this, int off, SortedDocValues sindex, int end) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.off = off;
+			  this.sindex = sindex;
+			  this.end = end;
+		  }
+
+		  public override int intVal(int doc)
+		  {
+			 return (end - sindex.getOrd(doc + off) - 1);
+		  }
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (o == null || (o.GetType() != typeof(ReverseOrdFieldSource)))
+		{
+			return false;
+		}
+		ReverseOrdFieldSource other = (ReverseOrdFieldSource)o;
+		return this.field.Equals(other.field);
+	  }
+
+	  private static readonly int hcode = typeof(ReverseOrdFieldSource).GetHashCode();
+	  public override int GetHashCode()
+	  {
+		return hcode + field.GetHashCode();
+	  }
+
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/ScaleFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/ScaleFloatFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/ScaleFloatFunction.cs
new file mode 100644
index 0000000..2b2f15c
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/ScaleFloatFunction.cs
@@ -0,0 +1,195 @@
+/*
+ * 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;
+using System.Collections.Generic;
+using Lucene.Net.Support;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// Scales values to be between min and max.
+	/// <para>This implementation currently traverses all of the source values to obtain
+	/// their min and max.
+	/// </para>
+	/// <para>This implementation currently cannot distinguish when documents have been
+	/// deleted or documents that have no value, and 0.0 values will be used for
+	/// these cases.  This means that if values are normally all greater than 0.0, one can
+	/// still end up with 0.0 as the min value to map from.  In these cases, an
+	/// appropriate map() function could be used as a workaround to change 0.0
+	/// to a value in the real range.
+	/// </para>
+	/// </summary>
+	public class ScaleFloatFunction : ValueSource
+	{
+	  protected internal readonly ValueSource source;
+	  protected internal readonly float min;
+	  protected internal readonly float max;
+
+	  public ScaleFloatFunction(ValueSource source, float min, float max)
+	  {
+		this.source = source;
+		this.min = min;
+		this.max = max;
+	  }
+
+	  public override string description()
+	  {
+		return "scale(" + source.description() + "," + min + "," + max + ")";
+	  }
+
+	  private class ScaleInfo
+	  {
+		internal float minVal;
+		internal float maxVal;
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: private ScaleInfo createScaleInfo(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  private ScaleInfo createScaleInfo(IDictionary context, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final java.util.List<org.apache.lucene.index.AtomicReaderContext> leaves = org.apache.lucene.index.ReaderUtil.getTopLevelContext(readerContext).leaves();
+		IList<AtomicReaderContext> leaves = ReaderUtil.getTopLevelContext(readerContext).leaves();
+
+		float minVal = float.PositiveInfinity;
+		float maxVal = float.NegativeInfinity;
+
+		foreach (AtomicReaderContext leaf in leaves)
+		{
+		  int maxDoc = leaf.reader().maxDoc();
+		  FunctionValues vals = source.getValues(context, leaf);
+		  for (int i = 0; i < maxDoc; i++)
+		  {
+
+		  float val = vals.floatVal(i);
+		  if ((float.floatToRawIntBits(val) & (0xff << 23)) == 0xff << 23)
+		  {
+			// if the exponent in the float is all ones, then this is +Inf, -Inf or NaN
+			// which don't make sense to factor into the scale function
+			continue;
+		  }
+		  if (val < minVal)
+		  {
+			minVal = val;
+		  }
+		  if (val > maxVal)
+		  {
+			maxVal = val;
+		  }
+		  }
+		}
+
+		if (minVal == float.PositiveInfinity)
+		{
+		// must have been an empty index
+		  minVal = maxVal = 0;
+		}
+
+		ScaleInfo scaleInfo = new ScaleInfo();
+		scaleInfo.minVal = minVal;
+		scaleInfo.maxVal = maxVal;
+		context[ScaleFloatFunction.this] = scaleInfo;
+		return scaleInfo;
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+
+		ScaleInfo scaleInfo = (ScaleInfo)context[ScaleFloatFunction.this];
+		if (scaleInfo == null)
+		{
+		  scaleInfo = createScaleInfo(context, readerContext);
+		}
+
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final float scale = (scaleInfo.maxVal-scaleInfo.minVal==0) ? 0 : (max-min)/(scaleInfo.maxVal-scaleInfo.minVal);
+		float scale = (scaleInfo.maxVal - scaleInfo.minVal == 0) ? 0 : (max - min) / (scaleInfo.maxVal - scaleInfo.minVal);
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final float minSource = scaleInfo.minVal;
+		float minSource = scaleInfo.minVal;
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final float maxSource = scaleInfo.maxVal;
+		float maxSource = scaleInfo.maxVal;
+
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.queries.function.FunctionValues vals = source.getValues(context, readerContext);
+		FunctionValues vals = source.getValues(context, readerContext);
+
+		return new FloatDocValuesAnonymousInnerClassHelper(this, this, scale, minSource, maxSource, vals);
+	  }
+
+	  private class FloatDocValuesAnonymousInnerClassHelper : FloatDocValues
+	  {
+		  private readonly ScaleFloatFunction outerInstance;
+
+		  private float scale;
+		  private float minSource;
+		  private float maxSource;
+		  private FunctionValues vals;
+
+		  public FloatDocValuesAnonymousInnerClassHelper(ScaleFloatFunction outerInstance, ScaleFloatFunction this, float scale, float minSource, float maxSource, FunctionValues vals) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.scale = scale;
+			  this.minSource = minSource;
+			  this.maxSource = maxSource;
+			  this.vals = vals;
+		  }
+
+		  public override float floatVal(int doc)
+		  {
+			return (vals.floatVal(doc) - minSource) * scale + outerInstance.min;
+		  }
+		  public override string ToString(int doc)
+		  {
+			return "scale(" + vals.ToString(doc) + ",toMin=" + outerInstance.min + ",toMax=" + outerInstance.max + ",fromMin=" + minSource + ",fromMax=" + maxSource + ")";
+		  }
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		source.createWeight(context, searcher);
+	  }
+
+	  public override int GetHashCode()
+	  {
+		int h = Number.FloatToIntBits(min);
+		h = h * 29;
+		h += Number.FloatToIntBits(max);
+		h = h * 29;
+		h += source.GetHashCode();
+		return h;
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (typeof(ScaleFloatFunction) != o.GetType())
+		{
+			return false;
+		}
+		ScaleFloatFunction other = (ScaleFloatFunction)o;
+		return this.min == other.min && this.max == other.max && this.source.Equals(other.source);
+	  }
+	}
+
+}s
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/ShortFieldSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/ShortFieldSource.cs b/src/Lucene.Net.Queries/Function/ValueSources/ShortFieldSource.cs
new file mode 100644
index 0000000..cbb6215
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/ShortFieldSource.cs
@@ -0,0 +1,132 @@
+using System;
+using System.Collections;
+using org.apache.lucene.queries.function;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+	/*
+	 * 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.
+	 */
+    /// <summary>
+	/// Obtains short field values from the <seealso cref="org.apache.lucene.search.FieldCache"/>
+	/// using <code>getShorts()</code>
+	/// and makes those values available as other numeric types, casting as needed.
+	/// 
+	/// </summary>
+	[Obsolete]
+	public class ShortFieldSource : FieldCacheSource
+	{
+
+	  internal readonly FieldCache.ShortParser parser;
+
+	  public ShortFieldSource(string field) : this(field, null)
+	  {
+	  }
+
+	  public ShortFieldSource(string field, FieldCache.ShortParser parser) : base(field)
+	  {
+		this.parser = parser;
+	  }
+
+	  public override string description()
+	  {
+		return "short(" + field + ')';
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.search.FieldCache.Shorts arr = cache.getShorts(readerContext.reader(), field, parser, false);
+		FieldCache.Shorts arr = cache.getShorts(readerContext.reader(), field, parser, false);
+
+		return new FunctionValuesAnonymousInnerClassHelper(this, arr);
+	  }
+
+	  private class FunctionValuesAnonymousInnerClassHelper : FunctionValues
+	  {
+		  private readonly ShortFieldSource outerInstance;
+
+		  private FieldCache.Shorts arr;
+
+		  public FunctionValuesAnonymousInnerClassHelper(ShortFieldSource outerInstance, FieldCache.Shorts arr)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.arr = arr;
+		  }
+
+		  public override sbyte byteVal(int doc)
+		  {
+			return (sbyte) arr.get(doc);
+		  }
+
+		  public override short shortVal(int doc)
+		  {
+			return arr.get(doc);
+		  }
+
+		  public override float floatVal(int doc)
+		  {
+			return (float) arr.get(doc);
+		  }
+
+		  public override int intVal(int doc)
+		  {
+			return (int) arr.get(doc);
+		  }
+
+		  public override long longVal(int doc)
+		  {
+			return (long) arr.get(doc);
+		  }
+
+		  public override double doubleVal(int doc)
+		  {
+			return (double) arr.get(doc);
+		  }
+
+		  public override string strVal(int doc)
+		  {
+			return Convert.ToString(arr.get(doc));
+		  }
+
+		  public override string ToString(int doc)
+		  {
+			return outerInstance.description() + '=' + shortVal(doc);
+		  }
+
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (o.GetType() != typeof(ShortFieldSource))
+		{
+			return false;
+		}
+		ShortFieldSource other = (ShortFieldSource) o;
+		return base.Equals(other) && (this.parser == null ? other.parser == null : this.parser.GetType() == other.parser.GetType());
+	  }
+
+	  public override int GetHashCode()
+	  {
+		int h = parser == null ? typeof(short?).GetHashCode() : parser.GetType().GetHashCode();
+		h += base.GetHashCode();
+		return h;
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/SimpleBoolFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/SimpleBoolFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/SimpleBoolFunction.cs
new file mode 100644
index 0000000..e80fd80
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/SimpleBoolFunction.cs
@@ -0,0 +1,103 @@
+/*
+ * 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;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// <seealso cref="BoolFunction"/> implementation which applies an extendible boolean
+	/// function to the values of a single wrapped <seealso cref="ValueSource"/>.
+	/// 
+	/// Functions this can be used for include whether a field has a value or not,
+	/// or inverting the boolean value of the wrapped ValueSource.
+	/// </summary>
+	public abstract class SimpleBoolFunction : BoolFunction
+	{
+	  protected internal readonly ValueSource source;
+
+	  public SimpleBoolFunction(ValueSource source)
+	  {
+		this.source = source;
+	  }
+
+	  protected internal abstract string name();
+
+	  protected internal abstract bool func(int doc, FunctionValues vals);
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.docvalues.BoolDocValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override BoolDocValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final org.apache.lucene.queries.function.FunctionValues vals = source.getValues(context, readerContext);
+		FunctionValues vals = source.getValues(context, readerContext);
+		return new BoolDocValuesAnonymousInnerClassHelper(this, this, vals);
+	  }
+
+	  private class BoolDocValuesAnonymousInnerClassHelper : BoolDocValues
+	  {
+		  private readonly SimpleBoolFunction outerInstance;
+
+		  private FunctionValues vals;
+
+		  public BoolDocValuesAnonymousInnerClassHelper(SimpleBoolFunction outerInstance, SimpleBoolFunction this, FunctionValues vals) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.vals = vals;
+		  }
+
+		  public override bool boolVal(int doc)
+		  {
+			return outerInstance.func(doc, vals);
+		  }
+		  public override string ToString(int doc)
+		  {
+			return outerInstance.name() + '(' + vals.ToString(doc) + ')';
+		  }
+	  }
+
+	  public override string description()
+	  {
+		return name() + '(' + source.description() + ')';
+	  }
+
+	  public override int GetHashCode()
+	  {
+		return source.GetHashCode() + name().GetHashCode();
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (this.GetType() != o.GetType())
+		{
+			return false;
+		}
+		SimpleBoolFunction other = (SimpleBoolFunction)o;
+		return this.source.Equals(other.source);
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		source.createWeight(context, searcher);
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/SimpleFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/SimpleFloatFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/SimpleFloatFunction.cs
new file mode 100644
index 0000000..73739df
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/SimpleFloatFunction.cs
@@ -0,0 +1,64 @@
+using System.Collections;
+/*
+ * 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 Lucene.Net.Index;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// A simple float function with a single argument
+	/// </summary>
+	 public abstract class SimpleFloatFunction : SingleFunction
+	 {
+	  public SimpleFloatFunction(org.apache.lucene.queries.function.ValueSource source) : base(source)
+	  {
+	  }
+
+	  protected internal abstract float func(int doc, FunctionValues vals);
+
+	  public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+		FunctionValues vals = source.getValues(context, readerContext);
+		return new FloatDocValuesAnonymousInnerClassHelper(this, this, vals);
+	  }
+
+	  private class FloatDocValuesAnonymousInnerClassHelper : FloatDocValues
+	  {
+		  private readonly SimpleFloatFunction outerInstance;
+
+		  private FunctionValues vals;
+
+		  public FloatDocValuesAnonymousInnerClassHelper(SimpleFloatFunction outerInstance, SimpleFloatFunction this, FunctionValues vals) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.vals = vals;
+		  }
+
+		  public override float floatVal(int doc)
+		  {
+			return outerInstance.func(doc, vals);
+		  }
+		  public override string ToString(int doc)
+		  {
+			return outerInstance.name() + '(' + vals.ToString(doc) + ')';
+		  }
+	  }
+	 }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/SingleFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/SingleFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/SingleFunction.cs
new file mode 100644
index 0000000..b4102fd
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/SingleFunction.cs
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// A function with a single argument
+	/// </summary>
+	 public abstract class SingleFunction : ValueSource
+	 {
+	  protected internal readonly ValueSource source;
+
+	  public SingleFunction(ValueSource source)
+	  {
+		this.source = source;
+	  }
+
+	  protected internal abstract string name();
+
+	  public override string description()
+	  {
+		return name() + '(' + source.description() + ')';
+	  }
+
+	  public override int GetHashCode()
+	  {
+		return source.GetHashCode() + name().GetHashCode();
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (this.GetType() != o.GetType())
+		{
+			return false;
+		}
+		SingleFunction other = (SingleFunction)o;
+		return this.name().Equals(other.name()) && this.source.Equals(other.source);
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		source.createWeight(context, searcher);
+	  }
+	 }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/SumFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/SumFloatFunction.cs b/src/Lucene.Net.Queries/Function/ValueSources/SumFloatFunction.cs
new file mode 100644
index 0000000..bcfd8a8
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/SumFloatFunction.cs
@@ -0,0 +1,48 @@
+/*
+ * 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 org.apache.lucene.queries.function;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+
+
+	/// <summary>
+	/// <code>SumFloatFunction</code> returns the sum of it's components.
+	/// </summary>
+	public class SumFloatFunction : MultiFloatFunction
+	{
+	  public SumFloatFunction(ValueSource[] sources) : base(sources)
+	  {
+	  }
+
+	  protected internal override string name()
+	  {
+		return "sum";
+	  }
+
+	  protected internal override float func(int doc, FunctionValues[] valsArr)
+	  {
+		float val = 0.0f;
+		foreach (FunctionValues vals in valsArr)
+		{
+		  val += vals.floatVal(doc);
+		}
+		return val;
+	  }
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/2b55e53c/src/Lucene.Net.Queries/Function/ValueSources/SumTotalTermFreqValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Queries/Function/ValueSources/SumTotalTermFreqValueSource.cs b/src/Lucene.Net.Queries/Function/ValueSources/SumTotalTermFreqValueSource.cs
new file mode 100644
index 0000000..1fc9aec
--- /dev/null
+++ b/src/Lucene.Net.Queries/Function/ValueSources/SumTotalTermFreqValueSource.cs
@@ -0,0 +1,124 @@
+/*
+ * 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;
+using org.apache.lucene.queries.function;
+using org.apache.lucene.queries.function.docvalues;
+
+namespace Lucene.Net.Queries.Function.ValueSources
+{
+    /// <summary>
+	/// <code>SumTotalTermFreqValueSource</code> returns the number of tokens.
+	/// (sum of term freqs across all documents, across all terms).
+	/// Returns -1 if frequencies were omitted for the field, or if 
+	/// the codec doesn't support this statistic.
+	/// @lucene.internal
+	/// </summary>
+	public class SumTotalTermFreqValueSource : ValueSource
+	{
+	  protected internal readonly string indexedField;
+
+	  public SumTotalTermFreqValueSource(string indexedField)
+	  {
+		this.indexedField = indexedField;
+	  }
+
+	  public virtual string name()
+	  {
+		return "sumtotaltermfreq";
+	  }
+
+	  public override string description()
+	  {
+		return name() + '(' + indexedField + ')';
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public org.apache.lucene.queries.function.FunctionValues getValues(java.util.Map context, org.apache.lucene.index.AtomicReaderContext readerContext) throws java.io.IOException
+	  public override FunctionValues getValues(IDictionary context, AtomicReaderContext readerContext)
+	  {
+		return (FunctionValues)context[this];
+	  }
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: @Override public void createWeight(java.util.Map context, org.apache.lucene.search.IndexSearcher searcher) throws java.io.IOException
+	  public override void createWeight(IDictionary context, IndexSearcher searcher)
+	  {
+		long sumTotalTermFreq = 0;
+		foreach (AtomicReaderContext readerContext in searcher.TopReaderContext.leaves())
+		{
+		  Fields fields = readerContext.reader().fields();
+		  if (fields == null)
+		  {
+			  continue;
+		  }
+		  Terms terms = fields.terms(indexedField);
+		  if (terms == null)
+		  {
+			  continue;
+		  }
+		  long v = terms.SumTotalTermFreq;
+		  if (v == -1)
+		  {
+			sumTotalTermFreq = -1;
+			break;
+		  }
+		  else
+		  {
+			sumTotalTermFreq += v;
+		  }
+		}
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final long ttf = sumTotalTermFreq;
+		long ttf = sumTotalTermFreq;
+		context[this] = new LongDocValuesAnonymousInnerClassHelper(this, this, ttf);
+	  }
+
+	  private class LongDocValuesAnonymousInnerClassHelper : LongDocValues
+	  {
+		  private readonly SumTotalTermFreqValueSource outerInstance;
+
+		  private long ttf;
+
+		  public LongDocValuesAnonymousInnerClassHelper(SumTotalTermFreqValueSource outerInstance, SumTotalTermFreqValueSource this, long ttf) : base(this)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.ttf = ttf;
+		  }
+
+		  public override long longVal(int doc)
+		  {
+			return ttf;
+		  }
+	  }
+
+	  public override int GetHashCode()
+	  {
+		return this.GetType().GetHashCode() + indexedField.GetHashCode();
+	  }
+
+	  public override bool Equals(object o)
+	  {
+		if (this.GetType() != o.GetType())
+		{
+			return false;
+		}
+		SumTotalTermFreqValueSource other = (SumTotalTermFreqValueSource)o;
+		return this.indexedField.Equals(other.indexedField);
+	  }
+	}
+
+}
\ No newline at end of file