You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/01/08 13:00:31 UTC

svn commit: r1556508 - in /lucene/dev/branches/lucene5376/lucene: expressions/src/java/org/apache/lucene/expressions/ server/src/java/org/apache/lucene/server/ server/src/java/org/apache/lucene/server/handlers/ server/src/test/org/apache/lucene/server/...

Author: mikemccand
Date: Wed Jan  8 12:00:31 2014
New Revision: 1556508

URL: http://svn.apache.org/r1556508
Log:
LUCENE-5207, LUCENE-5376: add expressions support to lucene server, so you can define a virtual field from any JS expression and then sort by that field or retrieve its values for all hits

Added:
    lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDefBindings.java   (with props)
    lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CannedScorer.java   (with props)
    lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java   (with props)
Modified:
    lucene/dev/branches/lucene5376/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
    lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDef.java
    lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/IndexState.java
    lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/Server.java
    lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java
    lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/RegisterFieldHandler.java
    lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java
    lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestIndexing.java
    lucene/dev/branches/lucene5376/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java

Modified: lucene/dev/branches/lucene5376/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java (original)
+++ lucene/dev/branches/lucene5376/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java Wed Jan  8 12:00:31 2014
@@ -30,7 +30,8 @@ import java.util.Map;
  * the context map by {@link ExpressionComparator}.
  */
 @SuppressWarnings({"rawtypes"})
