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 20:57:21 UTC

svn commit: r1556620 - in /lucene/dev/branches/lucene5376/lucene/server/src: java/org/apache/lucene/server/handlers/SearchHandler.java test/org/apache/lucene/server/ServerBaseTestCase.java test/org/apache/lucene/server/TestVirtualFields.java

Author: mikemccand
Date: Wed Jan  8 19:57:20 2014
New Revision: 1556620

URL: http://svn.apache.org/r1556620
Log:
LUCENE-5376: also allow dynamic expression per-request

Modified:
    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/ServerBaseTestCase.java
    lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java

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=1556620&r1=1556619&r2=1556620&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 19:57:20 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.server.handler
 
 import java.io.IOException;
 import java.text.BreakIterator;
+import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -33,6 +34,9 @@ import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.lucene.document.FieldType.NumericType;
+import org.apache.lucene.expressions.Bindings;
+import org.apache.lucene.expressions.Expression;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.facet.DrillDownQuery;
 import org.apache.lucene.facet.DrillSideways;
 import org.apache.lucene.facet.FacetResult;
@@ -56,6 +60,7 @@ 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.queries.function.ValueSource;
 import org.apache.lucene.queryparser.classic.MultiFieldQueryParser;
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.search.BooleanClause;
@@ -105,6 +110,7 @@ import org.apache.lucene.search.postings
 import org.apache.lucene.search.postingshighlight.WholeBreakIterator;
 import org.apache.lucene.server.Constants;
 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;
@@ -120,7 +126,6 @@ import org.apache.lucene.util.automaton.
 import net.minidev.json.JSONArray;
 import net.minidev.json.JSONObject;
 import net.minidev.json.JSONValue;
-import net.minidev.json.parser.ParseException;
 
 // nocommit why no double range faceting?
 
@@ -326,6 +331,11 @@ public class SearchHandler extends Handl
                   SEARCHER_VERSION_TYPE),
         new Param("startHit", "Which hit to start from (for pagination).", new IntType(), 0),
         new Param("topHits", "How many top hits to retrieve.", new IntType(), 10),
