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 [4/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/analys...

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,519 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.expressions.Expression;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
+import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager.SearcherAndTaxonomy;
+import org.apache.lucene.search.suggest.DocumentDictionary;
+import org.apache.lucene.search.suggest.DocumentValueSourceDictionary;
+import org.apache.lucene.search.suggest.InputIterator;
+import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs
+import org.apache.lucene.search.suggest.Lookup;
+import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester;
+import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester;
+import org.apache.lucene.search.suggest.analyzing.FuzzySuggester;
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.FromFileTermFreqIterator;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.*;
+import org.apache.lucene.server.params.PolyType.PolyEntry;
+import org.apache.lucene.util.BytesRef;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+import net.minidev.json.JSONValue;
+import net.minidev.json.parser.ParseException;
+
+import static org.apache.lucene.server.handlers.RegisterFieldHandler.ANALYZER_TYPE;
+
+/** Handles {@code buildSuggest}. */
+public class BuildSuggestHandler extends Handler {
+
+  private final static StructType TYPE =
+    new StructType(
+        new Param("indexName", "Index Name", new StringType()),
+        new Param("class", "Which suggester implementation to use",
+            new PolyType(Lookup.class,
+                         new PolyEntry("AnalyzingSuggester", "Suggester that first analyzes the surface form, adds the analyzed form to a weighted FST, and then does the same thing at lookup time (see @lucene:suggest:org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester)",
+                             new Param("analyzer", "Index and query analyzer", ANALYZER_TYPE),
+                             new Param("indexAnalyzer", "Index analyzer", ANALYZER_TYPE),
+                             new Param("queryAnalyzer", "Query analyzer", ANALYZER_TYPE),
+                             new Param("maxSurfaceFormsPerAnalyzedForm", "Maximum number of surface forms to keep for a single analyzed form", new IntType(), 256),
+                             new Param("maxGraphExpansions", "Maximum number of graph paths to expand from the analyzed from", new IntType(), -1),
+                             new Param("preserveSep", "True if token separators should be preserved when matching", new BooleanType(), true),
+                             new Param("exactFirst", "True if the exact match should always be returned first regardless of score", new BooleanType(), true)),
+                         new PolyEntry("FuzzySuggester", "Implements a fuzzy AnalyzingSuggester (see @lucene:suggest:org.apache.lucene.search.suggest.analyzing.FuzzySuggester)",
+                             new Param("analyzer", "Index and query analyzer", ANALYZER_TYPE),
+                             new Param("indexAnalyzer", "Index analyzer", ANALYZER_TYPE),
+                             new Param("queryAnalyzer", "Query analyzer", ANALYZER_TYPE),  
+                             new Param("maxSurfaceFormsPerAnalyzedForm", "Maximum number of surface forms to keep for a single analyzed form", new IntType(), 156),
+                             new Param("maxGraphExpansions", "Maximum number of graph paths to expand from the analyzed from", new IntType(), -1),
+                             new Param("preserveSep", "True if token separators should be preserved when matching", new BooleanType(), true),
+                             new Param("exactFirst", "True if the exact match should always be returned first regardless of score", new BooleanType(), true),
+                             new Param("minFuzzyLength", "Minimum key length before edits are allowed", new IntType(), FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH),
+                             new Param("nonFuzzyPrefix", "Key prefix where edits are not allowed", new IntType(), FuzzySuggester.DEFAULT_NON_FUZZY_PREFIX),
+                             new Param("maxEdits", "Maximum number of edits for fuzzy suggestions", new IntType(), FuzzySuggester.DEFAULT_MAX_EDITS),
+                             new Param("transpositions", "Whether transpositions are allowed", new BooleanType(), FuzzySuggester.DEFAULT_TRANSPOSITIONS),
+                             new Param("unicodeAware", "True if all edits are measured in unicode characters, not UTF-8 bytes", new BooleanType(), FuzzySuggester.DEFAULT_UNICODE_AWARE)),
+                         new PolyEntry("InfixSuggester", "A suggester that matches terms anywhere in the input text, not just as a prefix. (see @lucene:org:server.InfixSuggester)",
+                             new Param("analyzer", "Index and query analyzer", ANALYZER_TYPE),
+                             new Param("indexAnalyzer", "Index analyzer", ANALYZER_TYPE),
+                             new Param("queryAnalyzer", "Query analyzer", ANALYZER_TYPE))),
+                  "AnalyzingSuggester"),
+        // nocommit option to stream suggestions in over the wire too
+        new Param("source", "Where to get suggestions from",
+            new StructType(
+                  new Param("localFile", "Local file (to the server) to read suggestions + weights from; format is weight U+001F suggestion U+001F payload, one per line, with suggestion UTF-8 encoded.  If this option is used then searcher, suggestField, weightField/Expression, payloadField should not be specified.", new StringType()),
+                  new Param("searcher", "Specific searcher version to use for pull suggestions to build.  There are three different ways to specify a searcher version.",
+                            SearchHandler.SEARCHER_VERSION_TYPE),
+                  new Param("suggestField", "Field (from stored documents) containing the suggestion text", new StringType()),
+                  new Param("weightField", "Numeric field (from stored documents) containing the weight", new StringType()),
+                  new Param("weightExpression", "Alternative to weightField, an expression that's evaluated to the weight.  Note that any fields referenced in the expression must have been indexed with sort=true.", new StringType()),
+                  new Param("payloadField", "Optional binary or string field (from stored documents) containing the payload", new StringType()))),
+        new Param("suggestName", "Unique name for this suggest build.", new StringType())
+                   );
+
+  /** Sole constructor. */
+  public BuildSuggestHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Builds a new auto-suggester, loading suggestions via the provided local file path.";
+  }
+
+  /** Load all previously built suggesters. */
+  public void load(IndexState state, JSONObject saveState) throws IOException {
+    for(Map.Entry<String,Object> ent : saveState.entrySet()) {
+      String suggestName = ent.getKey();
+
+      JSONObject params = (JSONObject) ent.getValue();
+      String jsonOrig = params.toString();
+
+      Request r = new Request(null, null, params, TYPE);
+      // Must consume these up front since getSuggester
+      // won't:
+      r.getString("suggestName");
+      Request source = r.getStruct("source");
+      if (source.hasParam("localFile")) {
+        source.getString("localFile");
+      } else {
+        Request searcher = source.getStruct("searcher");
+        if (searcher.hasParam("indexGen")) {
+          searcher.getLong("indexGen");
+        } else {        
+          searcher.getString("snapshot");
+        }
+        source.getString("suggestField");
+        if (source.hasParam("weightField")) {
+          source.getString("weightField");
+        } else {
+          source.getString("weightExpression");
+        }
+        if (source.hasParam("payloadField")) {
+          source.getString("payloadField");
+        }
+      }
+      Lookup suggester = getSuggester(state, suggestName, r);
+      assert !Request.anythingLeft(params);
+
+      if ((suggester instanceof AnalyzingInfixSuggester) == false) {
+        File path = new File(state.rootDir, "suggest." + suggestName);
+        FileInputStream in = new FileInputStream(path);
+        try {
+          suggester.load(in);
+        } finally {
+          in.close();
+        }
+      }
+
+      try {
+        state.addSuggest(suggestName, (JSONObject) JSONValue.parseStrict(jsonOrig));
+      } catch (ParseException pe) {
+        // BUG
+        throw new RuntimeException(pe);
+      }
+    }
+  }
+
+  private Lookup getSuggester(IndexState state, String suggestName, Request r) throws IOException {
+
+    Request.PolyResult pr = r.getPoly("class");
+
+    Lookup oldSuggester = state.suggesters.get(suggestName);
+    if (oldSuggester != null && oldSuggester instanceof Closeable) {
+      ((Closeable) oldSuggester).close();
+      state.suggesters.remove(suggestName);
+    }
+    
+    String impl = pr.name;
+
+    r = pr.r;
+    Analyzer indexAnalyzer;
+    Analyzer queryAnalyzer;
+    // nocommit allow passing a field name, and we use that
+    // field name's analyzer?
+    if (r.hasParam("analyzer")) {
+      indexAnalyzer = queryAnalyzer = RegisterFieldHandler.getAnalyzer(state, r, "analyzer");
+    } else {
+      indexAnalyzer = RegisterFieldHandler.getAnalyzer(state, r, "indexAnalyzer");
+      queryAnalyzer = RegisterFieldHandler.getAnalyzer(state, r, "queryAnalyzer");
+    }
+    if (indexAnalyzer == null) {
+      r.fail("analyzer", "analyzer or indexAnalyzer must be specified");
+    }
+    if (queryAnalyzer == null) {
+      r.fail("analyzer", "analyzer or queryAnalyzer must be specified");
+    }
+    final Lookup suggester;
+
+    if (impl.equals("FuzzySuggester")) {
+      int options = 0;
+      if (r.getBoolean("preserveSep")) {
+        options |= AnalyzingSuggester.PRESERVE_SEP;
+      }
+      if (r.getBoolean("exactFirst")) {
+        options |= AnalyzingSuggester.EXACT_FIRST;
+      }
+      suggester = new FuzzySuggester(indexAnalyzer, queryAnalyzer,
+                                     options,
+                                     r.getInt("maxSurfaceFormsPerAnalyzedForm"),
+                                     r.getInt("maxGraphExpansions"),
+                                     true,
+                                     r.getInt("maxEdits"),
+                                     r.getBoolean("transpositions"),
+                                     r.getInt("nonFuzzyPrefix"),
+                                     r.getInt("minFuzzyLength"),
+                                     r.getBoolean("unicodeAware"));
+    } else if (impl.equals("AnalyzingSuggester")) {
+      int options = 0;
+      if (r.getBoolean("preserveSep")) {
+        options |= AnalyzingSuggester.PRESERVE_SEP;
+      }
+      if (r.getBoolean("exactFirst")) {
+        options |= AnalyzingSuggester.EXACT_FIRST;
+      }
+      suggester = new AnalyzingSuggester(indexAnalyzer, queryAnalyzer, options,
+                                         r.getInt("maxSurfaceFormsPerAnalyzedForm"),
+                                         r.getInt("maxGraphExpansions"), true);
+    } else if (impl.equals("InfixSuggester")) {
+      suggester = new AnalyzingInfixSuggester(state.matchVersion,
+                                              new File(state.rootDir, "suggest." + suggestName + ".infix"),
+                                              indexAnalyzer,
+                                              queryAnalyzer,
+                                              AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS) {
+          @Override
+          protected Object highlight(String text, Set<String> matchedTokens, String prefixToken) throws IOException {
+            
+            // We override the entire highlight method, to
+            // render directly to JSONArray instead of html
+            // string:
+
+            TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text));
+            CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+            OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
+            ts.reset();
+            JSONArray fragments = new JSONArray();
+            int upto = 0;
+            while (ts.incrementToken()) {
+              String token = termAtt.toString();
+              int startOffset = offsetAtt.startOffset();
+              int endOffset = offsetAtt.endOffset();
+              if (upto < startOffset) {
+                JSONObject o = new JSONObject();
+                fragments.add(o);
+                o.put("isHit", false);
+                o.put("text", text.substring(upto, startOffset));
+                upto = startOffset;
+              } else if (upto > startOffset) {
+                continue;
+              }
+
+              if (matchedTokens.contains(token)) {
+                // Token matches.
+                JSONObject o = new JSONObject();
+                fragments.add(o);
+                o.put("isHit", true);
+                o.put("text", text.substring(startOffset, endOffset));
+                upto = endOffset;
+              } else if (prefixToken != null && token.startsWith(prefixToken)) {
+                JSONObject o = new JSONObject();
+                fragments.add(o);
+                o.put("isHit", true);
+                o.put("text", text.substring(startOffset, startOffset+prefixToken.length()));
+                if (prefixToken.length() < token.length()) {
+                  o = new JSONObject();
+                  fragments.add(o);
+                  o.put("isHit", false);
+                  o.put("text", text.substring(startOffset+prefixToken.length(), startOffset+token.length()));
+                }
+                upto = endOffset;
+              }
+            }
+            ts.end();
+            int endOffset = offsetAtt.endOffset();
+            if (upto < endOffset) {
+              JSONObject o = new JSONObject();
+              fragments.add(o);
+              o.put("isHit", false);
+              o.put("text", text.substring(upto));
+            }
+            ts.close();
+
+            return fragments;
+          }
+        };
+    } else {
+      suggester = null;
+      assert false;
+    }
+
+    state.suggesters.put(suggestName, suggester);
+
+    return suggester;
+  }
+
+  /** Used to return highlighted result; see {@link
+   *  LookupResult#highlightKey} */
+  public static final class LookupHighlightFragment {
+    /** Portion of text for this fragment. */
+    public final String text;
+
+    /** True if this text matched a part of the user's
+     *  query. */
+    public final boolean isHit;
+
+    /** Sole constructor. */
+    public LookupHighlightFragment(String text, boolean isHit) {
+      this.text = text;
+      this.isHit = isHit;
+    }
+
+    @Override
+    public String toString() {
+      return "LookupHighlightFragment(text=" + text + " isHit=" + isHit + ")";
+    }
+  }
+
+  /** Wraps another {@link InputIterator} and counts how
+   *  many suggestions were seen. */
+  private static class CountingInputIterator implements InputIterator {
+
+    private final InputIterator other;
+    private int count;
+
+    public CountingInputIterator(InputIterator other) {
+      this.other = other;
+    }
+
+    @Override
+    public long weight() {
+      return other.weight();
+    }
+
+    @Override
+    public BytesRef next() throws IOException {
+      BytesRef result = other.next();
+      if (result != null) {
+        count++;
+      }
+
+      return result;
+    }
+
+    @Override
+    public BytesRef payload() {
+      return other.payload();
+    }
+
+    @Override
+    public boolean hasPayloads() {
+      return other.hasPayloads();
+    }
+
+    public int getCount() {
+      return count;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+
+    final String jsonOrig = r.toString();
+
+    final String suggestName = r.getString("suggestName");
+    if (!IndexState.isSimpleName(suggestName)) {
+      r.fail("suggestName", "invalid suggestName \"" + suggestName + "\": must be [a-zA-Z_][a-zA-Z0-9]*");
+    }
+
+    final Lookup suggester = getSuggester(state, suggestName, r);
+
+    Request source = r.getStruct("source");
+
+    InputIterator iterator = null;
+    final SearcherAndTaxonomy searcher;
+
+    if (source.hasParam("localFile")) {
+      final File localFile = new File(source.getString("localFile"));
+      if (!localFile.exists()) {
+        r.fail("localFile", "file does not exist");
+      }
+      if (!localFile.canRead()) {
+        r.fail("localFile", "cannot read file");
+      }
+      searcher = null;
+      // Pull suggestions from local file:
+      try {
+        iterator = new FromFileTermFreqIterator(localFile);
+      } catch (IOException ioe) {
+        r.fail("localFile", "cannot open file", ioe);
+      }
+    } else {
+
+      state.verifyStarted(r);
+
+      // Pull suggestions from stored docs:
+      if (source.hasParam("searcher")) {
+        // Specific searcher version:
+        searcher = SearchHandler.getSearcherAndTaxonomy(source, state, null);
+      } else {
+        // Just use current searcher version:
+        searcher = state.manager.acquire();
+      }
+      String suggestField = source.getString("suggestField");
+
+      String payloadField;
+      if (source.hasParam("payloadField")) {
+        payloadField = source.getString("payloadField");
+      } else {
+        payloadField = null;
+      }
+
+      DocumentDictionary dict;
+
+      if (source.hasParam("weightField")) {
+        // Weight is a field
+        String weightField = source.getString("weightField");
+        dict = new DocumentDictionary(searcher.searcher.getIndexReader(),
+                                      suggestField,
+                                      weightField,
+                                      payloadField);
+      } else {
+        // Weight is an expression; add bindings for all
+        // numeric DV fields:
+        Expression expr;
+        try {
+          expr = JavascriptCompiler.compile(source.getString("weightExpression"));
+        } catch (Exception e) {
+          source.fail("weightExpression", "expression does not compile", e);
+          // Dead code but compiler disagrees
+          expr = null;
+        }
+        
+        dict = new DocumentValueSourceDictionary(searcher.searcher.getIndexReader(),
+                                                 suggestField,
+                                                 expr.getValueSource(state.exprBindings),
+                                                 payloadField);
+      }
+
+      iterator = dict.getEntryIterator();
+    }
+
+    // nocommit return error if suggester already exists?
+
+    // nocommit need a DeleteSuggestHandler
+
+    final InputIterator iterator0 = iterator;
+    
+    return new FinishRequest() {
+
+      @Override
+      public String finish() throws IOException {
+
+        try {
+          suggester.build(iterator0);
+        } finally {
+          if (iterator0 instanceof Closeable) {
+            ((Closeable) iterator0).close();
+          }
+          if (searcher != null) {
+            state.manager.release(searcher);
+          }
+        }
+
+        File outFile = new File(state.rootDir, "suggest." + suggestName);
+        OutputStream out = new FileOutputStream(outFile);
+        boolean success = false;
+        try {
+          // nocommit look @ return value
+          suggester.store(out);
+          success = true;
+        } finally {
+          out.close();
+          if (!success) {
+            outFile.delete();
+          }
+        }
+
+        //System.out.println("buildSuggest: now add suggestName=" + suggestName + " to IndexState.name=" + state.name);
+
+        try {
+          state.addSuggest(suggestName, (JSONObject) JSONValue.parseStrict(jsonOrig));
+        } catch (ParseException pe) {
+          // BUG
+          throw new RuntimeException(pe);
+        }
+
+        JSONObject ret = new JSONObject();
+        if (suggester instanceof AnalyzingSuggester) {
+          ret.put("sizeInBytes", ((AnalyzingSuggester) suggester).sizeInBytes());
+        }
+        if (suggester instanceof AnalyzingInfixSuggester) {
+          ((AnalyzingInfixSuggester) suggester).commit();
+        }
+
+        ret.put("count", suggester.getCount());
+        return ret.toString();
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,141 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Reader;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.*;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+import static org.apache.lucene.server.IndexState.AddDocumentContext;
+
+/** Reads more than one { ... } request in a single
+ *  connection, but each request must be separated by at
+ *  least one whitespace char. */
+
+public class BulkAddDocumentHandler extends Handler {
+
+  private StructType TYPE = new StructType(
+                                     new Param("indexName", "Index name", new StringType()),
+                                     new Param("documents", "List of documents", new ListType(AddDocumentHandler.DOCUMENT_TYPE)));
+
+  /** Sole constructor. */
+  public BulkAddDocumentHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public boolean doStream() {
+    return true;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Add more than one document in a single request.  Returns the index generation (indexGen) that contains all added documents.";
+  }
+
+  @Override
+  public FinishRequest handle(IndexState state, Request r, Map<String,List<String>> params) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String handleStreamed(Reader reader, Map<String,List<String>> params) throws Exception {
+    JsonFactory jfactory = new JsonFactory();
+
+    JsonParser parser = jfactory.createJsonParser(reader);
+
+    if (parser.nextToken() != JsonToken.START_OBJECT) {
+      throw new IllegalArgumentException("expected JSON object");
+    }
+    if (parser.nextToken() != JsonToken.FIELD_NAME) {
+      throw new IllegalArgumentException("expected indexName first");
+    }
+    if (!parser.getText().equals("indexName")) {
+      throw new IllegalArgumentException("expected indexName first");
+    }
+    if (parser.nextToken() != JsonToken.VALUE_STRING) {
+      throw new IllegalArgumentException("indexName should be string");
+    }
+
+    IndexState state = globalState.get(parser.getText());
+    state.verifyStarted(null);
+    if (parser.nextToken() != JsonToken.FIELD_NAME) {
+      throw new IllegalArgumentException("expected documents next");
+    }
+    if (!parser.getText().equals("documents")) {
+      throw new IllegalArgumentException("expected documents after indexName");
+    }
+
+    if (parser.nextToken() != JsonToken.START_ARRAY) {
+      throw new IllegalArgumentException("documents should be a list");
+    }
+
+    AddDocumentContext ctx = new AddDocumentContext();
+
+    AddDocumentHandler addDocHandler = (AddDocumentHandler) globalState.getHandler("addDocument");
+    int count = 0;
+    while (true) {
+      Document doc = addDocHandler.parseDocument(state, parser);
+      if (doc == null) {
+        break;
+      }
+      globalState.indexService.submit(state.getAddDocumentJob(count, null, doc, ctx));
+      count++;
+    }
+
+    // nocommit this is ... lameish:
+    while (true) {
+      if (ctx.addCount.get() == count) {
+        break;
+      }
+      Thread.sleep(1);
+    }
+
+    JSONObject o = new JSONObject();
+    o.put("indexGen", state.writer.getGeneration());
+    o.put("indexedDocumentCount", count);
+    if (!ctx.errors.isEmpty()) {
+      JSONArray errors = new JSONArray();
+      o.put("errors", errors);
+      for(int i=0;i<ctx.errors.size();i++) {
+        JSONObject err = new JSONObject();
+        errors.add(err);
+        err.put("index", ctx.errorIndex.get(i));
+        err.put("exception", ctx.errors.get(i));
+      }
+    }
+    
+    return o.toString();
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentsHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentsHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,203 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.*;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+import static org.apache.lucene.server.IndexState.AddDocumentContext;
+
+/** Reads more than one { ... } request in a single
+ *  connection, but each request must be separated by at
+ *  least one whitespace char. */
+
+public class BulkAddDocumentsHandler extends Handler {
+
+  private StructType TYPE = new StructType(
+                                     new Param("indexName", "Index name", new StringType()),
+                                     new Param("documents", "List of documents",
+                                         new ListType(
+                                             new StructType(
+                                                 new Param("parent", "The (one) parent document for this block.  The value of this key is a single document that @addDocument expects.  Be sure to add an indexed field to only the parent document so that you can subsequently provide the filter that identifies only parent documents.",
+                                                           AddDocumentHandler.DOCUMENT_TYPE),
+                                                 new Param("children", "List of child documents.",
+                                                           new ListType(AddDocumentHandler.DOCUMENT_TYPE))))));
+
+  /** Sole constructor. */
+  public BulkAddDocumentsHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public boolean doStream() {
+    return true;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Add more than one document block in a single connection.  Each document should be its own JSON struct, matching @addDocuments, and then there must one whitespace character separating each document.  Returns the index generation (indexGen) that contains all added document blocks.";
+  }
+
+  @Override
+  public FinishRequest handle(IndexState state, Request r, Map<String,List<String>> params) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String handleStreamed(Reader reader, Map<String,List<String>> params) throws Exception {
+
+    JsonFactory jfactory = new JsonFactory();
+
+    JsonParser parser = jfactory.createJsonParser(reader);
+
+    if (parser.nextToken() != JsonToken.START_OBJECT) {
+      throw new IllegalArgumentException("expected JSON object");
+    }
+    if (parser.nextToken() != JsonToken.FIELD_NAME) {
+      throw new IllegalArgumentException("expected indexName first");
+    }
+    if (!parser.getText().equals("indexName")) {
+      throw new IllegalArgumentException("expected indexName first");
+    }
+    if (parser.nextToken() != JsonToken.VALUE_STRING) {
+      throw new IllegalArgumentException("indexName should be string");
+    }
+
+    IndexState state = globalState.get(parser.getText());
+    state.verifyStarted(null);
+    if (parser.nextToken() != JsonToken.FIELD_NAME) {
+      throw new IllegalArgumentException("expected documents next");
+    }
+    if (!parser.getText().equals("documents")) {
+      throw new IllegalArgumentException("expected documents after indexName");
+    }
+
+    if (parser.nextToken() != JsonToken.START_ARRAY) {
+      throw new IllegalArgumentException("documents should be a list");
+    }
+
+    int count = 0;
+    AddDocumentContext ctx = new AddDocumentContext();
+
+    AddDocumentHandler addDocHandler = (AddDocumentHandler) globalState.getHandler("addDocument");
+
+    // Parse as many doc blocks as there are:
+    while (true) {
+
+      List<Document> children = null;
+      Document parent = null;
+
+      JsonToken token = parser.nextToken();
+      if (token == JsonToken.END_ARRAY) {
+        break;
+      }
+      if (token != JsonToken.START_OBJECT) {
+        throw new IllegalArgumentException("expected object");
+      }
+
+      // Parse parent + children for this one doc block:
+      while(true) {
+        token = parser.nextToken();
+        if (token == JsonToken.END_OBJECT) {
+          // Done with parent + child in this block
+          break;
+        }
+        if (token != JsonToken.FIELD_NAME) {
+          throw new IllegalArgumentException("missing field name: " + token);
+        }
+        String f = parser.getText();
+        if (f.equals("children")) {
+          token = parser.nextToken();
+          if (token != JsonToken.START_ARRAY) {
+            throw new IllegalArgumentException("expected array for children");
+          }
+
+          children = new ArrayList<Document>();
+
+          // Parse each child:
+          while (true) {
+            Document doc = addDocHandler.parseDocument(state, parser);
+            if (doc == null) {
+              break;
+            }
+            children.add(doc);
+          }
+        } else if (f.equals("parent")) {
+          parent = addDocHandler.parseDocument(state, parser);
+        } else {
+          throw new IllegalArgumentException("unrecognized field name \"" + f + "\"");
+        }
+      }
+
+      if (parent == null) {
+        throw new IllegalArgumentException("missing parent");
+      }
+      if (children == null) {
+        throw new IllegalArgumentException("missing children");
+      }
+
+      // Parent is last:
+      children.add(parent);
+
+      globalState.indexService.submit(state.getAddDocumentsJob(count, null, children, ctx));
+      count++;
+    }
+
+    // nocommit this is ... lameish:
+    while (true) {
+      if (ctx.addCount.get() == count) {
+        break;
+      }
+      Thread.sleep(1);
+    }
+
+    JSONObject o = new JSONObject();
+    o.put("indexGen", state.writer.getGeneration());
+    o.put("indexedDocumentBlockCount", count);
+    if (!ctx.errors.isEmpty()) {
+      JSONArray errors = new JSONArray();
+      o.put("errors", errors);
+      for(int i=0;i<ctx.errors.size();i++) {
+        JSONObject err = new JSONObject();
+        errors.add(err);
+        err.put("index", ctx.errorIndex.get(i));
+        err.put("exception", ctx.errors.get(i));
+      }
+    }
+
+    return o.toString();
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,219 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Reader;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.ListType;
+import org.apache.lucene.server.params.Param;
+import org.apache.lucene.server.params.Request;
+import org.apache.lucene.server.params.StringType;
+import org.apache.lucene.server.params.StructType;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import net.minidev.json.JSONObject;
+
+import static org.apache.lucene.server.IndexState.AddDocumentContext;
+
+/** Handles {@code bulkUpdateDocument}. */
+public class BulkUpdateDocumentHandler extends Handler {
+
+  /** {@link StructType} for update document. */
+  public final static StructType UPDATE_DOCUMENT_TYPE =  new StructType(
+                                                 new Param("term", "Identifies which document to replace", 
+                                                     new StructType(
+                                                         new Param("field", "Field", new StringType()),
+                                                         new Param("term", "Text", new StringType()))));
+
+  static {
+    UPDATE_DOCUMENT_TYPE.params.putAll(AddDocumentHandler.DOCUMENT_TYPE.params);
+  }
+
+  private StructType TYPE = new StructType(
+                                     new Param("indexName", "Index name", new StringType()),
+                                     new Param("documents", "List of documents",
+                                         new ListType(UPDATE_DOCUMENT_TYPE)));
+
+  /** Sole constructor. */
+  public BulkUpdateDocumentHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public boolean doStream() {
+    return true;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Update more than one document in a single HTTP connection.  Each document should be its own JSON struct, matching @updateDocument, and then there must one whitespace character separating each document.  Returns the index generation (indexGen) that contains all updated documents.";
+  }
+
+  @Override
+  public FinishRequest handle(IndexState state, Request r, Map<String,List<String>> params) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String handleStreamed(Reader reader, Map<String,List<String>> params) throws Exception {
+    JsonFactory jfactory = new JsonFactory();
+
+    JsonParser parser = jfactory.createJsonParser(reader);
+
+    if (parser.nextToken() != JsonToken.START_OBJECT) {
+      throw new IllegalArgumentException("expected JSON object");
+    }
+    if (parser.nextToken() != JsonToken.FIELD_NAME) {
+      throw new IllegalArgumentException("expected indexName first");
+    }
+    if (!parser.getText().equals("indexName")) {
+      throw new IllegalArgumentException("expected indexName first");
+    }
+    if (parser.nextToken() != JsonToken.VALUE_STRING) {
+      throw new IllegalArgumentException("indexName should be string");
+    }
+
+    IndexState state = globalState.get(parser.getText());
+    state.verifyStarted(null);
+
+    if (parser.nextToken() != JsonToken.FIELD_NAME) {
+      throw new IllegalArgumentException("expected documents next");
+    }
+    if (!parser.getText().equals("documents")) {
+      throw new IllegalArgumentException("expected documents after indexName");
+    }
+    if (parser.nextToken() != JsonToken.START_ARRAY) {
+      throw new IllegalArgumentException("documents should be a list");
+    }
+
+    AddDocumentContext ctx = new AddDocumentContext();
+
+    AddDocumentHandler addDocHandler = (AddDocumentHandler) globalState.getHandler("addDocument");
+
+    // Parse any number of documents to update:
+    int count = 0;
+
+    while (true) {
+      JsonToken token = parser.nextToken();
+      if (token == JsonToken.END_ARRAY) {
+        break;
+      }
+      if (token != JsonToken.START_OBJECT) {
+        throw new IllegalArgumentException("missing object");
+      }
+
+      // Parse term: and fields:
+      Term updateTerm = null;
+
+      final Document doc = new Document();
+
+      while (true) {
+        token = parser.nextToken();
+        if (token == JsonToken.END_OBJECT) {
+          break;
+        }
+        if (token != JsonToken.FIELD_NAME) {
+          throw new IllegalArgumentException("missing field name");
+        }
+        String f = parser.getText();
+        if (f.equals("term")) {
+          if (parser.nextToken() != JsonToken.START_OBJECT) {
+            throw new IllegalArgumentException("missing object");
+          }
+
+          // TODO: allow field to be specified only once, then
+          // only text per document
+
+          String field=null, term=null;
+
+          while (parser.nextToken() != JsonToken.END_OBJECT) {
+            String f2 = parser.getText();
+            if (f2.equals("field")) {
+              if (parser.nextToken() != JsonToken.VALUE_STRING) {
+                throw new IllegalArgumentException("missing string value");
+              }
+              field = parser.getText();
+              // Ensure field is valid:
+              state.getField(field);
+            } else if (f2.equals("term")) {
+              if (parser.nextToken() != JsonToken.VALUE_STRING) {
+                throw new IllegalArgumentException("missing string value");
+              }
+              term = parser.getText();
+            } else {
+              throw new IllegalArgumentException("unexpected field " + f);
+            }
+          }
+          updateTerm = new Term(field, term);
+        } else if (f.equals("fields")) {
+          addDocHandler.parseFields(state, doc, parser);
+        } else {
+          boolean handled = false;
+          for(AddDocumentHandler.PostHandle postHandle : addDocHandler.postHandlers) {
+            if (postHandle.invoke(state, f, parser, doc)) {
+              handled = true;
+              break;
+            }
+          }
+          if (!handled) {
+            throw new IllegalArgumentException("unrecognized field " + parser.getText());
+          }
+        }
+      }
+
+      if (doc == null) {
+        throw new IllegalArgumentException("missing fields");
+      }
+      if (updateTerm == null) {
+        throw new IllegalArgumentException("missing term");
+      }
+
+      // TODO: this is dup'd code ... share better w/ AddDocHandler
+      globalState.indexService.submit(state.getAddDocumentJob(count, updateTerm, doc, ctx));
+      count++;
+    }
+    
+    // nocommit this is ... lameish:
+    while (true) {
+      if (ctx.addCount.get() == count) {
+        break;
+      }
+      Thread.sleep(1);
+    }
+
+    JSONObject o = new JSONObject();
+    o.put("indexGen", state.writer.getGeneration());
+    o.put("indexedDocumentCount", count);
+    return o.toString();
+  }
+}
+
+

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentsHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentsHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,238 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.*;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+import static org.apache.lucene.server.IndexState.AddDocumentContext;
+
+/** Reads more than one { ... } request in a single
+ *  connection, but each request must be separated by at
+ *  least one whitespace char. */
+
+public class BulkUpdateDocumentsHandler extends Handler {
+
+  private StructType TYPE = new StructType(
+                                     new Param("indexName", "Index name", new StringType()),
+                                     new Param("documents", "List of documents",
+                                         new ListType(
+                                             new StructType(
+                                                 new Param("term", "Identifies which document to replace", 
+                                                           new StructType(
+                                                                          new Param("field", "Field", new StringType()),
+                                                                          new Param("term", "Text", new StringType()))),
+                                                 new Param("parent", "The (one) parent document for this block.  The value of this key is a single document that @addDocument expects.  Be sure to add an indexed field to only the parent document so that you can subsequently provide the filter that identifies only parent documents.",
+                                                           AddDocumentHandler.DOCUMENT_TYPE),
+                                                 new Param("children", "List of child documents.",
+                                                           new ListType(AddDocumentHandler.DOCUMENT_TYPE))))));
+
+  /** Sole constructor. */
+  public BulkUpdateDocumentsHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public boolean doStream() {
+    return true;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Add more than one document block in a single connection.  Each document should be its own JSON struct, matching @addDocuments, and then there must one whitespace character separating each document.  Returns the index generation (indexGen) that contains all added document blocks.";
+  }
+
+  @Override
+  public FinishRequest handle(IndexState state, Request r, Map<String,List<String>> params) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String handleStreamed(Reader reader, Map<String,List<String>> params) throws Exception {
+
+    JsonFactory jfactory = new JsonFactory();
+
+    JsonParser parser = jfactory.createJsonParser(reader);
+
+    if (parser.nextToken() != JsonToken.START_OBJECT) {
+      throw new IllegalArgumentException("expected JSON object");
+    }
+    if (parser.nextToken() != JsonToken.FIELD_NAME) {
+      throw new IllegalArgumentException("expected indexName first");
+    }
+    if (!parser.getText().equals("indexName")) {
+      throw new IllegalArgumentException("expected indexName first");
+    }
+    if (parser.nextToken() != JsonToken.VALUE_STRING) {
+      throw new IllegalArgumentException("indexName should be string");
+    }
+
+    IndexState state = globalState.get(parser.getText());
+    state.verifyStarted(null);
+    if (parser.nextToken() != JsonToken.FIELD_NAME) {
+      throw new IllegalArgumentException("expected documents next");
+    }
+    if (!parser.getText().equals("documents")) {
+      throw new IllegalArgumentException("expected documents after indexName");
+    }
+
+    if (parser.nextToken() != JsonToken.START_ARRAY) {
+      throw new IllegalArgumentException("documents should be a list");
+    }
+
+    int count = 0;
+    AddDocumentContext ctx = new AddDocumentContext();
+
+    AddDocumentHandler addDocHandler = (AddDocumentHandler) globalState.getHandler("addDocument");
+
+    // Parse as many doc blocks as there are:
+    while (true) {
+
+      List<Document> children = null;
+      Document parent = null;
+      Term updateTerm = null;
+
+      JsonToken token = parser.nextToken();
+      if (token == JsonToken.END_ARRAY) {
+        break;
+      }
+      if (token != JsonToken.START_OBJECT) {
+        throw new IllegalArgumentException("expected object");
+      }
+
+      // Parse term + parent + children for this one doc block:
+      while(true) {
+        token = parser.nextToken();
+        if (token == JsonToken.END_OBJECT) {
+          // Done with parent + child in this block
+          break;
+        }
+        if (token != JsonToken.FIELD_NAME) {
+          throw new IllegalArgumentException("missing field name: " + token);
+        }
+        String f = parser.getText();
+        if (f.equals("term")) {
+          if (parser.nextToken() != JsonToken.START_OBJECT) {
+            throw new IllegalArgumentException("missing object");
+          }
+
+          // TODO: allow field to be specified only once, then
+          // only text per document
+
+          String field=null, term=null;
+
+          while (parser.nextToken() != JsonToken.END_OBJECT) {
+            String f2 = parser.getText();
+            if (f2.equals("field")) {
+              if (parser.nextToken() != JsonToken.VALUE_STRING) {
+                throw new IllegalArgumentException("missing string value");
+              }
+              field = parser.getText();
+              // Ensure field is valid:
+              state.getField(field);
+            } else if (f2.equals("term")) {
+              if (parser.nextToken() != JsonToken.VALUE_STRING) {
+                throw new IllegalArgumentException("missing string value");
+              }
+              term = parser.getText();
+            } else {
+              throw new IllegalArgumentException("unexpected field " + f);
+            }
+          }
+          updateTerm = new Term(field, term);
+        } else if (f.equals("children")) {
+          token = parser.nextToken();
+          if (token != JsonToken.START_ARRAY) {
+            throw new IllegalArgumentException("expected array for children");
+          }
+
+          children = new ArrayList<Document>();
+
+          // Parse each child:
+          while (true) {
+            Document doc = addDocHandler.parseDocument(state, parser);
+            if (doc == null) {
+              break;
+            }
+            children.add(doc);
+          }
+        } else if (f.equals("parent")) {
+          parent = addDocHandler.parseDocument(state, parser);
+        } else {
+          throw new IllegalArgumentException("unrecognized field name \"" + f + "\"");
+        }
+      }
+
+      if (parent == null) {
+        throw new IllegalArgumentException("missing parent");
+      }
+      if (children == null) {
+        throw new IllegalArgumentException("missing children");
+      }
+
+      // Parent is last:
+      children.add(parent);
+
+      globalState.indexService.submit(state.getAddDocumentsJob(count, updateTerm, children, ctx));
+      count++;
+    }
+
+    // nocommit this is ... lameish:
+    while (true) {
+      if (ctx.addCount.get() == count) {
+        break;
+      }
+      Thread.sleep(1);
+    }
+
+    JSONObject o = new JSONObject();
+    o.put("indexGen", state.writer.getGeneration());
+    o.put("indexedDocumentBlockCount", count);
+    if (!ctx.errors.isEmpty()) {
+      JSONArray errors = new JSONArray();
+      o.put("errors", errors);
+      for(int i=0;i<ctx.errors.size();i++) {
+        JSONObject err = new JSONObject();
+        errors.add(err);
+        err.put("index", ctx.errorIndex.get(i));
+        err.put("exception", ctx.errors.get(i));
+      }
+    }
+
+    return o.toString();
+  }
+}

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

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CommitHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CommitHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CommitHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CommitHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,60 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.*;
+
+/** Handles {@code commit}. */
+public class CommitHandler extends Handler {
+
+  private static StructType TYPE = new StructType(
+                                       new Param("indexName", "Index name", new StringType()));
+
+  /** Sole constructor. */
+  public CommitHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Commits all pending changes to durable storage.";
+  }
+  
+  @Override
+  public FinishRequest handle(final IndexState state, Request r, Map<String,List<String>> params) throws Exception {
+    return new FinishRequest() {
+      @Override
+      public String finish() throws IOException {
+        state.commit();
+        return "{}";
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CreateIndexHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CreateIndexHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CreateIndexHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CreateIndexHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,80 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.File;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.Param;
+import org.apache.lucene.server.params.Request;
+import org.apache.lucene.server.params.StringType;
+import org.apache.lucene.server.params.StructType;
+
+/** Handles {@code createIndex}. */
+public class CreateIndexHandler extends Handler {
+  private static StructType TYPE = new StructType(
+                                       new Param("indexName", "Index name", new StringType()),
+                                       new Param("rootDir", "Filesystem path where all state is stored", new StringType()));
+
+  /** Sole constructor. */
+  public CreateIndexHandler(GlobalState state) {
+    super(state);
+    requiresIndexName = false;
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Create an index";
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+    final String indexName = r.getString("indexName");
+    if (!IndexState.isSimpleName(indexName)) {
+      r.fail("indexName", "invalid indexName \"" + indexName + "\": must be [a-zA-Z_][a-zA-Z0-9]*");
+    }
+    final File rootDir;
+    if (r.hasParam("rootDir")) {
+      rootDir = new File(r.getString("rootDir"));
+    } else {
+      rootDir = null;
+    }
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws Exception {
+        try {
+          globalState.createIndex(indexName, rootDir);
+        } catch (IllegalArgumentException iae) {
+          r.fail("invalid indexName \"" + indexName + "\": " + iae.toString(), iae);
+        }
+
+        return "{}";
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CreateSnapshotHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CreateSnapshotHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CreateSnapshotHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/CreateSnapshotHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,136 @@
+package org.apache.lucene.server.handlers;
+
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager.SearcherAndTaxonomy;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.MyIndexSearcher;
+import org.apache.lucene.server.params.BooleanType; 
+import org.apache.lucene.server.params.Param; 
+import org.apache.lucene.server.params.Request; 
+import org.apache.lucene.server.params.StringType; 
+import org.apache.lucene.server.params.StructType; 
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+/** Handles {@code createSnapshot}. */
+public class CreateSnapshotHandler extends Handler {
+
+  final static StructType TYPE = new StructType(
+                               new Param("indexName", "Index Name", new StringType()),
+                               new Param("openSearcher", "Pass true if you intend to do searches against this snapshot, by passing searcher: {snapshot: X} to @search", new BooleanType(), false));
+                                          
+  @Override
+  public String getTopDoc() {
+    return "Creates a snapshot in the index, which is saved point-in-time view of the last commit in the index such that no files referenced by that snapshot will be deleted by ongoing indexing until the snapshot is released with @releaseSnapshot.  Note that this will reference the last commit, so be sure to call commit first if you have pending changes that you'd like to be included in the snapshot.<p>This can be used for backup purposes, i.e. after creating the snapshot you can copy all referenced files to backup storage, and then release the snapshot once complete.  To restore the backup, just copy all the files back and restart the server.  It can also be used for transactional purposes, i.e. if you sometimes need to search a specific snapshot instead of the current live index.<p>Creating a snapshot is very fast (does not require any file copying), but over time it will consume extra disk space as old segments are merged in the index.  Be sure to release the snapshot once you're
  done.  Snapshots survive shutdown and restart of the server.  Returns all protected filenames referenced by this snapshot: these files will not change and will not be deleted until the snapshot is released.  This returns the directories and files referenced by the snapshot.";
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  /** Sole constructor. */
+  public CreateSnapshotHandler(GlobalState state) {
+    super(state);
+  }
+
+  static void fillFiles(JSONObject o, String path, IndexCommit commit) throws IOException {
+    JSONArray arr = new JSONArray();
+    for(String sub : commit.getFileNames()) {
+      arr.add(sub);
+    }
+    o.put(path, arr);
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, Request r, Map<String,List<String>> params) throws Exception {
+    state.verifyStarted(r);
+
+    if (!state.hasCommit()) {
+      r.fail("this index has no commits; please call commit first");
+    }
+
+    // nocommit not thread safe vs commitHandler?
+    final boolean openSearcher = r.getBoolean("openSearcher");
+    
+    return new FinishRequest() {
+      @Override
+      public String finish() throws IOException {
+        IndexCommit c = state.snapshots.snapshot();
+        IndexCommit tc = state.taxoSnapshots.snapshot();
+        long stateGen = state.incRefLastCommitGen();
+
+        JSONObject result = new JSONObject();
+        
+        SegmentInfos sis = new SegmentInfos();
+        sis.read(state.origIndexDir, c.getSegmentsFileName());
+        state.snapshotGenToVersion.put(c.getGeneration(), sis.getVersion());
+
+        if (openSearcher) {
+          // nocommit share w/ SearchHandler's method:
+          // TODO: this "reverse-NRT" is silly ... we need a reader
+          // pool somehow:
+          SearcherAndTaxonomy s2 = state.manager.acquire();
+          try {
+            // This returns a new reference to us, which
+            // is decRef'd in the finally clause after
+            // search is done:
+            long t0 = System.nanoTime();
+            IndexReader r = DirectoryReader.openIfChanged((DirectoryReader) s2.searcher.getIndexReader(), c);
+            IndexSearcher s = new MyIndexSearcher(r, state);
+            try {
+              state.slm.record(s);
+            } finally {
+              s.getIndexReader().decRef();
+            }
+            long t1 = System.nanoTime();
+            result.put("newSnapshotSearcherOpenMS", ((t1-t0)/1000000.0));
+          } finally {
+            state.manager.release(s2);
+          }
+        }
+
+        // TODO: suggest state?
+
+        // nocommit must also snapshot snapshots state!?
+        // hard to think about
+
+        fillFiles(result, "index", c);
+        fillFiles(result, "taxonomy", tc);
+        JSONArray arr = new JSONArray();
+        arr.add("state." + stateGen);
+        result.put("state", arr);
+        result.put("id", c.getGeneration() + ":" + tc.getGeneration() + ":" + stateGen);
+
+        return result.toString();
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteAllDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteAllDocumentsHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteAllDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteAllDocumentsHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,65 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.*;
+import net.minidev.json.JSONObject;
+
+/** Handles {@code deleteAllDocuments}. */
+public class DeleteAllDocumentsHandler extends Handler {
+
+  private final static StructType TYPE =
+    new StructType(new Param("indexName", "Which index to search", new StringType()));
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Removes all documents in the index, but keeps all registered fields, settings and any built suggesters.";
+  }
+
+  /** Sole constructor. */
+  public DeleteAllDocumentsHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, Request r, Map<String,List<String>> params) throws Exception {
+    state.verifyStarted(r);
+    return new FinishRequest() {
+      @Override
+      public String finish() throws IOException {
+        // nocommit should also somehow reset taxo index?
+        long gen = state.writer.deleteAll();
+        JSONObject r = new JSONObject();
+        r.put("indexGen", gen);
+        return r.toString();
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteDocumentsHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteDocumentsHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,81 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.ListType;
+import org.apache.lucene.server.params.Param;
+import org.apache.lucene.server.params.Request;
+import org.apache.lucene.server.params.StringType;
+import org.apache.lucene.server.params.StructType;
+import net.minidev.json.JSONObject;
+
+/** Handles {@code deleteDocuments}. */
+public class DeleteDocumentsHandler extends Handler {
+  // TODO: support delete by query too:
+  // TODO: support bulk api?
+  final static StructType TYPE = new StructType(
+                               new Param("indexName", "Index name", new StringType()),
+                               new Param("field", "Field to match to identify the deleted documents.", new StringType()),
+                               new Param("values", "Values to delete.", new ListType(new StringType())));
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Delete documents.  Returns the index generation (indexGen) that reflext the deletion.";
+  }
+
+  /** Sole constructor. */
+  public DeleteDocumentsHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+    final String field = r.getString("field");
+    final List<Object> ids = r.getList("values");
+    final Term[] terms = new Term[ids.size()];
+    for(int i=0;i<terms.length;i++) {
+      // TODO: how to allow arbitrary binary keys?  how to
+      // pass binary data via json...?  byte array?
+      terms[i] = new Term(field, (String) ids.get(i));
+    }
+    state.verifyStarted(r);
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws IOException {
+        state.writer.deleteDocuments(terms);
+        JSONObject o = new JSONObject();
+        o.put("indexGen", state.writer.getGeneration());
+        return o.toString();
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteIndexHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteIndexHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteIndexHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteIndexHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,64 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.params.Param;
+import org.apache.lucene.server.params.Request;
+import org.apache.lucene.server.params.StringType;
+import org.apache.lucene.server.params.StructType;
+
+/** Handles {@code deleteIndex}. */
+public class DeleteIndexHandler extends Handler {
+  private static StructType TYPE = new StructType(
+                                       new Param("indexName", "Index name", new StringType()));
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Delete an index";
+  }
+
+  /** Sole constructor. */
+  public DeleteIndexHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+    return new FinishRequest() {
+      @Override
+      public String finish() throws IOException {
+        state.close();
+        state.deleteIndex();
+
+        return "{}";
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DocHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DocHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DocHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/DocHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,258 @@
+package org.apache.lucene.server.handlers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.server.params.FloatType;
+import org.apache.lucene.server.params.IntType;
+import org.apache.lucene.server.params.ListType;
+import org.apache.lucene.server.params.LongType;
+import org.apache.lucene.server.params.OrType;
+import org.apache.lucene.server.params.Param;
+import org.apache.lucene.server.params.PolyType;
+import org.apache.lucene.server.params.StringType;
+import org.apache.lucene.server.params.StructType;
+import org.apache.lucene.server.params.Type;
+import org.apache.lucene.server.params.WrapType;
+
+// TODO: use some ... standard markup language/processor!
+
+/** Handles generating live documentation, accessible via
+ *  http://localhost:4000/doc", from all registered
+ *  handlers. */
+public class DocHandler {
+
+  /** Sole constructor. */
+  public DocHandler() {
+  }
+
+  // nocommit this is wasteful ... use Apache commons?
+  private static String escapeHTML(String s) {
+    s = s.replaceAll("&", "&amp;");
+    s = s.replaceAll(">", "&gt;");
+    s = s.replaceAll("<", "&lt;");
+    s = s.replaceAll("\"", "&quot;");
+    return s;
+  }
+
+  private final static Pattern reLink = Pattern.compile("@([a-zA-Z0-9:\\-\\.]+)\\b");
+
+  private final static String LUCENE_DOC_VERSION = "4_2_0";
+
+  private static String expandLinks(String s) {
+    StringBuilder sb = new StringBuilder();
+    int upto = 0;
+    Matcher m = reLink.matcher(s);
+    while(true) {
+      if (!m.find(upto)) {
+        sb.append(s.substring(upto));
+        break;
+      }
+      String group = m.group(1);
+      String url;
+      String text;
+      if (group.startsWith("lucene:")) {
+        // External lucene link:
+        String[] parts = group.split(":");
+        if (parts.length == 3) {
+          url = "http://lucene.apache.org/core/" + LUCENE_DOC_VERSION + "/" + parts[1] + "/" + parts[2].replace('.', '/') + ".html";
+          text = "Lucene Javadocs";
+        } else {
+          throw new IllegalArgumentException("malformed lucene link: " + group);
+        }
+      } else {
+        // Internal link:
+        text = group;
+        url = "/doc?method=" + group;
+      }
+      sb.append(s.substring(upto, m.start(1)-1));
+      sb.append("<a href=\"");
+      sb.append(url);
+      sb.append("\">");
+      sb.append(text);
+      sb.append("</a>");
+      upto = m.end(1);
+    }
+    return sb.toString();
+  }
+
+  /** Generates documentation for the provided handlers.  If
+   *  params is non-empty, generates documentation for a
+   *  specific method. */
+  public String handle(Map<String,List<String>> params, Map<String,Handler> handlers) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("<html>");
+    sb.append("<head>");
+    sb.append("<meta http-equiv=\"content-type\" content=\"text/html; charset=UTF-8\">");
+    sb.append("</head>");
+    sb.append("<body>");
+    if (params.size() == 0) {
+      sb.append("<h1>Lucene Server</h1>");
+      sb.append("Available methods:\n");
+      sb.append("<dd>");
+      List<String> keys = new ArrayList<String>(handlers.keySet());
+      Collections.sort(keys);
+      for(String key : keys) {
+        sb.append("<dt><a href=\"/doc?method=");
+        sb.append(key);
+        sb.append("\">");
+        sb.append(escapeHTML(key));
+        sb.append("</a>");
+        sb.append("</dt>");
+        sb.append("<dd>");
+        Handler handler = handlers.get(key);
+        sb.append(expandLinks(handler.getTopDoc()));
+        sb.append("</dd>");
+      }
+      sb.append("</dd>");
+    } else {
+      String method = params.get("method").get(0);
+      Handler handler = handlers.get(method);
+      if (handler == null) {
+        throw new IllegalArgumentException("unknown method \"" + method + "\"");
+      }
+      sb.append("<h1>Lucene Server: ");
+      sb.append(method);
+      sb.append("</h1>");
+      sb.append(expandLinks(handler.getTopDoc()));
+      sb.append("<br>");
+      sb.append("<br><b>Parameters</b>:<br>");
+      StructType type = handler.getType();
+      Set<StructType> seen = Collections.newSetFromMap(new IdentityHashMap<StructType,Boolean>());
+      renderStructType(seen, sb, type);
+    }
+    sb.append("</body>");
+    sb.append("</html>");
+    return sb.toString();
+  }
+
+  private String simpleTypeToString(Type type) {
+    if (type instanceof IntType) {
+      return "int";
+    } else if (type instanceof FloatType) {
+      return "float";
+    } else if (type instanceof LongType) {
+      return "long";
+    } else if (type instanceof StringType) {
+      return "string";
+    } else if (type instanceof OrType) {
+      String s1 = null;
+      for(Type subType : ((OrType) type).types) {
+        String sub = simpleTypeToString(subType);
+        if (sub != null) {
+          if (s1 == null) {
+            s1 = sub;
+          } else {
+            s1 += " or " + sub;
+          }
+        } else {
+          return null;
+        }
+      }
+      return s1;
+    } else if (type instanceof ListType) {
+      String s1 = simpleTypeToString(((ListType) type).subType);
+      if (s1 != null) {
+        return "List of " + s1;
+      } else {
+        return null;
+      }
+    } else {
+      return null;
+    }
+  }
+
+  private void renderStructType(Set<StructType> seen, StringBuilder sb, StructType type) {
+    if (seen.contains(type)) {
+      sb.append("<br><br>See <a href=\"\">here</a>.");
+      return;
+    }
+    seen.add(type);
+    sb.append("<dl>");
+    List<String> args = new ArrayList<String>(type.params.keySet());
+    Collections.sort(args);
+    for(String arg : args) {
+      Param p = type.params.get(arg);
+      sb.append("<br>");
+      sb.append("<dt><code><b>");
+      sb.append(escapeHTML(arg));
+      sb.append("</b>");
+      String s = simpleTypeToString(p.type);
+      if (p.defaultValue != null) {
+        if (s != null) {
+          s += "; default: ";
+        } else {
+          s = "default: ";
+        }
+        s += p.defaultValue;
+      }
+      if (s != null) {
+        sb.append(" (");
+        sb.append(s);
+        sb.append(")");
+      }
+      sb.append("</code></dt>");
+      sb.append("<dd>");
+      sb.append(expandLinks(p.desc));
+
+      Type subType = p.type;
+      if (subType instanceof WrapType) {
+        subType = ((WrapType) subType).getWrappedType();
+      }
+
+      if (subType instanceof ListType) {
+        ListType lt = (ListType) subType;
+        sb.append("<br><br>");
+        if (lt.subType instanceof StructType) {
+          sb.append("List of:");
+          renderStructType(seen, sb, (StructType) lt.subType);
+        }
+      } else if (subType instanceof StructType) {
+        renderStructType(seen, sb, (StructType) subType);
+      } else if (subType instanceof PolyType) {
+        PolyType pt = (PolyType) subType;
+        List<String> polyKeys = new ArrayList<String>(pt.types.keySet());
+        Collections.sort(polyKeys);
+        sb.append("<dl>");
+        for(String key : polyKeys) {
+          sb.append("<dt>= ");
+          sb.append("<code>");
+          sb.append(escapeHTML(key));
+          sb.append("</code>");
+          sb.append("</dt>");
+          sb.append("<dd>");
+          sb.append(expandLinks(pt.types.get(key).desc));
+          renderStructType(seen, sb, pt.types.get(key).type);
+          sb.append("</dd>");
+        }
+        sb.append("</dl>");
+      }
+      sb.append("</dd>");
+    }
+    sb.append("</dl>");
+  }
+}
+