-class ScoreValueSource extends ValueSource {
+// nocomit i made this public...
+public class ScoreValueSource extends ValueSource {
 
   /**
    * <code>context</code> must contain a key "scorer" which is a {@link Scorer}.

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDef.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDef.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDef.java Wed Jan  8 12:00:31 2014
@@ -19,12 +19,14 @@ package org.apache.lucene.server;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.similarities.Similarity;
 
 /** Defines the type of one field. */
 public class FieldDef {
   /** Field name. */
   public final String name;
+
   /** {@link FieldType}, used during indexing. */
   public final FieldType fieldType;
 
@@ -73,10 +75,13 @@ public class FieldDef {
   /** Maximum age for recency boosting to have an effect (seconds). */
   public final long blendRange;
 
+  /** Only set for a virtual field (expression). */
+  public final ValueSource valueSource;
+
   /** Sole constructor. */
   public FieldDef(String name, FieldType fieldType, String valueType, String faceted, String postingsFormat, String docValuesFormat, boolean multiValued,
                   Similarity sim, Analyzer indexAnalyzer, Analyzer searchAnalyzer, boolean highlighted, String liveValuesIDField,
-                  String blendFieldName, float blendMaxBoost, long blendRange) {
+                  String blendFieldName, float blendMaxBoost, long blendRange, ValueSource valueSource) {
     this.name = name;
     this.fieldType = fieldType;
     if (fieldType != null) {
@@ -104,5 +109,6 @@ public class FieldDef {
     this.blendFieldName = blendFieldName;
     this.blendMaxBoost = blendMaxBoost;
     this.blendRange = blendRange;
+    this.valueSource = valueSource;
   }
 }

Added: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDefBindings.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDefBindings.java?rev=1556508&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDefBindings.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/FieldDefBindings.java Wed Jan  8 12:00:31 2014
@@ -0,0 +1,70 @@
+package org.apache.lucene.server;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Map;
+
+import org.apache.lucene.expressions.Bindings;
+import org.apache.lucene.expressions.ScoreValueSource;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
+import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
+import org.apache.lucene.queries.function.valuesource.IntFieldSource;
+import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+
+/** Implements {@link Bindings} on top of the registered
+ *  fields. */
+public final class FieldDefBindings extends Bindings {
+
+  private final Map<String,FieldDef> fields;
+
+  /** Sole constructor. */
+  public FieldDefBindings(Map<String,FieldDef> fields) {
+    this.fields = fields;
+  }
+
+  @Override
+  public ValueSource getValueSource(String name) {
+    if (name.equals("_score")) {
+      return new ScoreValueSource();
+    }
+    FieldDef fd = fields.get(name);
+    if (fd == null) {
+      throw new IllegalArgumentException("Invalid reference '" + name + "'");
+    }
+    if (fd.valueType.equals("virtual")) {
+      return fd.valueSource;
+    } else if (fd.fieldType != null && fd.fieldType.docValueType() == DocValuesType.NUMERIC) {
+      if (fd.valueType.equals("int")) {
+        return new IntFieldSource(name);
+      } else if (fd.valueType.equals("float")) {
+        return new FloatFieldSource(name);
+      } else if (fd.valueType.equals("long")) {
+        return new LongFieldSource(name);
+      } else if (fd.valueType.equals("double")) {
+        return new DoubleFieldSource(name);
+      } else {
+        assert false: "unknown numeric field type: " + fd.valueType;
+        return null;
+      }
+    } else {
+      throw new IllegalArgumentException("Field \'" + name + "\' cannot be used in an expression: it was not registered with sort=true");
+    }
+  }
+}

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/IndexState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/IndexState.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/IndexState.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/IndexState.java Wed Jan  8 12:00:31 2014
@@ -51,6 +51,7 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.lucene46.Lucene46Codec;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.expressions.Bindings;
 import org.apache.lucene.facet.CachedOrdinalsReader;
 import org.apache.lucene.facet.DocValuesOrdinalsReader;
 import org.apache.lucene.facet.FacetsConfig;
@@ -197,6 +198,12 @@ public class IndexState implements Close
 
   public final FacetsConfig facetsConfig = new FacetsConfig();
 
+  // nocommit need to handle dynamic exprs too; new Bindings
+  // wrapping this one:
+
+  /** {@link Bindings} to pass when evaluating expressions. */
+  public final Bindings exprBindings = new FieldDefBindings(fields);
+
   /** Tracks snapshot references to generations. */
   private static class SaveLoadRefCounts extends GenFileUtil<Map<Long,Integer>> {
 

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/Server.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/Server.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/Server.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/Server.java Wed Jan  8 12:00:31 2014
@@ -433,6 +433,11 @@ public class Server {
       if (e.getCause() instanceof RequestFailedException) {
         RequestFailedException rfe = (RequestFailedException) e.getCause();
         pw.write(rfe.path + ": " + rfe.reason);
+        // TODO?
+        //Throwable cause = rfe.getCause();
+        //if (cause != null) {
+        //pw.write("\n\nCaused by:\n\n" + cause);
+        //}
       } else {
         e.getCause().printStackTrace(pw);
       }

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java Wed Jan  8 12:00:31 2014
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.io.StringReader;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -34,11 +33,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.document.FieldType.NumericType;
 import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager.SearcherAndTaxonomy;
-import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.suggest.DocumentDictionary;
 import org.apache.lucene.search.suggest.DocumentExpressionDictionary;
 import org.apache.lucene.search.suggest.InputIterator;
@@ -47,7 +42,6 @@ import org.apache.lucene.search.suggest.
 import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester;
 import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester;
 import org.apache.lucene.search.suggest.analyzing.FuzzySuggester;
-import org.apache.lucene.server.FieldDef;
 import org.apache.lucene.server.FinishRequest;
 import org.apache.lucene.server.FromFileTermFreqIterator;
 import org.apache.lucene.server.GlobalState;
@@ -430,32 +424,10 @@ public class BuildSuggestHandler extends
       } else {
         // Weight is an expression; add bindings for all
         // numeric DV fields:
-        Set<SortField> sortFields = new HashSet<SortField>();
-        for(FieldDef fd : state.getAllFields().values()) {
-          if (fd.fieldType != null && fd.fieldType.docValueType() == DocValuesType.NUMERIC) {
-            SortField sortField;
-            if (fd.valueType.equals("int")) {
-              sortField = new SortField(fd.name, SortField.Type.INT);
-            } else if (fd.valueType.equals("float")) {
-              sortField = new SortField(fd.name, SortField.Type.FLOAT);
-            } else if (fd.valueType.equals("long")) {
-              sortField = new SortField(fd.name, SortField.Type.LONG);
-            } else if (fd.valueType.equals("double")) {
-              sortField = new SortField(fd.name, SortField.Type.DOUBLE);
-            } else {
-              // Dead code today, but if new enum value is
-              // added this is live:
-              sortField = null;
-              assert false: "missing enum value in switch";
-            }
-            sortFields.add(sortField);
-          }
-        }
-
         dict = new DocumentExpressionDictionary(searcher.searcher.getIndexReader(),
                                                 suggestField,
                                                 source.getString("weightExpression"),
-                                                sortFields,
+                                                state.exprBindings,
                                                 payloadField);
       }
 

Added: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CannedScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CannedScorer.java?rev=1556508&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CannedScorer.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CannedScorer.java Wed Jan  8 12:00:31 2014
@@ -0,0 +1,66 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+// TODO: somehow share w/ the many copies of this class in
+// Lucene ... but I don't want to lend the thing any
+// credibility!!
+final class CannedScorer extends Scorer {
+
+  float score;
+  int doc;
+
+  public CannedScorer(int doc, float score) {
+    super((Weight) null);
+    this.doc = doc;
+    this.score = score;
+  }
+
+  @Override
+  public float score() {
+    return score;
+  }
+    
+  @Override
+  public int freq() {
+    throw new UnsupportedOperationException(); // TODO: wtf does this class do?
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int nextDoc() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long cost() {
+    return 1;
+  }
+}

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/RegisterFieldHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/RegisterFieldHandler.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/RegisterFieldHandler.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/RegisterFieldHandler.java Wed Jan  8 12:00:31 2014
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
 import java.text.Collator;
+import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -70,13 +71,17 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.collation.CollationKeyAnalyzer;
 import org.apache.lucene.document.FieldType.NumericType;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.expressions.Expression;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.similarities.BM25Similarity;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.suggest.analyzing.SuggestStopFilter;
 import org.apache.lucene.server.FieldDef;
+import org.apache.lucene.server.FieldDefBindings;
 import org.apache.lucene.server.FinishRequest;
 import org.apache.lucene.server.GlobalState;
 import org.apache.lucene.server.IndexState;
@@ -87,7 +92,6 @@ import org.apache.lucene.util.NumericUti
 import org.apache.lucene.util.Version;
 import net.minidev.json.JSONObject;
 import net.minidev.json.JSONValue;
-import net.minidev.json.parser.ParseException;
 import com.ibm.icu.lang.UCharacter;
 import com.ibm.icu.lang.UProperty;
 import com.ibm.icu.lang.UScript;
@@ -304,7 +308,9 @@ public class RegisterFieldHandler extend
                                "long", "Long value.",
                                // TODO: this is hacked up now ... only supports fixed "recency" blending ... ideally we would accept
                                // a custom equation and parse & execute that:
+                               // nocommit name this "dynamic" instead of "virtual"?
                                "virtual", "Virtual (computed at search time) field, e.g. for blended sorting.")),
+        // nocommit rename to "search"?  ie, "I will search on/by this field's values"
         new Param("index", "True if the value should be indexed.", new BooleanType(), false),
         new Param("tokenize", "True if the value should be tokenized.", new BooleanType(), true),
         new Param("store", "True if the value should be stored.", new BooleanType(), false),
@@ -336,6 +342,7 @@ public class RegisterFieldHandler extend
                                "docsFreqsPositions", "Index doc ids, term frequences and positions.",
                                "docsFreqsPositionsOffsets", "Index doc ids, term frequencies, positions and offsets."),
                   "docsFreqsPositions"),
+        new Param("expression", "The JavaScript expression defining a virtual field's value (only used with type=virtual).", new StringType()),
         new Param("recencyScoreBlend", "Only used with type=virtual, to describe how the virtual field blends with score.",
                   new StructType(
                                  new Param("timeStampField", "Field holding timestamp value (must be type long, with sort=true)", new StringType()),
@@ -380,23 +387,63 @@ public class RegisterFieldHandler extend
   }
 
   private FieldDef parseOneVirtualFieldType(Request r, IndexState state, Map<String,FieldDef> pendingFieldDefs, String name, JSONObject o) {
-    Request r2 = r.getStruct("recencyScoreBlend");
-    String timeStampField = r2.getString("timeStampField");
-    FieldDef fd;
-    try {
-      fd = state.getField(timeStampField);
-    } catch (IllegalArgumentException iae) {
-      fd = pendingFieldDefs.get(timeStampField);
-      if (fd == null) {
-        r2.fail("timeStampField", "field \"" + timeStampField + "\" was not yet registered");
-      }
-    }
-    if (fd.fieldType.docValueType() != DocValuesType.NUMERIC) {
-      r2.fail("timeStampField", "field \"" + fd.name + "\" must be registered with type=long and sort=true");
-    }
-    float maxBoost = r2.getFloat("maxBoost");
-    long range = r2.getLong("range");
-    return new FieldDef(name, null, "virtual", null, null, null, true, null, null, null, false, null, fd.name, maxBoost, range);
+    if (r.hasParam("expression")) {
+      String exprString = r.getString("expression");
+      Expression expr;
+
+      try {
+        expr = JavascriptCompiler.compile(exprString);
+      } catch (ParseException pe) {
+        // Static error (e.g. bad JavaScript syntax):
+        r.fail("expression", "could not parse expression: " + pe, pe);
+
+        // Dead code but compiler disagrees:
+        expr = null;
+      } catch (IllegalArgumentException iae) {
+        // Static error (e.g. bad JavaScript syntax):
+        r.fail("expression", "could not parse expression: " + iae, iae);
+
+        // Dead code but compiler disagrees:
+        expr = null;
+      }
+
+      Map<String,FieldDef> allFields = new HashMap<String,FieldDef>(state.getAllFields());
+      allFields.putAll(pendingFieldDefs);
+
+      ValueSource values;
+      try {
+        values = expr.getValueSource(new FieldDefBindings(allFields));
+      } catch (RuntimeException re) {
+        // Dynamic error (e.g. referred to a field that
+        // doesn't exist):
+        r.fail("expression", "could not evaluate expression: " + re, re);
+
+        // Dead code but compiler disagrees:
+        values = null;
+      }
+
+      return new FieldDef(name, null, "virtual", null, null, null, true, null, null, null, false, null, null, 0.0f, 0L, values);
+
+    } else {
+      // nocommit cutover all tests to expression fields and remove this hack:
+      Request r2 = r.getStruct("recencyScoreBlend");
+      String timeStampField = r2.getString("timeStampField");
+      FieldDef fd;
+      try {
+        fd = state.getField(timeStampField);
+      } catch (IllegalArgumentException iae) {
+        fd = pendingFieldDefs.get(timeStampField);
+        if (fd == null) {
+          r2.fail("timeStampField", "field \"" + timeStampField + "\" was not yet registered");
+        }
+      }
+      if (fd.fieldType.docValueType() != DocValuesType.NUMERIC) {
+        r2.fail("timeStampField", "field \"" + fd.name + "\" must be registered with type=long and sort=true");
+      }
+      float maxBoost = r2.getFloat("maxBoost");
+      long range = r2.getLong("range");
+      return new FieldDef(name, null, "virtual", null, null, null, true, null, null, null, false, null, fd.name, maxBoost, range, null);
+    }
   }
 
   private FieldDef parseOneFieldType(Request r, IndexState state, Map<String,FieldDef> pendingFieldDefs, String name, JSONObject o) {
@@ -688,7 +735,7 @@ public class RegisterFieldHandler extend
     // nocommit facetsConfig.setIndexFieldName
     // nocommit facetsConfig.setRequireDimCount
 
-    return new FieldDef(name, ft, type, facet, pf, dvf, multiValued, sim, indexAnalyzer, searchAnalyzer, highlighted, liveValuesIDField, null, 0.0f, 0l);
+    return new FieldDef(name, ft, type, facet, pf, dvf, multiValued, sim, indexAnalyzer, searchAnalyzer, highlighted, liveValuesIDField, null, 0.0f, 0l, null);
   }
 
   /** Messy: we need this for indexed-but-not-tokenized
@@ -873,7 +920,7 @@ public class RegisterFieldHandler extend
       JSONObject o;
       try {
         o = (JSONObject) JSONValue.parseStrict(json);
-      } catch (ParseException pe) {
+      } catch (net.minidev.json.parser.ParseException pe) {
         // BUG
         throw new RuntimeException(pe);
       }
@@ -1048,6 +1095,11 @@ public class RegisterFieldHandler extend
 
       Set<String> seen = new HashSet<String>();
 
+      // We make two passes.  In the first pass, we do the
+      // "real" fields, and second pass does the virtual
+      // fields, so that any fields the virtual field
+      // references are guaranteed to exist, in a single
+      // request (or, from the saved json):
       for(int pass=0;pass<2;pass++) {
         Iterator<Map.Entry<String,Object>> it = r.getParams();
         while(it.hasNext()) {
@@ -1098,7 +1150,7 @@ public class RegisterFieldHandler extend
           JSONObject o;
           try {
             o = (JSONObject) JSONValue.parseStrict(saveStates.get(ent.getKey()));
-          } catch (ParseException pe) {
+          } catch (net.minidev.json.parser.ParseException pe) {
             // BUG
             assert false;
             throw new RuntimeException(pe);

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java Wed Jan  8 12:00:31 2014
@@ -41,20 +41,21 @@ import org.apache.lucene.facet.FacetsCol
 import org.apache.lucene.facet.LabelAndValue;
 import org.apache.lucene.facet.range.LongRange;
 import org.apache.lucene.facet.range.LongRangeFacetCounts;
-import org.apache.lucene.facet.range.Range;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts;
 import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager.SearcherAndTaxonomy;
 import org.apache.lucene.facet.taxonomy.TaxonomyFacetCounts;
 import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.BooleanFilter;
 import org.apache.lucene.queries.CommonTermsQuery;
+import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queryparser.classic.MultiFieldQueryParser;
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.search.BooleanClause;
@@ -631,13 +632,18 @@ public class SearchHandler extends Handl
     return config;
   }
 
-  private static Sort parseSort(long timeStamp, IndexState state, List<Object> fields) {
+  /** Decodes a list of Request into the corresponding
+   *  Sort. */
+  private static Sort parseSort(long timeStamp, IndexState state, List<Object> fields, List<String> sortFieldNames) {
     List<SortField> sortFields = new ArrayList<SortField>();
     for(Object _sub : fields) {
       Request sub = (Request) _sub;
 
       String fieldName = sub.getString("field");
       SortField sf;
+      if (sortFieldNames != null) {
+        sortFieldNames.add(fieldName);
+      }
       if (fieldName.equals("docid")) {
         sf = SortField.FIELD_DOC;
       } else if (fieldName.equals("score")) {
@@ -652,11 +658,13 @@ public class SearchHandler extends Handl
           fd = null;
         }
 
-        if (fd.blendFieldName != null) {
+        if (fd.valueSource != null) {
+          sf = fd.valueSource.getSortField(sub.getBoolean("reverse"));
+        } else if (fd.blendFieldName != null) {
           sf = new SortField(fd.name, new RecencyBlendedFieldComparatorSource(fd.blendFieldName, fd.blendMaxBoost, timeStamp, fd.blendRange), sub.getBoolean("reverse"));
         } else {
-
-          if (fd.fieldType.docValueType() == null) {
+          if ((fd.fieldType != null && fd.fieldType.docValueType() == null) ||
+              (fd.fieldType == null && fd.valueSource == null)) {
             sub.fail("field", "field \"" + fieldName + "\" was not registered with sort=true");
           }
 
@@ -709,7 +717,8 @@ public class SearchHandler extends Handl
   private void fillFields(IndexState state, HighlighterConfig highlighter, IndexSearcher s,
                           JSONObject result, ScoreDoc hit, Set<String> fields,
                           Map<String,String[]> highlights,
-                          int hiliteHitIndex, Sort sort) throws IOException {
+                          int hiliteHitIndex, Sort sort,
+                          List<String> sortFieldNames) throws IOException {
     //System.out.println("fillFields fields=" + fields);
     if (fields != null) {
 
@@ -725,21 +734,39 @@ public class SearchHandler extends Handl
         // We detect invalid field above:
         assert fd != null;
 
-        Object v = doc.get(name);
-        if (v != null) {
-          // We caught same field name above:
-          assert !result.containsKey(name);
+        // nocommit also allow pulling from doc values
+        if (fd.valueSource != null) {
+          List<AtomicReaderContext> leaves = s.getIndexReader().leaves();
+          AtomicReaderContext leaf = leaves.get(ReaderUtil.subIndex(hit.doc, leaves));
+          Map<String,Object> context = new HashMap<String,Object>();
+
+          // nocommit: should we only do this if "wantsScores"?
+          // where do we get that boolean!!
+          int docID = hit.doc - leaf.docBase;
+
+          // nocommit not quite right?  what if app didn't
+          // sort by score but uses it in the expr that it
+          // is sorting by?
+          context.put("scorer", new CannedScorer(docID, hit.score));
+          FunctionValues segValues = fd.valueSource.getValues(context, leaf);
+          result.put(name, segValues.doubleVal(docID));
+        } else {
+          Object v = doc.get(name);
+          if (v != null) {
+            // We caught same field name above:
+            assert !result.containsKey(name);
 
-          if (fd.multiValued == false) {
-            result.put(name, convertType(fd, v));
-          } else {
-            JSONArray arr = new JSONArray();
-            result.put(name, arr);
-            if (!(v instanceof List)) {
-              arr.add(convertType(fd, v));
+            if (fd.multiValued == false) {
+              result.put(name, convertType(fd, v));
             } else {
-              for(Object o : (List<Object>) v) {
-                arr.add(convertType(fd, o));
+              JSONArray arr = new JSONArray();
+              result.put(name, arr);
+              if (!(v instanceof List)) {
+                arr.add(convertType(fd, v));
+              } else {
+                for(Object o : (List<Object>) v) {
+                  arr.add(convertType(fd, o));
+                }
               }
             }
           }
@@ -768,21 +795,15 @@ public class SearchHandler extends Handl
         result.put("sortFields", o4);
         SortField[] sortFields = sort.getSort();
         for(int i=0;i<sortFields.length;i++) {
+          // We must use a separate list because an expr's
+          // SortField doesn't know the virtual field name
+          // (it returns the expression string from
+          // .getField):
+          String fieldName = sortFieldNames.get(i);
           if (fd.fields[i] instanceof BytesRef) {
-            o4.put(sortFields[i].getField(),
-                   ((BytesRef) fd.fields[i]).utf8ToString());
+            o4.put(fieldName, ((BytesRef) fd.fields[i]).utf8ToString());
           } else {
-            String field = sortFields[i].getField();
-            if (field == null) {
-              if (sortFields[i] == SortField.FIELD_DOC) {
-                field = "docid";
-              } else if (sortFields[i] == SortField.FIELD_SCORE) {
-                field = "score";
-              } else {
-                throw new AssertionError();
-              }
-            }
-            o4.put(field, fd.fields[i]);
+            o4.put(fieldName, fd.fields[i]);
           }
         }
       }
@@ -1099,7 +1120,7 @@ public class SearchHandler extends Handl
         Request childHits = pr.r.getStruct("childHits");
         BlockJoinQueryChild child = new BlockJoinQueryChild();
         if (childHits.hasParam("sort")) {
-          child.sort = parseSort(timeStamp, state, childHits.getList("sort"));
+          child.sort = parseSort(timeStamp, state, childHits.getList("sort"), null);
         }
         child.maxChildren = childHits.getInt("maxChildren");
         child.trackScores = childHits.getBoolean("trackScores");
@@ -1757,8 +1778,15 @@ public class SearchHandler extends Handl
         if (!highlight.equals("no") && !fd.highlighted) {
           r.fail("retrieveFields", "field \"" + field + "\" was not indexed with highlight=true");
         }
-        if (!fd.fieldType.stored()) {
-          // nocommit allow pulling from DV?  need separate dvFields?
+
+        // nocommit allow pulling from DV?  need separate
+        // dvFields?
+
+        if (fd.fieldType == null) {
+          if (fd.valueSource == null) {
+            r.fail("retrieveFields", "field \"" + field + "\" was not registered with store=true");
+          }
+        } else if (!fd.fieldType.stored()) {
           r.fail("retrieveFields", "field \"" + field + "\" was not registered with store=true");
         }
       }
@@ -1812,12 +1840,15 @@ public class SearchHandler extends Handl
       Sort groupSort = null;
       Sort sort;
       Request sortRequest;
+      List<String> sortFieldNames;
       if (r.hasParam("sort")) {
         sortRequest = r.getStruct("sort");
-        sort = parseSort(timeStamp, state, sortRequest.getList("fields"));
+        sortFieldNames = new ArrayList<String>();
+        sort = parseSort(timeStamp, state, sortRequest.getList("fields"), sortFieldNames);
       } else {
         sortRequest = null;
         sort = null;
+        sortFieldNames = null;
       }
 
       int topHits = r.getInt("topHits");
@@ -1838,7 +1869,7 @@ public class SearchHandler extends Handl
         }
 
         if (grouping.hasParam("sort")) {
-          groupSort = parseSort(timeStamp, state, grouping.getList("sort"));
+          groupSort = parseSort(timeStamp, state, grouping.getList("sort"), null);
         } else {
           groupSort = Sort.RELEVANCE;
         }
@@ -2155,7 +2186,7 @@ public class SearchHandler extends Handl
               if (fields != null || highlightFields != null) {
                 JSONObject o7 = new JSONObject();
                 o6.put("fields", o7);
-                fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, sort);
+                fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, sort, sortFieldNames);
               }
 
               hitIndex++;
@@ -2201,7 +2232,7 @@ public class SearchHandler extends Handl
               // nocommit where does parent score come
               // from ...
               ScoreDoc sd = new ScoreDoc(group.groupValue.intValue(), 0.0f);
-              fillFields(state, highlighter, s.searcher, o4, sd, fields, highlights, hitIndex, sort);
+              fillFields(state, highlighter, s.searcher, o4, sd, fields, highlights, hitIndex, sort, sortFieldNames);
             }
             hitIndex++;
 
@@ -2236,7 +2267,7 @@ public class SearchHandler extends Handl
               if (fields != null || highlightFields != null) {
                 JSONObject o7 = new JSONObject();
                 o6.put("fields", o7);
-                fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, child.sort);
+                fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, child.sort, null);
               }
 
               hitIndex++;
@@ -2265,7 +2296,7 @@ public class SearchHandler extends Handl
           if (fields != null || highlightFields != null) {
             JSONObject o4 = new JSONObject();
             o3.put("fields", o4);
-            fillFields(state, highlighter, s.searcher, o4, hit, fields, highlights, hitIndex, sort);
+            fillFields(state, highlighter, s.searcher, o4, hit, fields, highlights, hitIndex, sort, sortFieldNames);
           }
         }
       }

Modified: lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestIndexing.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestIndexing.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestIndexing.java Wed Jan  8 12:00:31 2014
@@ -28,6 +28,7 @@ import net.minidev.json.JSONArray;
 import net.minidev.json.JSONObject;
 
 public class TestIndexing extends ServerBaseTestCase {
+
   @BeforeClass
   public static void initClass() throws Exception {
     startServer();

Added: lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java?rev=1556508&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java Wed Jan  8 12:00:31 2014
@@ -0,0 +1,176 @@
+package org.apache.lucene.server;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONObject;
+
+public class TestVirtualFields extends ServerBaseTestCase {
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    startServer();
+    createAndStartIndex();
+    registerFields();
+    commit();
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+  }
+
+  private static void registerFields() throws Exception {
+    JSONObject o = new JSONObject();
+    put(o, "boost", "{type: float, sort: true}");
+    put(o, "text", "{type: text, analyzer: WhitespaceAnalyzer}");
+    put(o, "logboost", "{type: virtual, expression: ln(boost)}");
+    put(o, "scoreboost", "{type: virtual, expression: _score+ln(boost)}");
+    put(o, "id", "{type: int, sort: true, store: true, index: false}");
+    JSONObject o2 = new JSONObject();
+    o2.put("fields", o);
+    send("registerFields", o2);
+  }
+
+  // nocommit need dynamic exprs too (defined for one request)
+
+  /** Non-reversed sort by virtual field */
+  public void testSortByVirtualFieldStraight() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, boost: 1.0}}");
+    long gen = getLong(send("addDocument", "{fields: {id: 1, boost: 2.0}}"), "indexGen");
+    JSONObject result = send("search", "{query: MatchAllDocsQuery, sort: {fields: [{field: logboost}]}, retrieveFields: [id], searcher: {indexGen: " + gen + "}}");
+    assertEquals(2, getInt(result, "totalHits"));
+    assertEquals(0, getInt(result, "hits[0].fields.id"));
+    assertEquals(1, getInt(result, "hits[1].fields.id"));
+
+    assertEquals(0.0f, getFloat(result, "hits[0].fields.sortFields.logboost"), .0001f);
+    assertEquals(.6931f, getFloat(result, "hits[1].fields.sortFields.logboost"), .0001f);
+  }
+
+  /** Reversed sort by virtual field */
+  public void testSortByVirtualFieldReversed() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, boost: 1.0}}");
+    long gen = getLong(send("addDocument", "{fields: {id: 1, boost: 2.0}}"), "indexGen");
+    JSONObject result = send("search", "{query: MatchAllDocsQuery, sort: {fields: [{field: logboost, reverse: true}]}, retrieveFields: [id], searcher: {indexGen: " + gen + "}}");
+    assertEquals(2, getInt(result, "totalHits"));
+    assertEquals(1, getInt(result, "hits[0].fields.id"));
+    assertEquals(0, getInt(result, "hits[1].fields.id"));
+
+    assertEquals(.6931f, getFloat(result, "hits[0].fields.sortFields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat(result, "hits[1].fields.sortFields.logboost"), .0001f);
+  }
+
+  /** Sort by virtual field, and ask for its value */
+  public void testRetrieveVirtualFieldWithSort() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, boost: 1.0}}");
+    long gen = getLong(send("addDocument", "{fields: {id: 1, boost: 2.0}}"), "indexGen");
+    JSONObject result = send("search", "{query: MatchAllDocsQuery, sort: {fields: [{field: logboost}]}, retrieveFields: [id, logboost], searcher: {indexGen: " + gen + "}}");
+    assertEquals(2, getInt(result, "totalHits"));
+    assertEquals(0, getInt(result, "hits[0].fields.id"));
+    assertEquals(1, getInt(result, "hits[1].fields.id"));
+
+    assertEquals(0.0f, getFloat(result, "hits[0].fields.logboost"), .0001f);
+    assertEquals(.6931f, getFloat(result, "hits[1].fields.logboost"), .0001f);
+  }
+
+  /** Don't sort by virtual field, and ask for its value */
+  public void testRetrieveVirtualFieldWithoutSort() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, boost: 1.0}}");
+    long gen = getLong(send("addDocument", "{fields: {id: 1, boost: 2.0}}"), "indexGen");
+    JSONObject result = send("search", "{query: MatchAllDocsQuery, sort: {fields: [{field: id, reverse: true}]}, retrieveFields: [id, logboost], searcher: {indexGen: " + gen + "}}");
+    assertEquals(2, getInt(result, "totalHits"));
+    assertEquals(1, getInt(result, "hits[0].fields.id"));
+    assertEquals(0, getInt(result, "hits[1].fields.id"));
+
+    assertEquals(.6931f, getFloat(result, "hits[0].fields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat(result, "hits[1].fields.logboost"), .0001f);
+  }
+
+  public void testWithScore() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {text: 'the wind is howling like this swirling storm inside', id: 0, boost: 1.0}}");
+    long gen = getLong(send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}"), "indexGen");
+    JSONObject result = send("search", "{queryText: wind, sort: {fields: [{field: scoreboost, reverse: true}]}, retrieveFields: [id, logboost], searcher: {indexGen: " + gen + "}}");
+    assertEquals(2, getInt(result, "totalHits"));
+    assertEquals(1, getInt(result, "hits[0].fields.id"));
+    assertEquals(0, getInt(result, "hits[1].fields.id"));
+
+    assertEquals(.6931f, getFloat(result, "hits[0].fields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat(result, "hits[1].fields.logboost"), .0001f);
+  }
+
+  /** Also tries to retrieve the scoreboost */
+  // nocommit fixme: it currently fails because of NaN disaster
+  /*
+  public void testWithScore2() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {text: 'the wind is howling like this swirling storm inside', id: 0, boost: 1.0}}");
+    long gen = getLong(send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}"), "indexGen");
+    JSONObject result = send("search", "{queryText: wind, sort: {fields: [{field: scoreboost, reverse: true}]}, retrieveFields: [id, scoreboost, logboost], searcher: {indexGen: " + gen + "}}");
+    assertEquals(2, getInt(result, "totalHits"));
+    assertEquals(1, getInt(result, "hits[0].fields.id"));
+    assertEquals(0, getInt(result, "hits[1].fields.id"));
+
+    assertEquals(.80361f, getFloat(result, "hits[0].fields.scoreboost"), .0001f);
+    assertEquals(.11046f, getFloat(result, "hits[1].fields.scoreboost"), .0001f);
+
+    assertEquals(.6931f, getFloat(result, "hits[0].fields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat(result, "hits[1].fields.logboost"), .0001f);
+  }
+  */
+
+  public void testSyntaxError() throws Exception {
+    deleteAllDocs();
+    try {
+      send("registerFields", "{fields: {bad: {type: virtual, expression: 'ln(boost'}}}");
+    } catch (IOException ioe) {
+      String message = ioe.toString();
+      assertTrue(message.contains("registerFields > fields > bad > expression: could not parse expression"));
+      assertTrue(message.contains("unexpected token end of expression"));
+    }
+  }
+
+  public void testNonExistentField() throws Exception {
+    deleteAllDocs();
+    try {
+      send("registerFields", "{fields: {bad: {type: virtual, expression: 'ln(bad2)'}}}");
+    } catch (IOException ioe) {
+      String message = ioe.toString();
+      assertTrue(message.contains("registerFields > fields > bad > expression: could not evaluate expression"));
+      assertTrue(message.contains("Invalid reference 'bad2'"));
+    }
+  }
+
+  public void testExistentButNoDocValuesField() throws Exception {
+    deleteAllDocs();
+    try {
+      send("registerFields", "{fields: {bad2: {type: int, store: true}, bad: {type: virtual, expression: 'ln(bad2)'}}}");
+    } catch (IOException ioe) {
+      String message = ioe.toString();
+      assertTrue(message.contains("registerFields > fields > bad > expression: could not evaluate expression"));
+      assertTrue(message.contains("Field 'bad2' cannot be used in an expression: it was not registered with sort=true"));
+    }
+  }
+}

