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/03/16 19:11:10 UTC

svn commit: r1578133 [10/11] - in /lucene/dev/branches/lucene5376_2/lucene: ./ analysis/common/src/java/org/apache/lucene/analysis/charfilter/ analysis/common/src/java/org/apache/lucene/analysis/pattern/ analysis/common/src/java/org/apache/lucene/analy...

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestPlugins.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestPlugins.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestPlugins.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestPlugins.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,95 @@
+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.File;
+import java.io.FileOutputStream;
+import java.io.InputStream;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONObject;
+
+public class TestPlugins extends ServerBaseTestCase {
+
+  @BeforeClass
+  public static void init() throws Exception {
+    useDefaultIndex = false;
+    File tmpDir = TestUtil.getTempDir("TestPlugins");
+    tmpDir.mkdirs();
+    File zipFile = new File(tmpDir, "MockPlugin-0.1.zip");
+    ZipOutputStream os = new ZipOutputStream(new FileOutputStream(zipFile));
+    addToZip(os, "Mock/org/apache/lucene/server/MockPlugin.class", "MockPlugin.class");
+    addToZip(os, "Mock/lucene-server-plugin.properties", "MockPlugin-lucene-server-plugin.properties");
+    addToZip(os, "Mock/site/hello.txt", "MockPlugin-hello.txt");
+    os.close();
+    installPlugin(zipFile);
+    startServer();
+  }
+
+  static void addToZip(ZipOutputStream dest, String zipName, String resourcePath) throws Exception {
+    ZipEntry file = new ZipEntry(zipName);
+    dest.putNextEntry(file);
+    try (InputStream input = TestPlugins.class.getResourceAsStream(resourcePath)) {
+      byte buf[] = new byte[1024];
+      int numRead;
+      while ((numRead = input.read(buf)) >= 0) {
+        dest.write(buf, 0, numRead);
+      }
+    }
+    dest.closeEntry();
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+  }
+
+  // nocommit this test should install from zip file (call installPlugin(...))
+
+  public void testMockPlugin() throws Exception {
+
+    // Make sure docs reflect new mockFoobar:
+    String doc = httpLoad("doc?method=addDocument");
+    assertTrue(doc.indexOf("<b>mockFoobar</b>") != -1);
+
+    // nocommit test docs: verify foobar is there
+    // nocommit need a "list plugins" API: verify foobar is there
+    // nocommit send addDocument & verify change "took"
+
+    TestUtil.rmDir(new File("index"));
+    send("createIndex", "{rootDir: index}");
+    send("startIndex");
+    send("registerFields", "{fields: {id: {type: int, store: true, postingsFormat: Memory}, intfield: {type: int, store: true}}}");
+    long gen = getLong(send("addDocument", "{fields: {id: 0, mockFoobar: 7}}"), "indexGen");
+
+    JSONObject result = send("search", "{searcher: {indexGen: " + gen + "}, query: MatchAllDocsQuery, retrieveFields: [id, intfield]}");
+    assertEquals(1, getInt(result, "totalHits"));
+    assertEquals(14, getInt(result, "hits[0].fields.intfield"));
+    //System.out.println("got: " + prettyPrint(result));
+  }
+
+  /** Make sure a plugin can have/serve a static file. */
+  public void testStaticFile() throws Exception {
+    String contents = httpLoad("plugins/Mock/hello.txt");
+    assertEquals("hello world!\n", contents);
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSearch.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSearch.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSearch.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,361 @@
+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.File;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Set;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONObject;
+
+public class TestSearch extends ServerBaseTestCase {
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    useDefaultIndex = true;
+    curIndexName = "index";
+    startServer();
+    createAndStartIndex();
+    registerFields();
+    commit();
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+  }
+
+  private static void registerFields() throws Exception {
+    JSONObject o = new JSONObject();
+    put(o, "body", "{type: text, highlight: true, store: true, analyzer: {class: WhitespaceAnalyzer, matchVersion: LUCENE_43}, similarity: {class: BM25Similarity, b: 0.15}}");
+    put(o, "title", "{type: text, highlight: true, store: true, analyzer: {class: WhitespaceAnalyzer, matchVersion: LUCENE_43}, similarity: {class: BM25Similarity, b: 0.15}}");
+    put(o, "id", "{type: int, store: true, sort: true}");
+    JSONObject o2 = new JSONObject();
+    o2.put("fields", o);
+    send("registerFields", o2);
+  }
+
+  public void testPhraseQuery() throws Exception {
+    deleteAllDocs();
+    long gen = getLong(send("addDocument", "{fields: {body: 'the wizard of oz'}}"), "indexGen");
+    JSONObject result = send("search", "{query: {class: PhraseQuery, field: body, terms: [wizard, of, oz]}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(1, getInt(result, "totalHits"));
+
+    result = send("search", "{query: {class: PhraseQuery, field: body, terms: [wizard, oz]}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(0, getInt(result, "totalHits"));
+
+    result = send("search", "{query: {class: PhraseQuery, field: body, terms: [wizard, oz], slop: 1}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(1, getInt(result, "totalHits"));
+  }
+
+  public void testConstantScoreQuery() throws Exception {
+    deleteAllDocs();
+    long gen = getLong(send("addDocument", "{fields: {body: 'the wizard of oz'}}"), "indexGen");
+    JSONObject result = send("search", "{query: {class: TermQuery, field: body, term: wizard}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(1, getInt(result, "totalHits"));
+
+    result = send("search", "{query: {class: ConstantScoreQuery, boost: 10.0, query: {class: TermQuery, field: body, term: wizard}}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(1, getInt(result, "totalHits"));
+    assertEquals(10.0, getFloat(result, "hits[0].score"), .000001f);
+  }
+
+  public void testRegexpQuery() throws Exception {
+    deleteAllDocs();
+    long gen = getLong(send("addDocument", "{fields: {body: 'testing'}}"), "indexGen");
+    JSONObject r = send("search", "{query: {class: RegexpQuery, field: body, regexp: '.*est.*'}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(1, getInt(r, "totalHits"));
+    r = send("search", "{query: {class: RegexpQuery, field: body, regexp: '.*zest.*'}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(0, getInt(r, "totalHits"));
+  }
+
+  public void testTermRangeQuery() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {body: 'terma'}}");
+    send("addDocument", "{fields: {body: 'termb'}}");
+    long gen = getLong(send("addDocument", "{fields: {body: 'termc'}}"), "indexGen");
+
+    JSONObject result = send("search", "{query: {class: TermRangeQuery, field: body, lowerTerm: terma, upperTerm: termc, includeLower: true, includeUpper: true}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(3, getInt(result, "totalHits"));
+    result = send("search", "{query: {class: TermRangeQuery, field: body, lowerTerm: terma, upperTerm: termc, includeLower: false, includeUpper: false}, searcher: {indexGen: " + gen + "}}");
+    assertEquals(1, getInt(result, "totalHits"));
+  }
+
+  public void testMatchAllDocsQuery() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {body: 'terma'}}");
+    send("addDocument", "{fields: {body: 'termb'}}");
+    long gen = getLong(send("addDocument", "{fields: {body: 'termc'}}"), "indexGen");
+    assertEquals(3, getInt(send("search", "{query: {class: MatchAllDocsQuery}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+
+  public void testWildcardQuery() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {body: 'terma'}}");
+    send("addDocument", "{fields: {body: 'termb'}}");
+    long gen = getLong(send("addDocument", "{fields: {body: 'termc'}}"), "indexGen");
+    assertEquals(3, getInt(send("search", "{query: {class: WildcardQuery, field: body, term: 'term?'}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+
+  public void testFuzzyQuery() throws Exception {
+    deleteAllDocs();
+    long gen = getLong(send("addDocument", "{fields: {body: 'fantastic'}}"), "indexGen");
+    assertEquals(1, getInt(send("search", "{query: {class: FuzzyQuery, field: body, term: 'fantasic', maxEdits: 1}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(1, getInt(send("search", "{query: {class: FuzzyQuery, field: body, term: 'fantasic', maxEdits: 2}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(0, getInt(send("search", "{query: {class: FuzzyQuery, field: body, term: 'fantasc', maxEdits: 1}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(1, getInt(send("search", "{query: {class: FuzzyQuery, field: body, term: 'fantasc', maxEdits: 2}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(1, getInt(send("search", "{query: {class: FuzzyQuery, field: body, term: 'fantasc', maxEdits: 2, prefixLength: 4}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+
+  public void testCommonTermsQuery() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {body: 'fantastic'}}");
+    send("addDocument", "{fields: {body: 'fantastic four'}}");
+    long gen = getLong(send("addDocument", "{fields: {body: 'fantastic five'}}"), "indexGen");
+
+    assertEquals(1, getInt(send("search", "{query: {class: CommonTermsQuery, highFreqOccur: must, lowFreqOccur: must, maxTermFrequency: 0.5, field: body, terms: [fantastic, four]}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+
+  public void testMultiPhraseQuery() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {body: 'fantastic five is furious'}}");
+    long gen = getLong(send("addDocument", "{fields: {body: 'fantastic four is furious'}}"), "indexGen");
+
+    assertEquals(1, getInt(send("search", "{query: {class: MultiPhraseQuery, field: body, terms: [fantastic, five, is, furious]}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(2, getInt(send("search", "{query: {class: MultiPhraseQuery, field: body, terms: [fantastic, {term: furious, position: 3}]}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(2, getInt(send("search", "{query: {class: MultiPhraseQuery, field: body, terms: [fantastic, [five, four], {term: furious, position: 3}]}, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+
+  public void testClassicQPDefaultOperator() throws Exception {
+    deleteAllDocs();
+    long gen = getLong(send("addDocument", "{fields: {body: 'fantastic four is furious'}}"), "indexGen");
+    
+    assertEquals(1, getInt(send("search", "{queryParser: {class: classic, defaultOperator: or, defaultField: body}, queryText: 'furious five', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(0, getInt(send("search", "{queryParser: {class: classic, defaultOperator: and, defaultField: body}, queryText: 'furious five', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+
+  public void testMultiFieldQueryParser() throws Exception {
+    deleteAllDocs();
+    long gen = getLong(send("addDocument", "{fields: {body: 'fantastic four is furious', title: 'here is the title'}}"), "indexGen");
+    
+    assertEquals(1, getInt(send("search", "{queryParser: {class: MultiFieldQueryParser, defaultOperator: or, fields: [body, {field: title, boost: 2.0}]}, queryText: 'title furious', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(1, getInt(send("search", "{queryParser: {class: MultiFieldQueryParser, defaultOperator: and, fields: [body, {field: title, boost: 2.0}]}, queryText: 'title furious', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+
+  public void testSimpleQueryParser() throws Exception {
+    deleteAllDocs();
+    long gen = getLong(send("addDocument", "{fields: {body: 'fantastic four is furious', title: 'here is the title'}}"), "indexGen");
+    
+    assertEquals(1, getInt(send("search", "{queryParser: {class: SimpleQueryParser, defaultOperator: or, fields: [body, title], operators: [WHITESPACE]}, queryText: 'title furious', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(1, getInt(send("search", "{queryParser: {class: SimpleQueryParser, defaultOperator: and, fields: [body, title], operators: [WHITESPACE]}, queryText: 'title furious', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(0, getInt(send("search", "{queryParser: {class: SimpleQueryParser, defaultOperator: or, fields: [body, title], operators: [WHITESPACE]}, queryText: 't* f*', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(1, getInt(send("search", "{queryParser: {class: SimpleQueryParser, defaultOperator: or, fields: [body, title], operators: [WHITESPACE, PREFIX]}, queryText: 't* f*', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(0, getInt(send("search", "{queryParser: {class: SimpleQueryParser, defaultOperator: or, fields: [body, title], operators: [WHITESPACE]}, queryText: '\"furious title\"', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(0, getInt(send("search", "{queryParser: {class: SimpleQueryParser, defaultOperator: or, fields: [body, title], operators: [WHITESPACE, PHRASE]}, queryText: '\"furious title\"', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+    assertEquals(1, getInt(send("search", "{queryParser: {class: SimpleQueryParser, defaultOperator: or, fields: [body, title], operators: [WHITESPACE, PHRASE]}, queryText: '\"fantastic four\"', searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+
+  public void testNumericRangeQuery() throws Exception {
+    curIndexName = "nrq";
+    for(String type : new String[] {"int", "long", "float", "double"}) {
+      TestUtil.rmDir(new File("nrq"));
+      send("createIndex", "{rootDir: nrq}");
+      send("startIndex");
+      send("registerFields", String.format(Locale.ROOT, "{fields: {nf: {type: %s, search: true}}}", type));
+      send("addDocument", "{fields: {nf: 5}}");
+      send("addDocument", "{fields: {nf: 10}}");
+      long gen = getLong(send("addDocument", "{fields: {nf: 17}}"), "indexGen");
+
+      // Both min & max:
+      assertEquals(3, getInt(send("search",
+
+                                  String.format(Locale.ROOT, "{query: {class: NumericRangeQuery, field: nf, min: 5, max: 17, minInclusive: true, maxInclusive: true}, searcher: {indexGen: %d}}", gen)),
+                             "totalHits"));
+
+      // Leave min out:
+      assertEquals(3, getInt(send("search",
+                                  String.format(Locale.ROOT, "{query: {class: NumericRangeQuery, field: nf, max: 17, maxInclusive: true}, searcher: {indexGen: %d}}", gen)),
+                             "totalHits"));
+
+      // Leave min out, don't include max:
+      assertEquals(2, getInt(send("search",
+                                  String.format(Locale.ROOT, "{query: {class: NumericRangeQuery, field: nf, max: 17, maxInclusive: false}, searcher: {indexGen: %d}}", gen)),
+                             "totalHits"));
+
+      // Leave max out:
+      assertEquals(3, getInt(send("search",
+                                  String.format(Locale.ROOT, "{query: {class: NumericRangeQuery, field: nf, min: 5, minInclusive: true}, searcher: {indexGen: %d}}", gen)),
+                             "totalHits"));
+
+      // Leave max out, don't include max:
+      assertEquals(2, getInt(send("search",
+                                  String.format(Locale.ROOT, "{query: {class: NumericRangeQuery, field: nf, min: 5, minInclusive: false}, searcher: {indexGen: %d}}", gen)),
+                             "totalHits"));
+      send("stopIndex");
+      send("deleteIndex");
+    }
+  }
+
+  public void testSearchAfter() throws Exception {
+    deleteAllDocs();
+    long gen = 0;
+    for(int i=0;i<20;i++) {
+      gen = getLong(send("addDocument", "{fields: {body: 'this is the body', id: " + i + "}}"), "indexGen");
+    }
+
+    JSONObject lastPage = null;
+
+    Set<Integer> seenIDs = new HashSet<Integer>();
+
+    // Pull 4 pages with 5 hits per page:
+    for(int i=0;i<4;i++) {
+      String sa;
+      if (lastPage != null) {
+        sa = ", searchAfter: {lastDoc: " + getInt(lastPage, "searchState.lastDoc") + ", lastScore: " + getFloat(lastPage, "searchState.lastScore") + "}";
+      } else {
+        sa = "";
+      }
+
+      lastPage = send("search", "{query: MatchAllDocsQuery, topHits: 5, retrieveFields: [id], searcher: {indexGen: " + gen + "}" + sa + "}");
+      //System.out.println("i=" + i + ": " + lastPage);
+
+      // 20 total hits
+      assertEquals(20, getInt(lastPage, "totalHits"));
+      assertEquals(5, getInt(lastPage, "hits.length"));
+      for(int j=0;j<5;j++) {
+        seenIDs.add(getInt(lastPage, "hits[" + j + "].fields.id"));
+      }
+    }
+
+    assertEquals(20, seenIDs.size());
+  }
+
+  public void testSearchWithTimeout() throws Exception {
+    deleteAllDocs();
+    LineFileDocs docs = new LineFileDocs(random());
+    long charCountLimit = atLeast(10*1024);
+    long charCount = 0;
+    int id = 0;
+    while (charCount < charCountLimit) {
+      Document doc = docs.nextDoc();
+      charCount += doc.get("body").length() + doc.get("titleTokenized").length();
+      JSONObject fields = new JSONObject();
+      fields.put("title", doc.get("titleTokenized"));
+      fields.put("body", doc.get("body"));
+      fields.put("id", id++);
+      JSONObject o = new JSONObject();
+      o.put("fields", fields);
+      send("addDocument", o);
+    }
+
+    assertFailsWith("search",
+                    "{queryText: the, retrieveFields: [id], timeoutSec: 0.0}",
+                    "search > timeoutSec: must be > 0 msec");
+
+    // NOTE: does not actually test that we hit the timeout;
+    // just that we can specify timeoutSec
+    send("search", "{queryText: the, retrieveFields: [id], timeoutSec: 0.001}");
+  }
+
+  public void testSearchAfterWithSort() throws Exception {
+    deleteAllDocs();
+    long gen = 0;
+    for(int i=0;i<20;i++) {
+      gen = getLong(send("addDocument", "{fields: {body: 'this is the body', id: " + i + "}}"), "indexGen");
+    }
+
+    JSONObject lastPage = null;
+
+    Set<Integer> seenIDs = new HashSet<Integer>();
+
+    // Pull 4 pages with 5 hits per page:
+    for(int i=0;i<4;i++) {
+      String sa;
+      JSONObject o = new JSONObject();
+      o.put("query", "MatchAllDocsQuery");
+      o.put("topHits", 5);
+      put(o, "retrieveFields", "[id]");
+      put(o, "sort", "{fields: [{field: id}]}");
+      put(o, "searcher", "{indexGen: " + gen + "}");
+      if (lastPage != null) {
+        JSONObject o2 = new JSONObject();
+        o.put("searchAfter", o2);
+        o2.put("lastDoc", getInt(lastPage, "searchState.lastDoc"));
+        o2.put("lastFieldValues", get(lastPage, "searchState.lastFieldValues"));
+      } else {
+        sa = "";
+      }
+
+      lastPage = send("search", o);
+
+      // 20 total hits
+      assertEquals(20, getInt(lastPage, "totalHits"));
+      assertEquals(5, getInt(lastPage, "hits.length"));
+      for(int j=0;j<5;j++) {
+        seenIDs.add(getInt(lastPage, "hits[" + j + "].fields.id"));
+      }
+    }
+
+    assertEquals(20, seenIDs.size());
+  }
+
+  public void testRecencyBlendedSort() throws Exception {
+    curIndexName = "recency";
+    File dir = new File(TestUtil.getTempDir("recency"), "root");
+    send("createIndex", "{rootDir: " + dir.getAbsolutePath() + "}");
+    send("startIndex");
+    send("registerFields", "{fields: {timestamp: {type: long, search: false, sort: true}, body: {type: text, analyzer: StandardAnalyzer}}}");
+
+    long t = System.currentTimeMillis()/1000;
+    send("addDocument", "{fields: {body: 'this is some text', timestamp: " + (t-100) + "}}");
+    long gen = getLong(send("addDocument", "{fields: {body: 'this is some text', timestamp: " + t + "}}"), "indexGen");
+
+    for(int pass=0;pass<2;pass++) {
+      // Unboosted:
+      JSONObject result = send("search", "{queryText: text, searcher: {indexGen: " + gen + "}}");
+      assertEquals(2, getInt(result, "totalHits"));
+      assertEquals(0, getInt(result, "hits[0].doc"));
+      assertEquals(1, getInt(result, "hits[1].doc"));
+
+      // Blended relevance + recency changes the order:
+      t = System.currentTimeMillis()/1000;
+      // nocommit this isn't right: the boost gets down to 0
+      // when it's "recent"
+      result = send("search",
+                    "{queryText: text, virtualFields: [" + 
+                    "{name: age,   expression: '" + t + " - timestamp'}, " + 
+                    "{name: boost, expression: '(age >= 30) ? 1.0 : (2.0 * (30. - age) / 30)'}, " +
+                    "{name: blend, expression: 'boost * _score'}], " + 
+                    " sort: {fields: [{field: blend, reverse: true}]}, retrieveFields: [age, boost], searcher: {indexGen: " + gen + "}}");
+      assertEquals(2, getInt(result, "totalHits"));
+      assertEquals(1, getInt(result, "hits[0].doc"));
+      assertEquals(0, getInt(result, "hits[1].doc"));
+      assertTrue(getFloat(result, "hits[0].fields.boost") > 1.0f);
+      assertEquals(1.0, getFloat(result, "hits[1].fields.boost"), 0.0001f);
+
+      // Make sure this survives restart:
+      send("stopIndex");
+      send("startIndex");
+    }
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestServer.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestServer.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestServer.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,397 @@
+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 java.util.Locale;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+public class TestServer extends ServerBaseTestCase {
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    useDefaultIndex = true;
+    curIndexName = "index";
+    curIndexName = "index";
+    startServer();
+    createAndStartIndex();
+    registerFields();
+    commit();
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+  }
+
+  private static void registerFields() throws Exception {
+    JSONObject o = new JSONObject();
+    put(o, "body", "{type: text, highlight: true, store: true, analyzer: {class: StandardAnalyzer, matchVersion: LUCENE_43}, similarity: {class: BM25Similarity, b: 0.15}}");
+    put(o, "id", "{type: int, store: true, postingsFormat: Memory}");
+    put(o, "price", "{type: float, sort: true, search: true, store: true}");
+    put(o, "date", "{type: atom, search: false, store: true}");
+    put(o, "dateFacet", "{type: atom, search: false, store: false, facet: hierarchy}");
+    put(o, "author", "{type: text, search: false, facet: flat, group: true}");
+    JSONObject o2 = new JSONObject();
+    o2.put("indexName", "index");
+    o2.put("fields", o);
+    send("registerFields", o2);
+  }
+
+  // nocommit multi-valued field
+
+  // Returns gen for the added document
+  private long addDocument(int id, String author, String body, float price, String date) throws Exception {
+    JSONObject o = new JSONObject();
+    o.put("body", body);
+    o.put("author", author);
+    o.put("price", price);
+    o.put("id", id);
+    o.put("date", date);
+    JSONArray path = new JSONArray();
+    o.put("dateFacet", path);
+    for(String part : date.split("/")) {
+      path.add(part);
+    }
+
+    JSONObject o2 = new JSONObject();
+    o2.put("fields", o);
+    o2.put("indexName", "index");
+
+    JSONObject result = send("addDocument", o2);
+    return ((Number) result.get("indexGen")).longValue();
+  }
+
+  // nocommit add test making sure we catch extra unused params
+
+  private JSONObject search(String body) throws Exception {
+    return search(body, -1, null, false, true, null, null);
+  }
+
+  private JSONObject search(String query, long indexGen, String sortField, boolean reversed, boolean snippets, String groupField, String groupSortField) throws Exception {
+    JSONObject o = new JSONObject();
+    o.put("indexName", "index");
+    o.put("queryText", query);
+    if (indexGen != -1) {
+      JSONObject o2 = new JSONObject();
+      o.put("searcher", o2);
+      o2.put("indexGen", indexGen);
+    }
+
+    if (sortField != null) {
+      JSONObject sort = new JSONObject();
+      o.put("sort", sort);
+      sort.put("doDocScores", true);
+
+      JSONArray sortFields = new JSONArray();
+      sort.put("fields", sortFields);
+
+      JSONObject o2 = new JSONObject();
+      sortFields.add(o2);
+
+      o2.put("field", sortField);
+      o2.put("reverse", reversed);
+    }
+
+    if (groupField != null) {
+      String s = "{field: '" + groupField + "'";
+      if (groupSortField != null) {
+        s += ", sort: [{field: '" + groupSortField + "'}]";
+      }
+      s += "}";
+      put(o, "grouping", s);
+    }
+
+    put(o, "facets", "[{dim: dateFacet, topN: 10}]");
+    put(o, "retrieveFields", "[id, date, price, {field: body, highlight: " + (snippets ? "snippets" : "whole") + "}]");
+
+    return send("search", o);
+  }
+
+  public void testBasic() throws Exception {
+    deleteAllDocs();
+    long gen = addDocument(0, "Bob", "this is a test", 10f, "2012/10/17");
+    JSONObject o = search("test", gen, null, false, true, null, null);
+    assertEquals(1, getInt(o, "totalHits"));
+  }
+
+  public void testNumericSort() throws Exception {
+    deleteAllDocs();
+    addDocument(0, "Lisa", "this is a test", 10.99f, "2012/10/1");
+    long gen = addDocument(1, "Tom", "this is also a test", 14.99f, "2012/11/3");
+    JSONObject o = search("test", gen, "price", false, true, null, null);
+    assertEquals(2, ((Number) o.get("totalHits")).intValue());
+    JSONArray hits = (JSONArray) o.get("hits");
+    assertEquals(2, hits.size());
+
+    JSONObject hit = (JSONObject) hits.get(0);
+    assertEquals(0, ((JSONObject) hit.get("fields")).get("id"));
+
+    hit = (JSONObject) hits.get(1);
+    assertEquals(1, ((JSONObject) hit.get("fields")).get("id"));
+  }
+
+  public void testReverseNumericSort() throws Exception {
+    deleteAllDocs();
+    addDocument(0, "Frank", "this is a test", 10.99f, "2012/10/1");
+    long gen = addDocument(1, "Lisa", "this is also a test", 14.99f, "2012/11/3");
+    JSONObject o = search("test", gen, "price", true, true, null, null);
+    assertEquals(2, ((Number) o.get("totalHits")).intValue());
+
+    JSONArray hits = (JSONArray) o.get("hits");
+    assertEquals(2, hits.size());
+
+    JSONObject hit = (JSONObject) hits.get(0);
+    assertEquals(1, ((JSONObject) hit.get("fields")).get("id"));
+
+    hit = (JSONObject) hits.get(1);
+    assertEquals(0, ((JSONObject) hit.get("fields")).get("id"));
+  }
+
+  public void testPrevSearchState() throws Exception {
+    deleteAllDocs();
+    long gen = addDocument(0, "Tom", "this is a test.  here is a random sentence.  here is another sentence with test in it.", 10.99f, "2012/10/17");
+
+    JSONObject o = search("test", gen, null, false, false, null, null);
+    assertEquals(1, ((Number) o.get("totalHits")).intValue());
+
+    // Add another document
+    gen = addDocument(0, "Melanie", "this is a test.  here is a random sentence.  here is another sentence with test in it.", 10.99f, "2012/10/17");
+
+    JSONObject o2 = search("test", gen, null, false, false, null, null);
+    assertEquals(2, ((Number) o2.get("totalHits")).intValue());
+
+    // Now the first search does a follow-on search, so we
+    // should only see 1 document since it should be using
+    // the old searcher:
+    JSONObject o3 = new JSONObject();
+    o3.put("indexName", "index");
+    o3.put("queryText", "test");
+    put(o3, "searcher", "{version: " + get(o, "searchState.searcher") + "}");
+    //System.out.println("send: " + o3);
+    JSONObject o4 = send("search", o3);
+
+    assertEquals(1, ((Number) o4.get("totalHits")).intValue());
+  }
+
+  public void testInvalidFields() throws Exception {
+    deleteAllDocs();
+    addDocument(0, "Lisa", "this is a test.  here is a random sentence.  here is another sentence with test in it.", 10.99f, "2012/10/17");
+
+    JSONObject o3 = new JSONObject();
+    o3.put("queryText", "test");
+    JSONArray fields = new JSONArray();
+    o3.put("retrieveFields", fields);
+    fields.add("bogus");
+    try {
+      send("search", o3);
+      fail("did not hit exception");
+    } catch (IOException e) {
+      // expected
+    }
+
+    o3 = new JSONObject();
+    o3.put("queryText", "test");
+    JSONObject sort = new JSONObject();
+    o3.put("sort", sort);
+    JSONArray sortFields = new JSONArray();
+    sort.put("fields", sortFields);
+    
+    JSONObject sortField = new JSONObject();
+    sortFields.add(sortField);
+    sortField.put("field", "bogus2");
+    try {
+      send("search", o3);
+      fail("did not hit exception");
+    } catch (IOException e) {
+      // expected
+    }
+  }
+
+  public void testInvalidSearcherVersion() throws Exception {
+    deleteAllDocs();
+
+    JSONObject o3 = new JSONObject();
+    o3.put("queryText", "test");
+    JSONObject searchState = new JSONObject();
+    o3.put("priorSearchState", searchState);
+    searchState.put("searcher", 0);
+    searchState.put("lastDocID", 0);
+    try {
+      send("search", o3);
+      fail("didn't hit exception");
+    } catch (IOException e) {
+      // expected
+      //e.printStackTrace(System.out);
+    }
+  }
+
+  public void testMultiValuedString() throws Exception {
+    deleteAllDocs();
+
+    send("registerFields", "{fields: {authors: {type: text, search: true, store: true, facet: flat, multiValued: true, analyzer: {matchVersion: LUCENE_43, class: StandardAnalyzer}}}}");
+
+    JSONObject result = send("addDocument", "{fields: {authors: [Bob, Lisa]}}");
+
+    long indexGen = getInt(result, "indexGen");
+
+    result = send("search", "{searcher: {indexGen: " + indexGen + "}, queryText: 'authors:bob', retrieveFields: [authors]}");
+
+    assertEquals(1, getInt(result, "totalHits"));
+    assertEquals("[\"Bob\",\"Lisa\"]", getArray(result, "hits[0].fields.authors").toString());
+  }
+
+  public void testMultiValuedNumeric() throws Exception {
+    deleteAllDocs();
+
+    send("registerFields", "{fields: {ratings: {type: int, search: true, store: true, multiValued: true}}}");
+
+    JSONObject result = send("addDocument", "{fields: {body: 'here is a test', ratings: [17, 22]}}");
+
+    long indexGen = getInt(result, "indexGen");
+
+    result = send("search", "{searcher: {indexGen: " + indexGen + "}, queryText: 'body:test', retrieveFields: [ratings]}");
+
+    assertEquals(1, getInt(result, "totalHits"));
+    assertEquals("[17,22]", getArray(result, "hits[0].fields.ratings").toString());
+  }
+
+  public void testStandardAnalyzer() throws Exception {
+    deleteAllDocs();
+
+    send("registerFields", "{fields: {aTextField: {type: text, analyzer: {class: StandardAnalyzer, matchVersion: LUCENE_43}, store: true}}}");
+
+    JSONObject result = send("addDocument", "{fields: {aTextField: 'here is a test'}}");
+    long indexGen = getInt(result, "indexGen");
+
+    // nocommit: grrr need QP to understand schema
+    //o.put("queryText", "ratings:[16 TO 18]");
+
+    // search on a stop word should yield no results:
+    result = send("search", String.format(Locale.ROOT, "{searcher: {indexGen: %d}, queryText: 'aTextField:a'}", indexGen));
+    assertEquals(0, getInt(result, "totalHits"));
+  }
+
+  public void testStandardAnalyzerNoStopWords() throws Exception {
+    deleteAllDocs();
+
+    send("registerFields", "{fields: {aTextField2: {type: text, search: true, store: true, analyzer: {class: StandardAnalyzer, matchVersion: LUCENE_43, stopWords: []}}}}");
+
+    JSONObject result = send("addDocument", "{fields: {aTextField2: 'here is a test'}}");
+    long indexGen = getLong(result, "indexGen");
+
+    // nocommit: grrr need QP to understand schema
+    //o.put("queryText", "ratings:[16 TO 18]");
+
+    // search on a stop word should now yield one hit:
+    result = send("search", "{queryText: 'aTextField2:a', searcher: {indexGen: " + indexGen + "}}");
+    assertEquals(1, getInt(result, "totalHits"));
+  }
+
+  public void testEnglishAnalyzerNoStopWords() throws Exception {
+    deleteAllDocs();
+
+    send("registerFields", "{fields: {aTextField3: {type: text, search: true, store: true, analyzer: {class: EnglishAnalyzer, matchVersion: LUCENE_43, stopWords: []}}}}");
+    JSONObject result = send("addDocument", "{fields: {aTextField3: 'the cats in the hat'}}");
+    long indexGen = getLong(result, "indexGen");
+
+    // nocommit: grrr need QP to understand schema
+    //o.put("queryText", "ratings:[16 TO 18]");
+
+    // cats should stem to cat and get a match:
+    result = send("search", "{queryText: 'aTextField3:cat', searcher: {indexGen: " + indexGen + "}}");
+    assertEquals(1, getInt(result, "totalHits"));
+  }
+
+  public void testInvalidFieldName() throws Exception {
+    JSONObject o = new JSONObject();
+    JSONObject o2 = new JSONObject();
+    o.put("9", o2);
+    o2.put("type", "text");
+    try {
+      send("registerFields", o);
+      fail();
+    } catch (IOException ioe) {
+      // expected
+    }
+  }
+
+  public void testMoreThanOneValueOnSingleValuedField() throws Exception {
+    deleteAllDocs();
+    JSONObject o = new JSONObject();
+    JSONArray arr = new JSONArray();
+    o.put("author", arr);
+    arr.add("Author 1");
+    arr.add("Author 2");
+
+    try {
+      send("addDocument", o);
+      fail("expected exception");
+    } catch (IOException ioe) {
+      // expected
+    }
+  }
+
+  public void testServerRestart() throws Exception {
+    deleteAllDocs();
+    addDocument(0, "Bob", "this is a test", 10f, "2012/10/17");
+    send("commit");
+    shutdownServer();
+    startServer();
+    send("startIndex");
+    JSONObject o = search("test", 0, null, false, true, null, null);
+    assertEquals(1, ((Number) o.get("totalHits")).intValue());
+  }
+
+  public void testStatsHandler() throws Exception {
+    JSONObject result = send("stats");
+    //System.out.println("GOT: " + result);
+  }
+
+  public void testStuffAfterJSON() throws Exception {
+    // Extra whitespace should be OK:
+    sendRaw("stats", "{\"indexName\": \"index\"}  ");
+    
+    // ... but this should not:
+    try {
+      sendRaw("stats", "{\"indexName\": \"index\"}  bogus");
+      fail("did not hit exception");
+    } catch (IOException ioe) {
+      // expected
+      assertTrue(ioe.toString().indexOf("could not parse HTTP request data as JSON") != -1);
+    }
+  }
+
+  // nocommit assert that exact field name w/ error is in
+  // error message
+
+  // nocommit drillDowns test, single and multi valued
+
+  // nocommit testDocs
+
+  // nocommit need test case that screws up adding bulk docs
+  // (eg category path with empty string component) and
+  // verifies the error "comes through"
+
+  // nocommit need stress test that makes some index
+  // thousands of times to make sure nothing leaks...
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSettings.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSettings.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSettings.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSettings.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,59 @@
+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 org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONObject;
+
+public class TestSettings extends ServerBaseTestCase {
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    useDefaultIndex = true;
+    curIndexName = "index";
+    startServer();
+    createAndStartIndex();
+    registerFields();
+    commit();
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+  }
+
+  private static void registerFields() throws Exception {
+    send("registerFields", "{fields: {body: {type: text, analyzer: StandardAnalyzer}}}");
+  }
+
+  public void testNRTCachingDirSettings() throws Exception {
+    deleteAllDocs();
+    commit();
+    send("stopIndex");
+    JSONObject o = send("settings");
+    assertEquals(0, o.size());
+    // Turn off NRTCachingDir:
+    send("settings", "{nrtCachingDirectory.maxMergeSizeMB: 0.0, nrtCachingDirectory.maxSizeMB: 0.0}");
+    o = send("settings");
+    assertEquals(2, o.size());
+    send("startIndex");
+    long gen = getLong(send("addDocument", "{fields: {body: 'here is a test'}}"), "indexGen");
+    assertEquals(1, getInt(send("search", "{queryText: test, searcher: {indexGen: " + gen + "}}"), "totalHits"));
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSimpleSpatial.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSimpleSpatial.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSimpleSpatial.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSimpleSpatial.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,95 @@
+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.File;
+import java.util.Locale;
+
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+/** Simple example of how to do distance drill-downs. */
+public class TestSimpleSpatial extends ServerBaseTestCase {
+
+  @BeforeClass
+  public static void init() throws Exception {
+    useDefaultIndex = false;
+    curIndexName = "spatial";
+    startServer();
+    createAndStartIndex();
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+  }
+
+  public void test() throws Exception {
+
+    // Record latitude and longitude as double fields:
+    send("registerFields",
+         "{fields: {latitude: {type: double, storeDocValues: true}, longitude: {type: double, storeDocValues: true}}}");
+
+    // Index 3 documents each with its own location:
+    send("addDocument", "{fields: {latitude: 40.759011, longitude: -73.9844722}}");
+    send("addDocument", "{fields: {latitude: 40.718266, longitude: -74.007819}}");
+    send("addDocument", "{fields: {latitude: 40.7051157, longitude: -74.0088305}}");
+
+    // Search, matching all documents and counting distance
+    // facets from the home origin:
+    double homeLatitude = 40.7143528;
+    double homeLongitude = -74.0059731;
+    send("search",
+         "{query: MatchAllDocsQuery, " +
+         "virtualFields: [{name: distance, expression: 'haversin(" + homeLatitude + "," + homeLongitude + ",latitude,longitude)'}], " +
+         "facets: [{dim: distance, numericRanges: [" + 
+         "{label: '< 1 km', min: 0.0, minInclusive: true, max: 1.0, maxInclusive: false}," +
+         "{label: '< 2 km', min: 0.0, minInclusive: true, max: 2.0, maxInclusive: false}," +
+         "{label: '< 5 km', min: 0.0, minInclusive: true, max: 5.0, maxInclusive: false}," +
+         "{label: '< 10 km', min: 0.0, minInclusive: true, max: 10.0, maxInclusive: false}," +
+         "]}]}");
+
+    // Search, matching all documents and counting distance
+    // facets from the home origin:
+    assertEquals(3, getInt("totalHits"));
+    assertEquals("top: 3, < 1 km: 1, < 2 km: 2, < 5 km: 2, < 10 km: 3",
+                 TestFacets.formatFacetCounts(getObject("facets[0]")));
+
+    // Now drill-down on '< 2 KM':
+    send("search",
+         "{query: MatchAllDocsQuery, " +
+         "drillDowns: [{field: distance, numericRange: {label: '< 2 KM', min: 0.0, minInclusive: true, max: 2.0, maxInclusive: false}}], " +
+         "virtualFields: [{name: distance, expression: 'haversin(" + homeLatitude + "," + homeLongitude + ",latitude,longitude)'}], " +
+         "facets: [{dim: distance, numericRanges: [" + 
+         "{label: '< 1 km', min: 0.0, minInclusive: true, max: 1.0, maxInclusive: false}," +
+         "{label: '< 2 km', min: 0.0, minInclusive: true, max: 2.0, maxInclusive: false}," +
+         "{label: '< 5 km', min: 0.0, minInclusive: true, max: 5.0, maxInclusive: false}," +
+         "{label: '< 10 km', min: 0.0, minInclusive: true, max: 10.0, maxInclusive: false}," +
+         "]}]}");
+
+    assertEquals(2, getInt("totalHits"));
+
+    // Drill-sideways counts are unchanged after a single drill-down:
+    assertEquals("top: 3, < 1 km: 1, < 2 km: 2, < 5 km: 2, < 10 km: 3",
+                 TestFacets.formatFacetCounts(getObject("facets[0]")));
+  }
+}
+

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSnapshots.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSnapshots.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSnapshots.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSnapshots.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,172 @@
+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.File;
+import java.util.Map;
+
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+public class TestSnapshots extends ServerBaseTestCase {
+  
+  @BeforeClass
+  public static void initClass() throws Exception {
+    useDefaultIndex = true;
+    curIndexName = "index";
+    startServer();
+    createAndStartIndex();
+    registerFields();
+    commit();
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+  }
+
+  private static void registerFields() throws Exception {
+    send("registerFields", "{fields: {body: {type: text, analyzer: {class: EnglishAnalyzer, matchVersion: LUCENE_43}}, id: {type: atom, store: true}, facet: {type: atom, search: false, facet: flat}}}");
+  }
+
+  public void testBasic() throws Exception {
+    deleteAllDocs();
+
+    // Add one doc:
+    JSONObject o = send("addDocument", "{fields: {body: 'here is the body', id: '0', facet: 'facet'}}");
+    long indexGen = ((Number) o.get("indexGen")).longValue();
+    commit();
+
+    o = send("search", "{queryText: 'body:body', searcher: {indexGen:" + indexGen + "}}");
+    assertEquals(1, o.get("totalHits"));
+
+    // Take snapshot before making some changes:
+    JSONObject result = send("createSnapshot");
+    String snapshotID = getString(result, "id");
+
+    // Delete first doc, register new field, add two more docs:
+    send("deleteDocuments", "{field: id, values: ['0']}");
+    send("registerFields", "{fields: {field: {type: 'atom'}}}");
+    send("addDocument", "{fields: {body: 'here is the body', id: '1', facet: 'facet2', field: 'abc'}}");
+    send("addDocument", "{fields: {body: 'here is the body', id: '2', facet: 'facet2', field: 'abc'}}");
+    long indexGen2 = getLong("indexGen");
+    commit();
+
+    File backupDir = TestUtil.getTempDir("backup");
+
+    // Make sure all files in the snapshot still exist, even
+    // though we deleted that segment, and make a backup:
+    try {
+      for(Map.Entry<String,Object> ent : result.entrySet()) {
+        if (ent.getKey().equals("id")) {
+          continue;
+        }
+        File dirPath = new File(curIndexName, ent.getKey());
+        File destDir = new File(backupDir, ent.getKey());
+        destDir.mkdirs();
+        for (Object sub : ((JSONArray) ent.getValue())) {
+          String fileName = (String) sub;
+          File sourceFile = new File(dirPath, fileName);
+          assertTrue("file " + sourceFile + " does not exist", sourceFile.exists());
+          copyFile(sourceFile, new File(destDir, fileName));
+          //System.out.println("copied to " + new File(destDir, fileName));
+        }
+      }
+
+      // Make sure we can search the snapshot and only get 1 hit:
+      send("search", "{retrieveFields: [id], searcher: {snapshot: \"" + snapshotID + "\"}, query: MatchAllDocsQuery}");
+      assertEquals(1, getInt("totalHits"));
+      assertEquals("0", getString("hits[0].fields.id"));
+
+      // Make sure we can search the current searcher and we
+      // get 2 hits:
+      send("search", "{retrieveFields: [id], searcher: {indexGen: " + indexGen2 + "}, query: MatchAllDocsQuery}");
+      assertEquals(2, getInt("totalHits"));
+
+      // Bounce the server:
+      shutdownServer();
+      startServer();
+      send("startIndex");
+
+      // Make sure we can search the snapshot and still only get 1 hit:
+      send("search", "{retrieveFields: [id], searcher: {snapshot: \"" + snapshotID + "\"}, query: MatchAllDocsQuery}");
+      assertEquals(1, getInt("totalHits"));
+      assertEquals("0", getString("hits[0].fields.id"));
+
+      // Make sure we can search the current searcher and we
+      // get 2 hits:
+      send("search", "{retrieveFields: [id], query: MatchAllDocsQuery}");
+      assertEquals(2, getInt("totalHits"));
+
+      // Make sure files still exist (snapshot persisted):
+      for(Map.Entry<String,Object> ent : result.entrySet()) {
+        if (ent.getKey().equals("id")) {
+          continue;
+        }
+        File dirPath = new File(curIndexName, ent.getKey());
+        for (Object sub : ((JSONArray) ent.getValue())) {
+          String fileName = (String) sub;
+          File sourceFile = new File(dirPath, fileName);
+          assertTrue(sourceFile.exists());
+        }
+      }
+
+      // Make sure we can still search the snapshot:
+      send("search", "{retrieveFields: [id], searcher: {snapshot: \"" + snapshotID + "\"}, query: MatchAllDocsQuery}");
+      assertEquals(1, getInt("totalHits"));
+      assertEquals("0", getString("hits[0].fields.id"));
+
+      // Now, release the snapshot:
+      send("releaseSnapshot", "{id: \"" + snapshotID + "\"}");
+
+      // Make sure some files in the snapshot are now gone:
+      boolean someGone = false;
+      for(Map.Entry<String,Object> ent : result.entrySet()) {
+        if (ent.getKey().equals("id")) {
+          continue;
+        }
+        String dirPath = ent.getKey();
+        for (Object sub : ((JSONArray) ent.getValue())) {
+          String fileName = (String) sub;
+          if (!(new File(dirPath, fileName)).exists()) {
+            someGone = true;
+          }
+        }
+      }
+      assertTrue(someGone);
+
+      // Restart server against the backup image:
+      shutdownServer();
+      startServer();
+      send("startIndex");
+
+      // Make sure search is working, and both docs are visible:
+      o = send("search", "{queryText: 'body:body', searcher: {indexGen:" + indexGen + "}}");
+      assertEquals(2, o.get("totalHits"));
+
+    } finally {
+      TestUtil.rmDir(backupDir);
+    }
+  }
+
+  // TODO: threaded test, taking snapshot while threads are
+  // adding/deleting/committing
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSort.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSort.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSort.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,150 @@
+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.File;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Set;
+
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONObject;
+
+public class TestSort extends ServerBaseTestCase {
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    useDefaultIndex = true;
+    curIndexName = "index";
+    startServer();
+    createAndStartIndex();
+    registerFields();
+    commit();
+  }
+
+  private static void registerFields() throws Exception {
+    JSONObject o = new JSONObject();
+    put(o, "atom", "{type: atom, sort: true}");
+    put(o, "int", "{type: int, sort: true}");
+    put(o, "float", "{type: float, sort: true}");
+    put(o, "long", "{type: long, sort: true}");
+    put(o, "double", "{type: double, sort: true}");
+    put(o, "text", "{type: text, analyzer: WhitespaceAnalyzer}");
+    put(o, "id", "{type: int, store: true}");
+    JSONObject o2 = new JSONObject();
+    o2.put("fields", o);
+    send("registerFields", o2);
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+  }
+
+  public void testMissingLastAtom() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, atom: a}}");
+    send("addDocument", "{fields: {id: 1, atom: b}}");
+    // field is missing:
+    send("addDocument", "{fields: {id: 2}}");
+    long gen = getLong("indexGen");
+
+    verifySort("atom");
+  }
+
+  public void testMissingLastInt() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, int: -7}}");
+    send("addDocument", "{fields: {id: 1, int: 7}}");
+    // field is missing:
+    send("addDocument", "{fields: {id: 2}}");
+    verifySort("int");
+  }
+
+  public void testMissingLastLong() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, long: -7}}");
+    send("addDocument", "{fields: {id: 1, long: 7}}");
+    // field is missing:
+    send("addDocument", "{fields: {id: 2}}");
+    verifySort("long");
+  }
+
+  public void testMissingLastFloat() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, float: -7}}");
+    send("addDocument", "{fields: {id: 1, float: 7}}");
+    // field is missing:
+    send("addDocument", "{fields: {id: 2}}");
+    verifySort("float");
+  }
+
+  public void testMissingLastDouble() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, double: -7}}");
+    send("addDocument", "{fields: {id: 1, double: 7}}");
+    // field is missing:
+    send("addDocument", "{fields: {id: 2}}");
+    verifySort("double");
+  }
+
+  public void testNoSortOnText() throws Exception {
+    assertFailsWith("registerFields",
+                    "{fields: {bad: {type: text, sort: true, analyzer: WhitespaceAnalyzer}}}",
+                    "registerFields > fields > bad > sort: cannot sort text fields; use atom instead");
+  }
+
+  private void verifySort(String field) throws Exception {
+
+    long gen = getLong("indexGen");
+
+    // missing is (annoyingly) first by default:
+    send("search",
+         "{query: MatchAllDocsQuery, topHits: 3, retrieveFields: [id], searcher: {indexGen: " + gen + "}, sort: {fields: [{field: " + field + "}]}}");
+    assertEquals(3, getInt("totalHits"));
+    assertEquals(2, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+    assertEquals(1, getInt("hits[2].fields.id"));
+
+    // reverse, missing is (annoyingly) first by default:
+    send("search",
+         "{query: MatchAllDocsQuery, topHits: 3, retrieveFields: [id], searcher: {indexGen: " + gen + "}, sort: {fields: [{field: " + field + ", reverse: true}]}}");
+    assertEquals(3, getInt("totalHits"));
+    assertEquals(1, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+    assertEquals(2, getInt("hits[2].fields.id"));
+
+    // missing last:
+    send("search",
+         "{query: MatchAllDocsQuery, topHits: 3, retrieveFields: [id], searcher: {indexGen: " + gen + "}, sort: {fields: [{field: " + field + ", missingLast: true}]}}");
+    assertEquals(3, getInt("totalHits"));
+    assertEquals(0, getInt("hits[0].fields.id"));
+    assertEquals(1, getInt("hits[1].fields.id"));
+    assertEquals(2, getInt("hits[2].fields.id"));
+
+    // reverse, missing last:
+    send("search",
+         "{query: MatchAllDocsQuery, topHits: 3, retrieveFields: [id], searcher: {indexGen: " + gen + "}, sort: {fields: [{field: " + field + ", reverse: true, missingLast: true}]}}");
+    assertEquals(3, getInt("totalHits"));
+    assertEquals(2, getInt("hits[0].fields.id"));
+    assertEquals(1, getInt("hits[1].fields.id"));
+    assertEquals(0, getInt("hits[2].fields.id"));
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSuggest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSuggest.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSuggest.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestSuggest.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,299 @@
+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.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+public class TestSuggest extends ServerBaseTestCase {
+
+  static File tempFile;
+  
+  @BeforeClass
+  public static void initClass() throws Exception {
+    useDefaultIndex = true;
+    curIndexName = "index";
+    startServer();
+    createAndStartIndex();
+    commit();
+    File tempDir = TestUtil.getTempDir("TestSuggest");
+    tempDir.mkdirs();
+    tempFile = new File(tempDir, "suggest.in");
+  }
+
+  @AfterClass
+  public static void fini() throws Exception {
+    shutdownServer();
+    tempFile = null;
+  }
+
+  public void testAnalyzingSuggest() throws Exception {
+    Writer fstream = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8");
+    BufferedWriter out = new BufferedWriter(fstream);
+    out.write("5\u001flucene\u001ffoobar\n");
+    out.write("10\u001flucifer\u001ffoobar\n");
+    out.write("15\u001flove\u001ffoobar\n");
+    out.write("5\u001ftheories take time\u001ffoobar\n");
+    out.write("5\u001fthe time is now\u001ffoobar\n");
+    out.close();
+
+    JSONObject result = send("buildSuggest", "{source: {localFile: '" + tempFile.getAbsolutePath() + "'}, class: 'AnalyzingSuggester', suggestName: 'suggest', indexAnalyzer: EnglishAnalyzer, queryAnalyzer: {tokenizer: Standard, tokenFilters: [EnglishPossessive, LowerCase, PorterStem]]}}");
+    assertEquals(5, result.get("count"));
+    //commit();
+
+    for(int i=0;i<2;i++) {
+      result = send("suggestLookup", "{text: 'l', suggestName: 'suggest'}");
+      assertEquals(3, get(result, "results.length"));
+
+      assertEquals("love", get(result, "results[0].key"));
+      assertEquals(15, get(result, "results[0].weight"));
+      assertEquals("foobar", get(result, "results[0].payload"));
+
+      assertEquals("lucifer", get(result, "results[1].key"));
+      assertEquals(10, get(result, "results[1].weight"));
+      assertEquals("foobar", get(result, "results[1].payload"));
+
+      assertEquals("lucene", get(result, "results[2].key"));
+      assertEquals(5, get(result, "results[2].weight"));
+      assertEquals("foobar", get(result, "results[2].payload"));
+
+      result = send("suggestLookup", "{text: 'the', suggestName: 'suggest'}");
+      assertEquals(1, get(result, "results.length"));
+
+      assertEquals("theories take time", get(result, "results[0].key"));
+      assertEquals(5, get(result, "results[0].weight"));
+      assertEquals("foobar", get(result, "results[0].payload"));
+
+      result = send("suggestLookup", "{text: 'the ', suggestName: 'suggest'}");
+      assertEquals(0, get(result, "results.length"));
+
+      // Make sure suggest survives server restart:
+      shutdownServer();
+      startServer();
+      send("startIndex");
+    }
+  }
+
+  public void testInfixSuggest() throws Exception {
+    Writer fstream = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8");
+    BufferedWriter out = new BufferedWriter(fstream);
+    out.write("15\u001flove lost\u001ffoobar\n");
+    out.close();
+
+    send("buildSuggest", "{source: {localFile: '" + tempFile.getAbsolutePath() + "'}, class: InfixSuggester, suggestName: suggest2, analyzer: {tokenizer: Whitespace, tokenFilters: [LowerCase]}}");
+    assertEquals(1, getInt("count"));
+    //commit();
+
+    for(int i=0;i<2;i++) {
+      System.out.println("i=" + i);
+      send("suggestLookup", "{text: lost, suggestName: suggest2}");
+      assertEquals(15, getLong("results[0].weight"));
+      assertEquals("love <font color=red>lost</font>", toString(getArray("results[0].key")));
+      assertEquals("foobar", getString("results[0].payload"));
+
+      send("suggestLookup", "{text: lo, suggestName: suggest2}");
+      assertEquals(15, getLong("results[0].weight"));
+      assertEquals("<font color=red>lo</font>ve <font color=red>lo</font>st", toString(getArray("results[0].key")));
+      assertEquals("foobar", getString("results[0].payload"));
+
+      // Make sure suggest survives server restart:    
+      shutdownServer();
+      startServer();
+      send("startIndex");
+    }
+  }
+
+  public void testInfixSuggestNRT() throws Exception {
+    Writer fstream = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8");
+    BufferedWriter out = new BufferedWriter(fstream);
+    out.write("15\u001flove lost\u001ffoobar\n");
+    out.close();
+
+    send("buildSuggest", "{source: {localFile: '" + tempFile.getAbsolutePath() + "'}, class: InfixSuggester, suggestName: suggestnrt, analyzer: {tokenizer: Whitespace, tokenFilters: [LowerCase]}}");
+    assertEquals(1, getInt("count"));
+
+    for(int i=0;i<2;i++) {
+      send("suggestLookup", "{text: lost, suggestName: suggestnrt}");
+      assertEquals(15, getLong("results[0].weight"));
+      assertEquals("love <font color=red>lost</font>", toString(getArray("results[0].key")));
+      assertEquals("foobar", getString("results[0].payload"));
+
+      send("suggestLookup", "{text: lo, suggestName: suggestnrt}");
+      assertEquals(15, getLong("results[0].weight"));
+      assertEquals("<font color=red>lo</font>ve <font color=red>lo</font>st", toString(getArray("results[0].key")));
+      assertEquals("foobar", getString("results[0].payload"));
+
+      // Make sure suggest survives server restart:    
+      shutdownServer();
+      startServer();
+      send("startIndex");
+    }
+
+    // Now update the suggestions:
+    fstream = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8");
+    out = new BufferedWriter(fstream);
+    out.write("10\u001flove lost\u001ffoobaz\n");
+    out.write("20\u001flove found\u001ffooboo\n");
+    out.close();
+
+    send("updateSuggest", "{source: {localFile: '" + tempFile.getAbsolutePath() + "'}, suggestName: suggestnrt}");
+    assertEquals(2, getInt("count"));
+
+    for(int i=0;i<2;i++) {
+      send("suggestLookup", "{text: lost, suggestName: suggestnrt}");
+      assertEquals(10, getLong("results[0].weight"));
+      assertEquals("love <font color=red>lost</font>", toString(getArray("results[0].key")));
+      assertEquals("foobaz", getString("results[0].payload"));
+
+      send("suggestLookup", "{text: lo, suggestName: suggestnrt}");
+      assertEquals(2, getInt("results.length"));
+      assertEquals(20, getLong("results[0].weight"));
+      assertEquals("<font color=red>lo</font>ve found", toString(getArray("results[0].key")));
+      assertEquals("fooboo", getString("results[0].payload"));
+
+      assertEquals(10, getLong("results[1].weight"));
+      assertEquals("<font color=red>lo</font>ve <font color=red>lo</font>st", toString(getArray("results[1].key")));
+      assertEquals("foobaz", getString("results[1].payload"));
+
+      // Make sure suggest survives server restart:    
+      shutdownServer();
+      startServer();
+      send("startIndex");
+    }
+  }
+
+  public String toString(JSONArray fragments) {
+    StringBuilder sb = new StringBuilder();
+    for(Object _o : fragments) {
+      JSONObject o = (JSONObject) _o;
+      if ((Boolean) o.get("isHit")) {
+        sb.append("<font color=red>");
+      }
+      sb.append(o.get("text").toString());
+      if ((Boolean) o.get("isHit")) {
+        sb.append("</font>");
+      }
+    }
+    return sb.toString();
+  }
+
+  public void testFuzzySuggest() throws Exception {
+    Writer fstream = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8");
+    BufferedWriter out = new BufferedWriter(fstream);
+    out.write("15\u001flove lost\u001ffoobar\n");
+    out.close();
+
+    JSONObject result = send("buildSuggest", "{source: {localFile: '" + tempFile.getAbsolutePath() + "'}, class: 'FuzzySuggester', suggestName: 'suggest3', analyzer: {tokenizer: Whitespace, tokenFilters: [LowerCase]}}");
+    assertEquals(1, result.get("count"));
+    //commit();
+
+    for(int i=0;i<2;i++) {
+      // 1 transposition and this is prefix of "love":
+      result = send("suggestLookup", "{text: 'lvo', suggestName: 'suggest3'}");
+      assertEquals(15, get(result, "results[0].weight"));
+      assertEquals("love lost", get(result, "results[0].key"));
+      assertEquals("foobar", get(result, "results[0].payload"));
+
+      // Make sure suggest survives server restart:    
+      shutdownServer();
+      startServer();
+      send("startIndex");
+    }
+  }
+
+  /** Build a suggest, pulling suggestions/weights/payloads from stored fields. */
+  public void testFromStoredFields() throws Exception {
+    curIndexName = "storedSuggest";
+    TestUtil.rmDir(new File("storedsuggest"));
+    send("createIndex", "{rootDir: storedsuggest}");
+    send("settings", "{directory: FSDirectory, matchVersion: LUCENE_46}");
+    send("startIndex");
+    send("registerFields",
+         "{fields: {text: {type: text, store: true, search: false}," + 
+                  "weight: {type: float, store: true, search: false}," +
+                  "payload: {type: text, store: true, search: false}}}");
+    send("addDocument", "{fields: {text: 'the cat meows', weight: 1, payload: 'payload1'}}");
+    long indexGen = getLong(send("addDocument", "{fields: {text: 'the dog barks', weight: 2, payload: 'payload2'}}"), "indexGen");
+
+    JSONObject result = send("buildSuggest", "{source: {searcher: {indexGen: " + indexGen + "}, suggestField: text, weightField: weight, payloadField: payload}, class: 'AnalyzingSuggester', suggestName: 'suggest', analyzer: {tokenizer: Whitespace, tokenFilters: [LowerCase]}}");
+    // nocommit count isn't returned for stored fields source:
+    //assertEquals(2, result.get("count"));
+
+    for(int i=0;i<2;i++) {
+      result = send("suggestLookup", "{text: the, suggestName: suggest}");
+      assertEquals(2, getInt(result, "results[0].weight"));
+      assertEquals("the dog barks", get(result, "results[0].key"));
+      assertEquals("payload2", get(result, "results[0].payload"));
+      assertEquals(1, getInt(result, "results[1].weight"));
+      assertEquals("the cat meows", get(result, "results[1].key"));
+      assertEquals("payload1", get(result, "results[1].payload"));
+
+      // Make sure suggest survives server restart:    
+      shutdownServer();
+      startServer();
+      send("startIndex");
+    }
+  }
+
+  /** Build a suggest, pulling suggestions/payloads from
+   *  stored fields, and weight from an expression */
+  public void testFromStoredFieldsWithWeightExpression() throws Exception {
+    curIndexName = "storedsuggestexpr";
+    TestUtil.rmDir(new File(curIndexName));
+    send("createIndex", "{rootDir: " + curIndexName + "}");
+    send("settings", "{directory: FSDirectory, matchVersion: LUCENE_46}");
+    send("startIndex");
+    send("registerFields",
+         "{" +
+         "fields: {text: {type: text, store: true, search: false}," + 
+                  "negWeight: {type: float, sort: true}," +
+                  "payload: {type: text, store: true, search: false}}}");
+    send("addDocument", "{fields: {text: 'the cat meows', negWeight: -1, payload: 'payload1'}}");
+    long indexGen = getLong(send("addDocument", "{fields: {text: 'the dog barks', negWeight: -2, payload: 'payload2'}}"), "indexGen");
+
+    JSONObject result = send("buildSuggest", "{source: {searcher: {indexGen: " + indexGen + "}, suggestField: text, weightExpression: -negWeight, payloadField: payload}, class: 'AnalyzingSuggester', suggestName: 'suggest', analyzer: {tokenizer: Whitespace, tokenFilters: [LowerCase]}}");
+    // nocommit count isn't returned for stored fields source:
+    //assertEquals(2, result.get("count"));
+
+    for(int i=0;i<2;i++) {
+      result = send("suggestLookup", "{text: the, suggestName: suggest}");
+      assertEquals(2, getInt(result, "results[0].weight"));
+      assertEquals("the dog barks", get(result, "results[0].key"));
+      assertEquals("payload2", get(result, "results[0].payload"));
+      assertEquals(1, getInt(result, "results[1].weight"));
+      assertEquals("the cat meows", get(result, "results[1].key"));
+      assertEquals("payload1", get(result, "results[1].payload"));
+
+      // Make sure suggest survives server restart:    
+      shutdownServer();
+      startServer();
+      send("startIndex");
+    }
+  }
+
+  // nocommit test full build over an already built suggester
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/test/org/apache/lucene/server/TestVirtualFields.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,267 @@
+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 {
+    useDefaultIndex = true;
+    curIndexName = "index";
+    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, search: false}");
+    JSONObject o2 = new JSONObject();
+    o2.put("fields", o);
+    send("registerFields", o2);
+  }
+
+  /** Non-reversed sort by virtual field */
+  public void testSortByVirtualFieldStraight() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {id: 0, boost: 1.0}}");
+    send("addDocument", "{fields: {id: 1, boost: 2.0}}");
+    send("search", "{query: MatchAllDocsQuery, sort: {fields: [{field: logboost}]}, retrieveFields: [id]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(0, getInt("hits[0].fields.id"));
+    assertEquals(1, getInt("hits[1].fields.id"));
+
+    assertEquals(0.0f, getFloat("hits[0].fields.sortFields.logboost"), .0001f);
+    assertEquals(.6931f, getFloat("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}}");
+    send("addDocument", "{fields: {id: 1, boost: 2.0}}");
+    send("search", "{query: MatchAllDocsQuery, sort: {fields: [{field: logboost, reverse: true}]}, retrieveFields: [id]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(1, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+
+    assertEquals(.6931f, getFloat("hits[0].fields.sortFields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat("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}}");
+    send("addDocument", "{fields: {id: 1, boost: 2.0}}");
+    send("search", "{query: MatchAllDocsQuery, sort: {fields: [{field: logboost}]}, retrieveFields: [id, logboost]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(0, getInt("hits[0].fields.id"));
+    assertEquals(1, getInt("hits[1].fields.id"));
+
+    assertEquals(0.0f, getFloat("hits[0].fields.logboost"), .0001f);
+    assertEquals(.6931f, getFloat("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}}");
+    send("addDocument", "{fields: {id: 1, boost: 2.0}}");
+    send("search", "{query: MatchAllDocsQuery, sort: {fields: [{field: id, reverse: true}]}, retrieveFields: [id, logboost]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(1, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+
+    assertEquals(.6931f, getFloat("hits[0].fields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat("hits[1].fields.logboost"), .0001f);
+  }
+
+  public void testFieldUsingAnother() throws Exception {
+    deleteAllDocs();
+    send("registerFields", "{fields: {scoreboost2: {type: virtual, expression: '2*scoreboost'}}}");
+
+    send("addDocument", "{fields: {text: 'the wind is howling like this swirling storm inside', id: 0, boost: 1.0}}");
+    send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}");
+    send("search", "{queryText: wind, sort: {fields: [{field: scoreboost2, reverse: true}]}, retrieveFields: [id, scoreboost2]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(1, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+
+    assertEquals(1.60721f, getFloat("hits[0].fields.scoreboost2"), .0001f);
+    assertEquals(0.22092f, getFloat("hits[1].fields.scoreboost2"), .0001f);
+
+    assertEquals(1.60721f, getFloat("hits[0].fields.sortFields.scoreboost2"), .0001f);
+    assertEquals(0.22092f, getFloat("hits[1].fields.sortFields.scoreboost2"), .0001f);
+  }
+
+  public void testWithScore1() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {text: 'the wind is howling like this swirling storm inside', id: 0, boost: 1.0}}");
+    send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}");
+    send("search", "{queryText: wind, sort: {fields: [{field: scoreboost, reverse: true}]}, retrieveFields: [id, logboost]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(1, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+
+    assertEquals(.6931f, getFloat("hits[0].fields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat("hits[1].fields.logboost"), .0001f);
+  }
+
+  /** Also tries to retrieve the scoreboost */
+  public void testWithScore2() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {text: 'the wind is howling like this swirling storm inside', id: 0, boost: 1.0}}");
+    send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}");
+    send("search", "{queryText: wind, sort: {fields: [{field: scoreboost, reverse: true}]}, retrieveFields: [id, scoreboost, logboost]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(1, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+
+    assertEquals(.80361f, getFloat("hits[0].fields.scoreboost"), .0001f);
+    assertEquals(.11046f, getFloat("hits[1].fields.scoreboost"), .0001f);
+
+    assertEquals(.6931f, getFloat("hits[0].fields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat("hits[1].fields.logboost"), .0001f);
+  }
+
+  /** Sort by not score, and try to retrieve expression
+   *  using score. */
+  public void testWithScore3() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {text: 'the wind is howling like this swirling storm inside', id: 0, boost: 1.0}}");
+    send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}");
+    send("search", "{queryText: wind, sort: {fields: [{field: id, reverse: true}]}, retrieveFields: [id, scoreboost, logboost]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(1, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+
+    assertEquals(.80361f, getFloat("hits[0].fields.scoreboost"), .0001f);
+    assertEquals(.11046f, getFloat("hits[1].fields.scoreboost"), .0001f);
+
+    assertEquals(.6931f, getFloat("hits[0].fields.logboost"), .0001f);
+    assertEquals(0.0f, getFloat("hits[1].fields.logboost"), .0001f);
+  }
+
+  public void testSyntaxError() throws Exception {
+    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"));
+      assertTrue(message.contains("unexpected token end of expression"));
+    }
+  }
+
+  public void testNonExistentField() throws Exception {
+    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"));
+      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)'}}}");
+      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}}");
+    send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}");
+
+    // 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 testCycles1() throws Exception {
+    assertFailsWith("registerFields", "{fields: {bad: {type: virtual, expression: ln(bad)}}}",
+                    "registerFields > fields > bad > expression: could not evaluate expression: java.lang.IllegalArgumentException: Invalid reference 'bad'");
+  }
+
+  public void testCycles2() throws Exception {
+    assertFailsWith("search",
+                    "{queryText: wind, virtualFields: [{name: bad, expression: ln(bad)}]}",
+                    "search > virtualFields[0] > expression: could not evaluate expression: java.lang.IllegalArgumentException: Invalid reference 'bad'");
+  }
+
+  public void testRetrievedDynamicField() throws Exception {
+    deleteAllDocs();
+    send("addDocument", "{fields: {text: 'the wind is howling like this swirling storm inside', id: 0, boost: 1.0}}");
+    send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}");
+    send("search",
+         "{queryText: wind, virtualFields: [{name: scoreboost3, expression: 3*scoreboost}], sort: {fields: [{field: id, reverse: true}]}, retrieveFields: [id, scoreboost3]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(1, getInt("hits[0].fields.id"));
+    assertEquals(0, getInt("hits[1].fields.id"));
+
+    assertEquals(2.41082, getFloat("hits[0].fields.scoreboost3"), .0001f);
+    assertEquals(0.33138, getFloat("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}}");
+    send("addDocument", "{fields: {text: 'I am one with the wind and sky', id: 1, boost: 2.0}}");
+    send("search",
+         "{queryText: wind, virtualFields: [{name: scoreboost3, expression: 3*scoreboost}], sort: {fields: [{field: scoreboost3}]}, retrieveFields: [id]}");
+    assertEquals(2, getInt("totalHits"));
+    assertEquals(0, getInt("hits[0].fields.id"));
+    assertEquals(1, getInt("hits[1].fields.id"));
+
+    assertEquals(0.33138, getFloat("hits[0].fields.sortFields.scoreboost3"), .0001f);
+    assertEquals(2.41082, getFloat("hits[1].fields.sortFields.scoreboost3"), .0001f);
+  }
+}

Modified: lucene/dev/branches/lucene5376_2/lucene/suggest/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/suggest/build.xml?rev=1578133&r1=1578132&r2=1578133&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/suggest/build.xml (original)
+++ lucene/dev/branches/lucene5376_2/lucene/suggest/build.xml Sun Mar 16 18:11:07 2014
@@ -35,7 +35,7 @@
     <path refid="base.classpath"/>
   </path>
 
-  <target name="javadocs" depends="javadocs-queries,javadocs-misc,compile-core">
+  <target name="javadocs" depends="javadocs-queries,javadocs-analyzers-common,javadocs-misc,compile-core">
     <invoke-module-javadoc>
       <links>
         <link href="../analyzers-common"/>