+        new Param("virtualFields", "Defines virtual fields (name'd dynamic expressions) for this query.",
+                  new ListType(
+                      new StructType(
+                          new Param("name", "Virtual field's name.  Must be different from registered fields and any other virtual fields.", new StringType()),
+                          new Param("expression", "JavaScript expression defining this field's values.", new StringType())))),
         new Param("searchAfter", "Only return hits after the specified hit; this is useful for deep paging",
                   new StructType(
                                  new Param("lastDoc", "Last docID of the previous page.", new IntType()),
@@ -634,7 +644,7 @@ public class SearchHandler extends Handl
 
   /** Decodes a list of Request into the corresponding
    *  Sort. */
-  private static Sort parseSort(long timeStamp, IndexState state, List<Object> fields, List<String> sortFieldNames) {
+  private static Sort parseSort(long timeStamp, IndexState state, List<Object> fields, List<String> sortFieldNames, Map<String,FieldDef> dynamicFields) {
     List<SortField> sortFields = new ArrayList<SortField>();
     for(Object _sub : fields) {
       Request sub = (Request) _sub;
@@ -649,11 +659,9 @@ public class SearchHandler extends Handl
       } else if (fieldName.equals("score")) {
         sf = SortField.FIELD_SCORE;
       } else {
-        FieldDef fd;
-        try {
-          fd = state.getField(fieldName);
-        } catch (IllegalArgumentException iae) {
-          sub.fail("field", iae.toString());
+        FieldDef fd = dynamicFields.get(fieldName);
+        if (fd == null) {
+          sub.fail("field", "field \"" + fieldName + "\" was not registered and was not specified as a dynamicField");
           // Dead code but compiler disagrees:
           fd = null;
         }
@@ -718,7 +726,8 @@ public class SearchHandler extends Handl
                           JSONObject result, ScoreDoc hit, Set<String> fields,
                           Map<String,String[]> highlights,
                           int hiliteHitIndex, Sort sort,
-                          List<String> sortFieldNames) throws IOException {
+                          List<String> sortFieldNames,
+                          Map<String,FieldDef> dynamicFields) throws IOException {
     //System.out.println("fillFields fields=" + fields);
     if (fields != null) {
 
@@ -729,7 +738,7 @@ public class SearchHandler extends Handl
       Map<String,Object> doc = highlighter.getDocument(state, s, hit.doc);
 
       for (String name : fields) {
-        FieldDef fd = state.getField(name);
+        FieldDef fd = dynamicFields.get(name);
 
         // We detect invalid field above:
         assert fd != null;
@@ -774,12 +783,14 @@ public class SearchHandler extends Handl
       for (Map.Entry<String,String[]> ent : highlights.entrySet()) {
         String v = ent.getValue()[hiliteHitIndex];
         if (v != null) {
+          Object o;
           try {
-            result.put(ent.getKey(), JSONValue.parseStrict(v));
-          } catch (ParseException pe) {
+            o = JSONValue.parseStrict(v);
+          } catch (net.minidev.json.parser.ParseException pe) {
             // BUG
             throw new RuntimeException(pe);
           }
+          result.put(ent.getKey(), o);
         }
       }
     }
@@ -820,7 +831,7 @@ public class SearchHandler extends Handl
     }
   }
 
-  private static Filter parseFilter(long timeStamp, Request topRequest, IndexState state, Request r) {
+  private static Filter parseFilter(long timeStamp, Request topRequest, IndexState state, Request r, Map<String,FieldDef> dynamicFields) {
     Filter f;
 
     Request.PolyResult pr = r.getPoly("class");
@@ -831,12 +842,12 @@ public class SearchHandler extends Handl
       }
       f = new QueryWrapperFilter(new TermQuery(new Term(fd.name, "1")));
     } else if (pr.name.equals("QueryWrapperFilter")) {
-      return new QueryWrapperFilter(parseQuery(timeStamp, topRequest, state, pr.r.getStruct("query"), null, null));
+      return new QueryWrapperFilter(parseQuery(timeStamp, topRequest, state, pr.r.getStruct("query"), null, null, dynamicFields));
     } else if (pr.name.equals("BooleanFilter")) {
       BooleanFilter bf = new BooleanFilter();
       for (Object o : pr.r.getList("subFilters")) {
         Request sub = (Request) o;
-        bf.add(parseFilter(timeStamp, topRequest, state, sub.getStruct("filter")),
+        bf.add(parseFilter(timeStamp, topRequest, state, sub.getStruct("filter"), dynamicFields),
                parseBooleanOccur(sub.getEnum("occur")));
       }
       f = bf;
@@ -860,7 +871,7 @@ public class SearchHandler extends Handl
         }
         f = state.cachedFilters.get(id);
         if (f == null) {
-          CachingWrapperFilter cwf = new CachingWrapperFilter(parseFilter(timeStamp, topRequest, state, fr)) {
+          CachingWrapperFilter cwf = new CachingWrapperFilter(parseFilter(timeStamp, topRequest, state, fr, dynamicFields)) {
               @Override
               protected DocIdSet cacheImpl(DocIdSetIterator iterator, AtomicReader reader) throws IOException {
                 // nocommit let caller control which Bits
@@ -906,7 +917,8 @@ public class SearchHandler extends Handl
 
   @SuppressWarnings("unchecked")
   private static Query parseQuery(long timeStamp, Request topRequest, IndexState state, Request r, String field,
-                                  Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector) {
+                                  Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector,
+                                  Map<String,FieldDef> dynamicFields) {
     Query q;
     Request.PolyResult pr = r.getPoly("class");
     if (r.hasParam("field")) {
@@ -927,7 +939,7 @@ public class SearchHandler extends Handl
       for(Object o : r2.getList("subQueries")) {
         Request r3 = (Request) o;
         BooleanClause.Occur occur = parseBooleanOccur(r3.getEnum("occur"));
-        bq.add(parseQuery(timeStamp, topRequest, state, r3.getStruct("query"), field, useBlockJoinCollector), occur);
+        bq.add(parseQuery(timeStamp, topRequest, state, r3.getStruct("query"), field, useBlockJoinCollector, dynamicFields), occur);
       }
       q = bq;
     } else if (pr.name.equals("CommonTermsQuery")) {
@@ -944,9 +956,9 @@ public class SearchHandler extends Handl
       q = ctq;
     } else if (pr.name.equals("ConstantScoreQuery")) {
       if (pr.r.hasParam("query")) {
-        q = new ConstantScoreQuery(parseQuery(timeStamp, topRequest, state, pr.r.getStruct("query"), field, useBlockJoinCollector));
+        q = new ConstantScoreQuery(parseQuery(timeStamp, topRequest, state, pr.r.getStruct("query"), field, useBlockJoinCollector, dynamicFields));
       } else {
-        q = new ConstantScoreQuery(parseFilter(timeStamp, topRequest, state, pr.r.getStruct("filter")));
+        q = new ConstantScoreQuery(parseFilter(timeStamp, topRequest, state, pr.r.getStruct("filter"), dynamicFields));
       }
     } else if (pr.name.equals("FuzzyQuery")) {
       if (field == null) {
@@ -1089,8 +1101,8 @@ public class SearchHandler extends Handl
       }
       q = new TermQuery(new Term(field, pr.r.getString("term")));
     } else if (pr.name.equals("ToParentBlockJoinQuery")) {
-      Query childQuery = parseQuery(timeStamp, topRequest, state, pr.r.getStruct("childQuery"), field, useBlockJoinCollector);
-      Filter parentsFilter = parseFilter(timeStamp, topRequest, state, pr.r.getStruct("parentsFilter"));
+      Query childQuery = parseQuery(timeStamp, topRequest, state, pr.r.getStruct("childQuery"), field, useBlockJoinCollector, dynamicFields);
+      Filter parentsFilter = parseFilter(timeStamp, topRequest, state, pr.r.getStruct("parentsFilter"), dynamicFields);
       String scoreModeString = pr.r.getEnum("scoreMode");
       ScoreMode scoreMode;
       if (scoreModeString.equals("None")) {
@@ -1116,7 +1128,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"), null);
+          child.sort = parseSort(timeStamp, state, childHits.getList("sort"), null, dynamicFields);
         }
         child.maxChildren = childHits.getInt("maxChildren");
         child.trackScores = childHits.getBoolean("trackScores");
@@ -1129,7 +1141,7 @@ public class SearchHandler extends Handl
       DisjunctionMaxQuery dmq = new DisjunctionMaxQuery(r2.getFloat("tieBreakMultiplier"));
       q = dmq;
       for(Object o : subQueries) {
-        dmq.add(parseQuery(timeStamp, topRequest, state, (Request) o, field, useBlockJoinCollector));
+        dmq.add(parseQuery(timeStamp, topRequest, state, (Request) o, field, useBlockJoinCollector, dynamicFields));
       }
     } else if (pr.name.equals("text")) {
       Request r2 = pr.r;
@@ -1422,7 +1434,7 @@ public class SearchHandler extends Handl
   }
 
   /** Fold in any drillDowns requests into the query. */
-  private static DrillDownQuery addDrillDowns(long timeStamp, IndexState state, Request r, Query q) {
+  private static DrillDownQuery addDrillDowns(long timeStamp, IndexState state, Request r, Query q, Map<String,FieldDef> dynamicFields) {
     // Always create a DrillDownQuery; if there
     // are no drill-downs it will just rewrite to the
     // original query:
@@ -1437,7 +1449,7 @@ public class SearchHandler extends Handl
 
           if (fr.hasParam("query")) {
             // Drill down by query:
-            ddq.add(fd.name, parseQuery(timeStamp, null, state, r.getStruct("query"), fd.name, null));
+            ddq.add(fd.name, parseQuery(timeStamp, null, state, r.getStruct("query"), fd.name, null, dynamicFields));
           } else {
             String[] path;
             if (fr.isString("value")) {
@@ -1460,7 +1472,8 @@ public class SearchHandler extends Handl
   }
 
   private static Query extractQuery(IndexState state, Request r, long timeStamp,
-                                    Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector) throws Exception {
+                                    Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector,
+                                    Map<String,FieldDef> dynamicFields) throws Exception {
     Query q;
     if (r.hasParam("queryText")) {
       QueryParser queryParser = createQueryParser(state, r, null);
@@ -1479,7 +1492,7 @@ public class SearchHandler extends Handl
         q = null;
       }
     } else if (r.hasParam("query")) {
-      q = parseQuery(timeStamp, r, state, r.getStruct("query"), null, useBlockJoinCollector);
+      q = parseQuery(timeStamp, r, state, r.getStruct("query"), null, useBlockJoinCollector, dynamicFields);
     } else {
       q = new MatchAllDocsQuery();
     }
@@ -1683,6 +1696,61 @@ public class SearchHandler extends Handl
     }
   }
 
+  /** Parses any virtualFields, which define dynamic
+   *  (expression) fields for this one request. */
+  private static Map<String,FieldDef> getDynamicFields(IndexState state, Request r) {
+    Map<String,FieldDef> dynamicFields;
+    if (r.hasParam("virtualFields")) {
+      dynamicFields = new HashMap<String,FieldDef>();
+      dynamicFields.putAll(state.getAllFields());
+      Bindings bindings = new FieldDefBindings(dynamicFields);
+      for(Object o : r.getList("virtualFields")) {
+        Request oneField = (Request) o;
+        String name = oneField.getString("name");
+        String exprString = oneField.getString("expression");
+
+        Expression expr;
+        try {
+          expr = JavascriptCompiler.compile(exprString);
+        } catch (ParseException pe) {
+          // Static error (e.g. bad JavaScript syntax):
+          oneField.fail("expressoin", "could not parse expression: " + pe, pe);
+
+          // Dead code but compiler disagrees:
+          expr = null;
+        } catch (IllegalArgumentException iae) {
+          // Static error (e.g. bad JavaScript syntax):
+          oneField.fail("expression", "could not parse expression: " + iae, iae);
+
+          // Dead code but compiler disagrees:
+          expr = null;
+        }
+
+        ValueSource values;
+        try {
+          values = expr.getValueSource(bindings);
+        } catch (RuntimeException re) {
+          // Dynamic error (e.g. referred to a field that
+          // doesn't exist):
+          oneField.fail("expression", "could not evaluate expression: " + re, re);
+
+          // Dead code but compiler disagrees:
+          values = null;
+        }
+
+        FieldDef fd = new FieldDef(name, null, "virtual", null, null, null, true, null, null, null, false, null, null, 0.0f, 0L, values);
+
+        if (dynamicFields.put(name, fd) != null) {
+          oneField.fail("name", "registered field or dynamic field \"" + name + "\" already exists");
+        }
+      }
+    } else {
+      dynamicFields = state.getAllFields();
+    }
+
+    return dynamicFields;
+  }
+
   @SuppressWarnings("unchecked")
   @Override
   public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
@@ -1705,11 +1773,14 @@ public class SearchHandler extends Handl
 
     JSONObject diagnostics = new JSONObject();
 
-    Query q = extractQuery(state, r, timeStamp, useBlockJoinCollector);
+    Map<String,FieldDef> dynamicFields = getDynamicFields(state, r);
+
+    Query q = extractQuery(state, r, timeStamp, useBlockJoinCollector, dynamicFields);
 
     final Filter filter;
     if (r.hasParam("filter")) {
-      filter = parseFilter(timeStamp, r, state, r.getStruct("filter"));
+      // nocommit allow Filter against dynamic field?
+      filter = parseFilter(timeStamp, r, state, r.getStruct("filter"), dynamicFields);
     } else {
       filter = null;
     }
@@ -1759,11 +1830,10 @@ public class SearchHandler extends Handl
           r.fail("retrieveFields", "field \"" + field + "\" cannot be retrieved more than once");
         }       
         fieldSeen.add(field);
-        FieldDef fd;
-        try {
-          fd = state.getField(field);
-        } catch (IllegalArgumentException iae) {
-          r.fail("retrieveFields", iae.toString());
+
+        FieldDef fd = dynamicFields.get(field);
+        if (fd == null) {
+          r.fail("retrieveFields", "field \"" + field + "\" was not registered and was not specified as a dynamicField");
           // Dead code but compiler disagrees:
           fd = null;
         }
@@ -1833,7 +1903,7 @@ public class SearchHandler extends Handl
       // in-order collectors
       //Weight w = s.createNormalizedWeight(q2);
 
-      DrillDownQuery ddq = addDrillDowns(timeStamp, state, r, q);
+      DrillDownQuery ddq = addDrillDowns(timeStamp, state, r, q, dynamicFields);
 
       diagnostics.put("drillDownQuery", q.toString());
 
@@ -1850,7 +1920,7 @@ public class SearchHandler extends Handl
       if (r.hasParam("sort")) {
         sortRequest = r.getStruct("sort");
         sortFieldNames = new ArrayList<String>();
-        sort = parseSort(timeStamp, state, sortRequest.getList("fields"), sortFieldNames);
+        sort = parseSort(timeStamp, state, sortRequest.getList("fields"), sortFieldNames, dynamicFields);
       } else {
         sortRequest = null;
         sort = null;
@@ -1875,7 +1945,7 @@ public class SearchHandler extends Handl
         }
 
         if (grouping.hasParam("sort")) {
-          groupSort = parseSort(timeStamp, state, grouping.getList("sort"), null);
+          groupSort = parseSort(timeStamp, state, grouping.getList("sort"), null, dynamicFields);
         } else {
           groupSort = Sort.RELEVANCE;
         }
@@ -2199,7 +2269,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, sortFieldNames);
+                fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, sort, sortFieldNames, dynamicFields);
               }
 
               hitIndex++;
@@ -2245,7 +2315,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, sortFieldNames);
+              fillFields(state, highlighter, s.searcher, o4, sd, fields, highlights, hitIndex, sort, sortFieldNames, dynamicFields);
             }
             hitIndex++;
 
@@ -2280,7 +2350,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, null);
+                fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, child.sort, null, dynamicFields);
               }
 
               hitIndex++;
@@ -2309,7 +2379,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, sortFieldNames);
+            fillFields(state, highlighter, s.searcher, o4, hit, fields, highlights, hitIndex, sort, sortFieldNames, dynamicFields);
           }
         }
       }