Modified: lucene/dev/branches/lucene5376/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java?rev=1556508&r1=1556507&r2=1556508&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java (original)
+++ lucene/dev/branches/lucene5376/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java Wed Jan  8 12:00:31 2014
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.document.NumericDocValuesField; // javadocs
+import org.apache.lucene.expressions.Bindings;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.SimpleBindings;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
@@ -66,7 +67,7 @@ import org.apache.lucene.util.BytesRefIt
 public class DocumentExpressionDictionary extends DocumentDictionary {
   
   private final ValueSource weightsValueSource;
-  
+
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms and computes the corresponding weights of the term by compiling the
@@ -77,7 +78,7 @@ public class DocumentExpressionDictionar
       String weightExpression, Set<SortField> sortFields) {
     this(reader, field, weightExpression, sortFields, null);
   }
-  
+
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payloadField</code> for the corresponding payloads
@@ -87,6 +88,17 @@ public class DocumentExpressionDictionar
    */
   public DocumentExpressionDictionary(IndexReader reader, String field,
       String weightExpression, Set<SortField> sortFields, String payload) {
+    this(reader, field, weightExpression, buildBindings(sortFields), payload);
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms, <code>payloadField</code> for the corresponding payloads
+   * and computes the corresponding weights of the term by compiling the
+   * user-defined <code>weightExpression</code> using the provided bindings.
+   */
+  public DocumentExpressionDictionary(IndexReader reader, String field,
+      String weightExpression, Bindings bindings, String payload) {
     super(reader, field, null, payload);
     Expression expression = null;
     try {
@@ -94,12 +106,15 @@ public class DocumentExpressionDictionar
     } catch (ParseException e) {
       throw new RuntimeException();
     }
+    weightsValueSource = expression.getValueSource(bindings);
+  }
+
+  private static Bindings buildBindings(Set<SortField> sortFields) {
     SimpleBindings bindings = new SimpleBindings();
     for (SortField sortField: sortFields) {
       bindings.add(sortField);
     }
-    
-    weightsValueSource = expression.getValueSource(bindings);
+    return bindings;
   }
   
   /**