You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/12/24 14:39:24 UTC

svn commit: r1553272 [4/10] - in /lucene/dev/branches/lucene5376/lucene/server: ./ plugins/ plugins/BinaryDocument/ plugins/BinaryDocument/src/ plugins/BinaryDocument/src/java/ plugins/BinaryDocument/src/java/org/ plugins/BinaryDocument/src/java/org/ap...

Added: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BuildSuggestHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,404 @@
+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.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.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+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 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;
+
+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 break out separate TYPEs for each impl
+        // nocommit option to stream suggestions in over the wire too
+        new Param("localFile", "File to read suggestions + weights from; format is weight:suggestion, one per line, with suggestion UTF-8 encoded.", new StringType()),
+        new Param("suggestName", "Unique name for this suggest build.", new StringType())
+                   );
+
+  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.";
+  }
+
+  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();
+
+      Request r = new Request(null, null, params, TYPE);
+      // Must consume these up front since getSuggester
+      // won't:
+      r.getString("suggestName");
+      r.getString("localFile");
+      Lookup suggester = getSuggester(state, suggestName, r);
+      assert !Request.anythingLeft(params);
+
+      if (!(suggester instanceof AnalyzingInfixSuggester)) {
+        File path = new File(state.rootDir, "suggest." + suggestName);
+        FileInputStream in = new FileInputStream(path);
+        try {
+          suggester.load(in);
+        } finally {
+          in.close();
+        }
+      }
+    }
+  }
+
+  private Lookup getSuggester(IndexState state, String suggestName, Request r) throws IOException {
+
+    Request.PolyResult pr = r.getPoly("class");
+    
+    String impl = pr.name;
+
+    r = pr.r;
+    Analyzer indexAnalyzer;
+    Analyzer queryAnalyzer;
+    if (r.hasParam("analyzer")) {
+      indexAnalyzer = queryAnalyzer = RegisterFieldHandler.getAnalyzer(state.matchVersion, r, "analyzer");
+    } else {
+      indexAnalyzer = RegisterFieldHandler.getAnalyzer(state.matchVersion, r, "indexAnalyzer");
+      queryAnalyzer = RegisterFieldHandler.getAnalyzer(state.matchVersion, 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 Query finishQuery(BooleanQuery in, boolean allTermsRequired) {
+            // nocommit not general
+            List<BooleanClause> clauses = in.clauses();
+            if (clauses.size() >= 2 && allTermsRequired) {
+              String t1 = getTerm(clauses.get(clauses.size()-2).getQuery());
+              String t2 = getTerm(clauses.get(clauses.size()-1).getQuery());
+              if (t1.equals(t2)) {
+                BooleanQuery sub = new BooleanQuery();
+                BooleanClause other = clauses.get(clauses.size()-2);
+                sub.add(new BooleanClause(clauses.get(clauses.size()-2).getQuery(), BooleanClause.Occur.SHOULD));
+                sub.add(new BooleanClause(clauses.get(clauses.size()-1).getQuery(), BooleanClause.Occur.SHOULD));
+                clauses.subList(clauses.size()-2, clauses.size()).clear();
+                clauses.add(new BooleanClause(sub, BooleanClause.Occur.MUST));
+              }
+            }
+            return in;
+          }
+
+          private String getTerm(Query query) {
+            if (query instanceof TermQuery) {
+              return ((TermQuery) query).getTerm().text();
+            } else if (query instanceof PrefixQuery) {
+              return ((PrefixQuery) query).getPrefix().text();
+            } else {
+              return null;
+            }
+          }
+          */
+          
+          /*
+          @Override
+          protected void addNonMatch(StringBuilder sb, String text) {
+            if (sb.size() > 0) {
+              sb.append(',');
+            }
+            sb.append('"');
+          }
+
+          @Override
+          protected void addPrefixMatch(StringBuilder sb, String surface, String analyzed, String prefixToken) {
+            prefixToken = prefixToken.toLowerCase();
+            String surfaceLower = surface.toLowerCase();
+            sb.append("<font color=red>");
+            if (surfaceLower.startsWith(prefixToken)) {
+              sb.append(surface.substring(0, prefixToken.length()));
+              sb.append("</font>");
+              sb.append(surface.substring(prefixToken.length()));
+            } else {
+              sb.append(surface);
+              sb.append("</font>");
+            }
+          }
+
+          @Override
+          protected void addWholeMatch(StringBuilder sb, String surface, String analyzed) {
+            sb.append("<font color=red>");
+            sb.append(surface);
+            sb.append("</font>");
+          }
+          */
+
+          @Override
+          protected Object highlight(String text, Set<String> matchedTokens, String prefixToken) throws IOException {
+            TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text));
+            CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+            OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
+            ts.reset();
+            List<LookupHighlightFragment> fragments = new ArrayList<LookupHighlightFragment>();
+            int upto = 0;
+            while (ts.incrementToken()) {
+              String token = termAtt.toString();
+              int startOffset = offsetAtt.startOffset();
+              int endOffset = offsetAtt.endOffset();
+              if (upto < startOffset) {
+                fragments.add(new LookupHighlightFragment(text.substring(upto, startOffset), false));
+                upto = startOffset;
+              } else if (upto > startOffset) {
+                continue;
+              }
+
+              if (matchedTokens.contains(token)) {
+                // Token matches.
+                fragments.add(new LookupHighlightFragment(text.substring(startOffset, endOffset), true));
+                upto = endOffset;
+              } else if (prefixToken != null && token.startsWith(prefixToken)) {
+                fragments.add(new LookupHighlightFragment(text.substring(startOffset, startOffset+prefixToken.length()), true));
+                if (prefixToken.length() < token.length()) {
+                  fragments.add(new LookupHighlightFragment(text.substring(startOffset+prefixToken.length(), startOffset+token.length()), false));
+                }
+                upto = endOffset;
+              }
+            }
+            ts.end();
+            int endOffset = offsetAtt.endOffset();
+            if (upto < endOffset) {
+              fragments.add(new LookupHighlightFragment(text.substring(upto), false));
+            }
+            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 + ")";
+    }
+  }
+
+  @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 File localFile = new File(r.getString("localFile"));
+    if (!localFile.exists()) {
+      r.fail("localFile", "file does not exist");
+    }
+    if (!localFile.canRead()) {
+      r.fail("localFile", "cannot read file");
+    }
+
+    final Lookup suggester = getSuggester(state, suggestName, r);
+
+    FromFileTermFreqIterator iterator = null;
+    try {
+      iterator = new FromFileTermFreqIterator(localFile);
+    } catch (IOException ioe) {
+      r.fail("localFile", "cannot open file", ioe);
+    }
+
+    // nocommit return error if suggester already exists?
+
+    // nocommit need a DeleteSuggestHandler
+
+    final FromFileTermFreqIterator finalIterator = iterator;    
+
+    return new FinishRequest() {
+
+      @Override
+      public String finish() throws IOException {
+
+        try {
+          suggester.build(finalIterator);
+        } finally {
+          finalIterator.close();
+        }
+
+        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();
+          }
+        }
+
+        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());
+        }
+        ret.put("count", finalIterator.suggestCount);
+        return ret.toString();
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,143 @@
+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 java.util.concurrent.atomic.AtomicInteger;
+
+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;
+import static org.apache.lucene.server.IndexState.DocumentAndFacets;
+
+/** 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)));
+
+  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());
+    if (!state.started()) {
+      throw new IllegalArgumentException("index \"" + state.name + "\" isn't started: cannot add document");
+    }
+    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) {
+      DocumentAndFacets 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/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentsHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkAddDocumentsHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,205 @@
+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 java.util.concurrent.atomic.AtomicInteger;
+
+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;
+import static org.apache.lucene.server.IndexState.DocumentAndFacets;
+
+/** 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))))));
+
+  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());
+    if (!state.started()) {
+      throw new IllegalArgumentException("index \"" + state.name + "\" isn't started: cannot add documents");
+    }
+    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<DocumentAndFacets> children = null;
+      DocumentAndFacets 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<DocumentAndFacets>();
+
+          // Parse each child:
+          while (true) {
+            DocumentAndFacets 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/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,221 @@
+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 java.util.concurrent.atomic.AtomicInteger;
+
+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.AnyType;
+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.apache.lucene.server.params.Type;
+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;
+import static org.apache.lucene.server.IndexState.DocumentAndFacets;
+
+public class BulkUpdateDocumentHandler extends Handler {
+
+  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)));
+
+  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 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());
+    if (!state.started()) {
+      throw new IllegalArgumentException("index \"" + state.name + "\" isn't started: cannot update document");
+    }
+
+    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 DocumentAndFacets doc = new DocumentAndFacets();
+
+      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/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentsHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/BulkUpdateDocumentsHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,240 @@
+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 java.util.concurrent.atomic.AtomicInteger;
+
+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;
+import static org.apache.lucene.server.IndexState.DocumentAndFacets;
+
+/** 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))))));
+
+  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());
+    if (!state.started()) {
+      throw new IllegalArgumentException("index \"" + state.name + "\" isn't started: cannot add documents");
+    }
+    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<DocumentAndFacets> children = null;
+      DocumentAndFacets 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<DocumentAndFacets>();
+
+          // Parse each child:
+          while (true) {
+            DocumentAndFacets 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/lucene/server/src/java/org/apache/lucene/server/handlers/CommitHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CommitHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CommitHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CommitHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,58 @@
+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.*;
+
+public class CommitHandler extends Handler {
+
+  private static StructType TYPE = new StructType(
+                                       new Param("indexName", "Index name", new StringType()));
+
+  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/lucene/server/src/java/org/apache/lucene/server/handlers/CreateIndexHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CreateIndexHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CreateIndexHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CreateIndexHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,76 @@
+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;
+import org.apache.lucene.server.params.Type;
+
+public class CreateIndexHandler extends Handler {
+  private static StructType TYPE = new StructType(
+                                       new Param("indexName", "Index name", new StringType()),
+                                       // nocommit what about RAMDir...
+                                       // nocommit should we only allow rootDir "under" the globalRoot?
+                                       new Param("rootDir", "Filesystem path where all state is stored", new StringType()));
+
+  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 String rootDir = r.getString("rootDir");
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws Exception {
+        try {
+          globalState.createIndex(indexName, new File(rootDir));
+        } catch (IllegalArgumentException iae) {
+          r.fail("invalid indexName \"" + indexName + "\": " + iae.toString());
+        }
+
+        return "{}";
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CreateSnapshotHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CreateSnapshotHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CreateSnapshotHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/CreateSnapshotHandler.java Tue Dec 24 13:39:22 2013
@@ -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.search.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 org.apache.lucene.server.params.Type; 
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+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;
+  }
+
+  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 {
+    if (!state.hasCommit()) {
+      r.fail("this index has no commits; please call commit first");
+    }
+
+    if (!state.started()) { 
+      r.fail("this index hasn't been started yet");
+    }
+
+    // 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) {
+          // 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);
+            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/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteAllDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteAllDocumentsHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteAllDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteAllDocumentsHandler.java Tue Dec 24 13:39:22 2013
@@ -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;
+
+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.";
+  }
+
+  public DeleteAllDocumentsHandler(GlobalState state) {
+    super(state);
+  }
+
+  @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 {
+        if (!state.started()) {
+          throw new IllegalArgumentException("index \"" + state.name + "\" isn't started: cannot add document");
+        }
+        // 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/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteDocumentsHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteDocumentsHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,82 @@
+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 org.apache.lucene.server.params.Type;
+import net.minidev.json.JSONObject;
+
+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.";
+  }
+
+  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));
+    }
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws IOException {
+        if (!state.started()) {
+          r.fail("indexName",  "call startIndex first");
+        }
+        state.writer.deleteDocuments(terms);
+        JSONObject o = new JSONObject();
+        o.put("indexGen", state.writer.getGeneration());
+        return o.toString();
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteIndexHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteIndexHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteIndexHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DeleteIndexHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,67 @@
+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;
+import org.apache.lucene.server.params.Type;
+
+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";
+  }
+
+  public DeleteIndexHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+    if (state.started()) {
+      r.fail("index \"" + state.name + "\" cannot be deleted: it is still running; call stopIndex first");
+    }
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws IOException {
+        state.close();
+        state.deleteIndex();
+
+        return "{}";
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DocHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DocHandler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DocHandler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/DocHandler.java Tue Dec 24 13:39:22 2013
@@ -0,0 +1,248 @@
+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!
+
+public class 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();
+  }
+
+  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>");
+  }
+}
+

Added: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/Handler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/Handler.java?rev=1553272&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/Handler.java (added)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/Handler.java Tue Dec 24 13:39:22 2013
@@ -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.Reader;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+import org.apache.lucene.server.FinishRequest;
+import org.apache.lucene.server.GlobalState;
+import org.apache.lucene.server.IndexState;
+import org.apache.lucene.server.PreHandle;
+import org.apache.lucene.server.params.*;
+
+// nocommit needs ChannelHandlerContext too?
+
+public abstract class Handler {
+
+  /** Processes request into a FinishRequest, which is then
+   *  invoked to actually make changes.  We do this two-step
+   *  process so that we can fail if there are unhandled
+   *  params, without having made any changes to the index. */
+  public abstract FinishRequest handle(IndexState state, Request request, Map<String,List<String>> params) throws Exception;
+  public abstract StructType getType();
+  public abstract String getTopDoc();
+
+  protected final GlobalState globalState;
+  
+  public boolean requiresIndexName = true;
+
+  protected Handler(GlobalState globalState) {
+    this.globalState = globalState;
+  }
+
+  public boolean doStream() {
+    return false;
+  }
+
+  public String handleStreamed(Reader reader, Map<String,List<String>> params) throws Exception {
+    throw new UnsupportedOperationException();
+  }
+
+  public final List<PreHandle> preHandlers = new CopyOnWriteArrayList<PreHandle>();
+
+  public synchronized void addPreHandle(PreHandle h) {
+    preHandlers.add(h);
+  }
+}