Modified: lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/ServerBaseTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/ServerBaseTestCase.java?rev=1556620&r1=1556619&r2=1556620&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/ServerBaseTestCase.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/ServerBaseTestCase.java Wed Jan  8 19:57:20 2014
@@ -356,6 +356,10 @@ public abstract class ServerBaseTestCase
         String name = path.substring(tokStart, upto-1);
         if (name.length() != 0) {
           o = ((JSONObject) o).get(name);
+          if (o == null) {
+            // Likely a test bug: try to help out:
+            throw new IllegalArgumentException("path " + path.substring(0, tokStart-1) + " does not have member ." + name);
+          }
         }
         tokStart = upto;
         if (ch == '[') {
@@ -370,6 +374,10 @@ public abstract class ServerBaseTestCase
         o = new Integer(((JSONArray) o).size());
       } else {
         o = ((JSONObject) o).get(name);
+        if (o == null) {
+          // Likely a test bug: try to help out:
+          throw new IllegalArgumentException("path " + path.substring(0, tokStart) + " does not have member ." + name);
+        }
       }
     }
     return o;

Modified: 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=1556620&r1=1556619&r2=1556620&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java Wed Jan  8 19:57:20 2014
@@ -178,6 +178,7 @@ public class TestVirtualFields extends S
     deleteAllDocs();
     try {
       send("registerFields", "{fields: {bad: {type: virtual, expression: 'ln(boost'}}}");
+      fail("didn't hit exception");
     } catch (IOException ioe) {
       String message = ioe.toString();
       assertTrue(message.contains("registerFields > fields > bad > expression: could not parse expression"));
@@ -189,6 +190,7 @@ public class TestVirtualFields extends S
     deleteAllDocs();
     try {
       send("registerFields", "{fields: {bad: {type: virtual, expression: 'ln(bad2)'}}}");
+      fail("didn't hit exception");
     } catch (IOException ioe) {
       String message = ioe.toString();
       assertTrue(message.contains("registerFields > fields > bad > expression: could not evaluate expression"));
@@ -200,10 +202,55 @@ public class TestVirtualFields extends S
     deleteAllDocs();
     try {
       send("registerFields", "{fields: {bad2: {type: int, store: true}, bad: {type: virtual, expression: 'ln(bad2)'}}}");
+      fail("didn't hit exception");
     } 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"));
     }
   }
+
+  public void testDynamicFieldSameName() 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");
+
+    // It's an error to try to define a dynamic field name
+    // that already exists:
+    try {
+      send("search", "{queryText: wind, virtualFields: [{name: scoreboost, expression: 2*_score}]}");
+      fail("didn't hit exception");
+    } catch (IOException ioe) {
+      String message = ioe.toString();
+      assertTrue(message.contains("search > virtualFields[0] > name: registered field or dynamic field \"scoreboost\" already exists"));
+    }
+  }
+
+  public void testRetrievedDynamicField() 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, virtualFields: [{name: scoreboost3, expression: 3*scoreboost}], sort: {fields: [{field: id, reverse: true}]}, retrieveFields: [id, scoreboost3], 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(2.41082, getFloat(result, "hits[0].fields.scoreboost3"), .0001f);
+    assertEquals(0.33138, getFloat(result, "hits[1].fields.scoreboost3"), .0001f);
+  }
+
+  public void testSortedDynamicField() 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, virtualFields: [{name: scoreboost3, expression: 3*scoreboost}], sort: {fields: [{field: scoreboost3}]}, 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.33138, getFloat(result, "hits[0].fields.sortFields.scoreboost3"), .0001f);
+    assertEquals(2.41082, getFloat(result, "hits[1].fields.sortFields.scoreboost3"), .0001f);
+  }
 }