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 [3/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/SVJSONPassageFormatter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/SVJSONPassageFormatter.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/SVJSONPassageFormatter.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/SVJSONPassageFormatter.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,159 @@
+package org.apache.lucene.server;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.text.BreakIterator;
+import java.util.Locale;
+
+import org.apache.lucene.search.postingshighlight.Passage;
+import org.apache.lucene.search.postingshighlight.PassageFormatter;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+// nocommit move something like this to Lucene?
+
+/** Formats highlight hits for single-valued fields,
+ *  trimming snippets that are too large at word
+ *  boundaries. */
+public class SVJSONPassageFormatter extends PassageFormatter {
+  private final int maxSnippetLength;
+  private final BreakIterator wordBI = BreakIterator.getWordInstance(Locale.ROOT);
+
+  /** Sole constructor. */
+  public SVJSONPassageFormatter(int maxSnippetLength) {
+    this.maxSnippetLength = maxSnippetLength;
+  }
+
+  @Override
+  public Object format(Passage[] passages, String content) {
+    JSONArray result = new JSONArray();
+
+    int pos = 0;
+    JSONArray snippet = null;
+
+    Passage lastPassage = null;
+    JSONObject lastSnippetObj = null;
+
+    int totLength = 0;
+
+    for (Passage passage : passages) {
+      if (snippet == null || passage.getStartOffset() > pos) {
+        if (lastPassage != null) {
+          lastSnippetObj.put("endOffset", lastPassage.getEndOffset());
+        }
+
+        // Make a new snippet
+        JSONObject snippetObj = new JSONObject();
+        lastSnippetObj = snippetObj;
+        snippetObj.put("startOffset", passage.getStartOffset());
+        snippet = new JSONArray();
+        snippetObj.put("parts", snippet);
+        result.add(snippetObj);
+        pos = passage.getStartOffset();
+        totLength = 0;
+      }
+
+      // TODO: make this 4 settable
+      int limit = Math.min(4, passage.getNumMatches());
+      
+      for (int i = 0; i < limit; i++) {
+        int start = passage.getMatchStarts()[i];
+        // Must at least start within passage:
+        assert start < passage.getEndOffset();
+        int end = passage.getMatchEnds()[i];
+        // it's possible to have overlapping terms
+        if (start > pos) {
+          String s = trim(content.substring(pos, start), i>0, true);
+          totLength += s.length();
+          snippet.add(s);
+          pos = start;
+        }
+        if (end > pos) {
+          JSONObject hit = new JSONObject();
+          snippet.add(hit);
+          hit.put("term", passage.getMatchTerms()[i].utf8ToString());
+          String s = content.substring(Math.max(pos, start), end);
+          totLength += s.length();
+          hit.put("text", s);
+          pos = end;
+        }
+        // TODO: make this 3*maxSnippetLength settable
+        if (totLength > 3*maxSnippetLength) {
+          break;
+        }
+      }
+      if (totLength < 3*maxSnippetLength) {
+        // its possible a "term" from the analyzer could span a sentence boundary.
+        snippet.add(trim(content.substring(pos, Math.max(pos, passage.getEndOffset())), passage.getNumMatches() != 0, false));
+      }
+      pos = passage.getEndOffset();
+      lastPassage = passage;
+    }
+
+    if (lastPassage != null) {
+      lastSnippetObj.put("endOffset", lastPassage.getEndOffset());
+    }
+
+    return result;
+  }
+
+  /** Find last word boundary before offset. */
+  private int wordBack(int offset) {
+    int x = wordBI.preceding(offset);
+    if (x < offset-15) {
+      x = offset;
+    }
+    if (x < 0) {
+      x = 0;
+    }
+    return x;
+  }
+
+  /** Find next word boundary after offset. */
+  private int wordForwards(int offset) {
+    int x = wordBI.following(offset);
+    if (x > offset+15) {
+      x = offset;
+    }
+    return x;
+  }
+
+  private String trim(String in, boolean hasMatchBeforeStart, boolean hasMatchAfterEnd) {
+    if (in.length() <= maxSnippetLength) {
+      return in;
+    }
+
+    wordBI.setText(in);
+
+    if (hasMatchBeforeStart) {
+      if (hasMatchAfterEnd) {
+        if (in.length() <= 2*maxSnippetLength) {
+          return in;
+        } else {
+          return in.substring(0, wordBack(maxSnippetLength)) + " ... " + in.substring(wordForwards(in.length()-maxSnippetLength));
+        }
+      } else {
+        return in.substring(0, wordBack(maxSnippetLength)) + " ... ";
+      }
+    } else if (hasMatchAfterEnd) {
+      return " ... " + in.substring(wordForwards(in.length()-maxSnippetLength));
+    } else {
+      return in.substring(0, wordBack(maxSnippetLength)) + " ... ";
+    }
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/Server.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/Server.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/Server.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/Server.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,707 @@
+package org.apache.lucene.server;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.server.handlers.*;
+import org.apache.lucene.server.http.HttpStaticFileServerHandler;
+import org.apache.lucene.server.params.*;
+import org.apache.lucene.server.params.PolyType.PolyEntry;
+import org.apache.lucene.util.NamedThreadFactory;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.channel.UpstreamMessageEvent;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.http.DefaultHttpRequest;
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
+import org.jboss.netty.handler.codec.http.HttpChunk;
+//import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
+import org.jboss.netty.handler.codec.http.HttpContentCompressor;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.jboss.netty.handler.codec.http.HttpVersion;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.jboss.netty.util.CharsetUtil;
+import net.minidev.json.JSONObject;
+import net.minidev.json.JSONStyleIdent;
+import net.minidev.json.parser.ContainerFactory;
+import net.minidev.json.parser.JSONParser;
+import net.minidev.json.parser.ParseException;
+
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+//import static org.jboss.netty.handler.codec.http.HttpHeaders.*;
+//import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.*;
+//import static org.jboss.netty.handler.codec.http.HttpMethod.*;
+
+
+// nocommit move under http
+
+/** Main entry point for the HTTP server. */
+public class Server {
+
+  final GlobalState globalState;
+
+  final SimpleChannelUpstreamHandler staticFileHandler;
+
+  /** The actual port server bound to (only interesting if
+   *  you passed port=0 to let the OS assign one). */
+  public int actualPort;
+
+  /** Handles the incoming request. */
+  private class Dispatcher extends SimpleChannelUpstreamHandler {
+
+    private Writer pipedWriter;
+    private Throwable[] pipedExc;
+    private Thread pipeThread;
+    private boolean firstMessage = true;
+    private final String[] pipedResult = new String[1];
+    private ByteArrayOutputStream chunkedRequestBytes;
+    private String command;
+    private Handler handler;
+    private boolean doKeepAlive = true;
+    private Map<String,List<String>> params;
+
+    private byte[] processRequest(String request) throws Exception {
+      //System.out.println("request: " + request + " this=" + this);
+      JSONObject requestData;
+      if (request != null) {
+        Object o = null;
+        try {
+          //System.out.println("request " + request);
+          o = new JSONParser(JSONParser.MODE_STRICTEST).parse(request, ContainerFactory.FACTORY_SIMPLE);
+        } catch (ParseException pe) {
+          IllegalArgumentException iae = new IllegalArgumentException("could not parse HTTP request data as JSON");
+          iae.initCause(pe);
+          throw iae;
+        }
+        if (!(o instanceof JSONObject)) {
+          throw new IllegalArgumentException("HTTP request data must be a JSON struct { .. }");
+        }
+        requestData = (JSONObject) o;
+      } else {
+        requestData = new JSONObject();
+      }
+
+      Request r = new Request(null, command, requestData, handler.getType());
+      IndexState state;
+      if (handler.requiresIndexName) {
+        String indexName = r.getString("indexName");
+        state = globalState.get(indexName);
+      } else {
+        state = null;
+      }
+
+      FinishRequest finish;
+      try {
+        for(PreHandle h : handler.preHandlers) {
+          h.invoke(r);
+        }
+        // TODO: for "compute intensive" (eg search)
+        // handlers, we should use a separate executor?  And
+        // we should more gracefully handle the "Too Busy"
+        // case by accepting the connection, seeing backlog
+        // is too much, and sending HTTP 500 back
+        finish = handler.handle(state, r, params);
+      } catch (RequestFailedException rfe) {
+        String details = null;
+        if (rfe.param != null) {
+
+          // nocommit this seems to not help, ie if a
+          // handler threw an exception on a specific
+          // parameter, it means something went wrong w/
+          // that param, and it's not (rarely?) helpful to
+          // then list all the other valid params?
+
+          /*
+          if (rfe.request.getType() != null) {
+            Param p = rfe.request.getType().params.get(rfe.param);
+            if (p != null) {
+              if (p.type instanceof StructType) {
+                List<String> validParams = new ArrayList<String>(((StructType) p.type).params.keySet());
+                Collections.sort(validParams);
+                details = "valid params are: " + validParams.toString();
+              } else if (p.type instanceof ListType && (((ListType) p.type).subType instanceof StructType)) {
+                List<String> validParams = new ArrayList<String>(((StructType) ((ListType) p.type).subType).params.keySet());
+                Collections.sort(validParams);
+                details = "each element in the array may have these params: " + validParams.toString();
+              }
+            }
+          }
+          */
+        } else {
+          List<String> validParams = new ArrayList<String>(rfe.request.getType().params.keySet());
+          Collections.sort(validParams);
+          details = "valid params are: " + validParams.toString();
+        }
+
+        if (details != null) {
+          rfe = new RequestFailedException(rfe, details);
+        }
+
+        throw rfe;
+      }
+
+      // We remove params as they are accessed, so if
+      // anything is left it means it wasn't used:
+      if (Request.anythingLeft(requestData)) {
+        assert request != null;
+        JSONObject fullRequest;
+        try {
+          fullRequest = (JSONObject) new JSONParser(JSONParser.MODE_STRICTEST).parse(request, ContainerFactory.FACTORY_SIMPLE);          
+        } catch (ParseException pe) {
+          // The request parsed originally...:
+          assert false;
+
+          // Dead code but compiler disagrees:
+          fullRequest = null;
+        }
+
+        // Pretty print the leftover (unhandled) params:
+        String pretty = requestData.toJSONString(new JSONStyleIdent());
+        String s = "unrecognized parameters:\n" + pretty;
+        String details = findFirstWrongParam(handler.getType(), fullRequest, requestData, new ArrayList<String>());
+        if (details != null) {
+          s += "\n\n" + details;
+        }
+        throw new IllegalArgumentException(s);
+      }
+
+      String response = finish.finish();
+      //System.out.println("  response: " + response);
+
+      return response.getBytes("UTF-8");
+    }
+
+    private void sendError(ChannelHandlerContext ctx, HttpResponseStatus status, String details) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+      response.setHeader(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=UTF-8");
+      response.setContent(ChannelBuffers.copiedBuffer(
+                                                      "Failure: " + status.toString() + "\r\nDetails: " + details,
+                                                      CharsetUtil.UTF_8));
+      ChannelFuture later = ctx.getChannel().write(response);
+      if (!doKeepAlive) {
+        // Close the connection as soon as the error message is sent.
+        later.addListener(ChannelFutureListener.CLOSE);
+      }
+    }
+
+    /** Serves a static file from a plugin */
+    private void handlePlugin(ChannelHandlerContext ctx, HttpRequest request) throws Exception {
+      String uri = request.getUri();
+      int idx = uri.indexOf('/', 9);
+      if (idx == -1) {
+        sendError(ctx, HttpResponseStatus.BAD_REQUEST, "URL should be /plugin/name/...");
+        return;
+      }
+          
+      String pluginName = uri.substring(9, idx);
+      File pluginsDir = new File(globalState.stateDir, "plugins");
+      File pluginDir = new File(pluginsDir, pluginName);
+      if (!pluginDir.exists()) {
+        sendError(ctx, HttpResponseStatus.BAD_REQUEST, "plugin \"" + pluginName + "\" does not exist");
+      }
+
+      staticFileHandler.messageReceived(ctx, new UpstreamMessageEvent(ctx.getChannel(),
+                                                                      new DefaultHttpRequest(HttpVersion.HTTP_1_1,
+                                                                                             HttpMethod.GET,
+                                                                                             pluginName + "/site/" + uri.substring(idx+1)),
+                                                                      null));
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
+
+      byte[] responseBytes = null;
+
+      if (firstMessage) {
+        firstMessage = false;
+        HttpRequest request = (HttpRequest) e.getMessage();
+
+        // nocommit do we need to check this per-chunk?
+        String s = request.getHeader("Connection");
+        if (s != null && s.toLowerCase(Locale.ROOT).equals(HttpHeaders.Values.CLOSE)) {
+          doKeepAlive = false;
+        }
+
+        String uri = request.getUri();
+        if (uri.startsWith("/plugin")) {
+          handlePlugin(ctx, request);
+          return;
+        }
+
+        QueryStringDecoder decoder = new QueryStringDecoder(uri);
+        command = decoder.getPath().substring(1);
+
+        params = decoder.getParameters();
+
+        if (command.equals("doc")) {
+          String html = globalState.docHandler.handle(params, globalState.getHandlers());
+          responseBytes = html.getBytes("UTF-8");
+          Channel ch = e.getChannel();
+          HttpResponse r = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
+          r.setHeader(HttpHeaders.Names.CONTENT_TYPE, "text-html;charset=utf-8");
+          r.setHeader(HttpHeaders.Names.CONTENT_LENGTH, ""+responseBytes.length);
+          r.setContent(ChannelBuffers.copiedBuffer(responseBytes));
+          ch.write(r);
+          return;
+        }
+
+        // Dispatch by command:
+        try {
+          handler = globalState.getHandler(command);
+        } catch (IllegalArgumentException iae) {
+          throw new IllegalArgumentException("unrecognized method \"" + command + "\"");
+        }
+
+        if (handler.doStream()) {
+          //System.out.println("DO STREAM");
+          if (!request.isChunked()) {
+            //System.out.println("  not chunked");
+            // nocommit must get encoding, not assume UTF-8:
+
+            // Streaming handler but request wasn't chunked;
+            // just read all bytes into StringReader:
+            String result = handler.handleStreamed(new StringReader(request.getContent().toString(CharsetUtil.UTF_8)), params);
+            responseBytes = result.getBytes("UTF-8");
+          } else {
+            //System.out.println("  is chunked");
+            final Pipe p = new Pipe(1.0f);
+            pipedWriter = p.getWriter();
+            final Reader pipedReader = p.getReader();
+            pipedExc = new Throwable[1];
+            pipeThread = new Thread() {
+                @Override
+                public void run() {
+                  try {
+                    pipedResult[0] = handler.handleStreamed(pipedReader, params);
+                  } catch (Throwable t) {
+                    // nocommit use Future/call so exc is forwarded
+                    pipedExc[0] = t;
+                    p.close();
+                    //throw new RuntimeException(t);
+                  }
+                }
+              };
+            pipeThread.setName("Chunked HTTP");
+            pipeThread.start();
+          }
+        } else if (request.isChunked()) {
+          chunkedRequestBytes = new ByteArrayOutputStream();
+        } else {
+
+          String data;
+          if (request.getMethod() == HttpMethod.POST) {
+            /*
+              System.out.println("BUFFER: " + request.getContent());
+              if (command.equals("addDocument")) {
+              byte[] buffer = new byte[10];
+              while(true) {
+              request.getContent().readBytes(buffer);
+              System.out.println("here: " + buffer);
+              }
+              }
+            */
+            //ChannelBuffer cb = request.getContent();
+            //System.out.println("cp cap: " + cb.capacity() + " readable=" + cb.readableBytes() + " chunked=" + request.isChunked());
+            // nocommit must get encoding, not assume UTF-8:
+            data = request.getContent().toString(CharsetUtil.UTF_8);
+          } else {
+            data = null;
+          }
+
+          responseBytes = processRequest(data);
+        }
+      } else if (chunkedRequestBytes != null) {
+        if (pipedExc != null) {
+          maybeThrowPipeExc();
+        }
+        HttpChunk chunk = (HttpChunk) e.getMessage();
+        if (chunk.isLast()) {
+          // nocommit must verify encoding is UTF-8:
+          responseBytes = processRequest(chunkedRequestBytes.toString("UTF-8"));
+        } else {
+          // nocommit can we do single copy?
+          ChannelBuffer cb = chunk.getContent();
+          byte[] copy = new byte[cb.readableBytes()];
+          cb.readBytes(copy);
+          chunkedRequestBytes.write(copy);
+        }
+      } else {
+        try {
+          HttpChunk chunk = (HttpChunk) e.getMessage();
+          if (chunk.isLast()) {
+            pipedWriter.close();
+            pipedWriter = null;
+            pipeThread.join();
+            pipeThread = null;
+            responseBytes = pipedResult[0].getBytes("UTF-8");
+            firstMessage = true;
+          } else {
+            // nocommit must get encoding, not assume UTF-8:
+            ChannelBuffer cb = chunk.getContent();
+            byte[] copy = new byte[cb.readableBytes()];
+            cb.readBytes(copy);
+            char[] arr = IndexState.utf8ToCharArray(copy, 0, copy.length);
+            pipedWriter.write(arr, 0, arr.length);
+            //System.out.println("write another " + arr.length);
+          }
+        } catch (Throwable t) {
+          maybeThrowPipeExc();
+          throw new RuntimeException(t);
+        }
+      }
+
+      // sendError(ctx, METHOD_NOT_ALLOWED);
+      // sendError(ctx, FORBIDDEN);
+      // sendError(ctx, NOT_FOUND);
+      if (responseBytes != null) {
+        Channel ch = e.getChannel();
+        if (ch.isConnected()) {
+          HttpResponse r = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
+          r.setHeader(HttpHeaders.Names.CONTENT_TYPE, "text-plain; charset=utf-8");
+          r.setHeader(HttpHeaders.Names.CONTENT_LENGTH, ""+responseBytes.length);
+          if (doKeepAlive) {
+            r.setHeader(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
+          }
+          r.setContent(ChannelBuffers.copiedBuffer(responseBytes));
+          ChannelFuture f = ch.write(r);
+          if (!doKeepAlive) {
+            f.addListener(ChannelFutureListener.CLOSE);
+          } else {
+            firstMessage = true;
+          }
+        }
+      }
+
+      super.messageReceived(ctx, e);
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
+      //System.out.println("SVR.exceptionCaught: " + e);
+      //e.getCause().printStackTrace(System.out);
+
+      Channel ch = e.getChannel();
+      if (!ch.isConnected()) {
+        return;
+      }
+      // Just send full stack trace back to client:
+      Writer sw = new StringWriter();
+      PrintWriter pw = new PrintWriter(sw);
+      if (e.getCause() instanceof RequestFailedException) {
+        RequestFailedException rfe = (RequestFailedException) e.getCause();
+        pw.write(rfe.path + ": " + rfe.reason);
+        // TODO?
+        //Throwable cause = rfe.getCause();
+        //if (cause != null) {
+        //pw.write("\n\nCaused by:\n\n" + cause);
+        //}
+      } else {
+        e.getCause().printStackTrace(pw);
+      }
+
+      HttpResponse r = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.BAD_REQUEST);
+      //HttpResponse r = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
+      r.setHeader(HttpHeaders.Names.CONTENT_TYPE, "text-plain; charset=utf-8");
+      byte[] bytes = sw.toString().getBytes("UTF-8");
+      r.setHeader(HttpHeaders.Names.CONTENT_LENGTH, "" + bytes.length);
+      r.setContent(ChannelBuffers.copiedBuffer(bytes));
+      try {
+        ch.write(r).addListener(ChannelFutureListener.CLOSE);
+      } catch (Throwable t) {
+      }
+    }
+
+    private void maybeThrowPipeExc() throws Exception {
+      Throwable t = pipedExc[0];
+      if (t != null) {
+        if (pipedWriter != null) {
+          pipedWriter.close();
+        }
+        if (t instanceof Exception) {
+          throw (Exception) t;
+        } else if (t instanceof RuntimeException) {
+          throw (RuntimeException) t;
+        } else if (t instanceof Error) {
+          throw (Error) t;
+        } else {
+          throw new RuntimeException(t);
+        }
+      }
+    }
+  }
+
+  private static PolyEntry findPolyType(JSONObject fullRequest, StructType type) {
+    for(Map.Entry<String,Param> param : type.params.entrySet()) {
+      if (param.getValue().type instanceof PolyType) {
+        Object v = fullRequest.get(param.getKey());
+        if (v != null && v instanceof String) {
+          PolyEntry polyEntry = ((PolyType) param.getValue().type).types.get((String) v);
+          if (polyEntry != null) {
+            return polyEntry;
+          }
+        }
+      }
+    }
+
+    return null;
+  }
+
+  // TODO: should we find ALL wrong params?
+  // nocommit improve this: it needs to recurse into arrays too
+  private static String findFirstWrongParam(StructType type, JSONObject fullRequest, JSONObject r, List<String> path) {
+    PolyEntry polyEntry = findPolyType(fullRequest, type);
+    for(Map.Entry<String,Object> ent : r.entrySet()) {
+      String param = ent.getKey();
+      if (!type.params.containsKey(param) && !type.params.containsKey("*") && (polyEntry == null || !polyEntry.type.params.containsKey(param))) {
+
+        List<String> validParams = null;
+        String extra = "";
+        if (polyEntry != null) {
+          validParams = new ArrayList<String>(polyEntry.type.params.keySet());
+          extra = " for class=" + polyEntry.name;
+        } else {
+        // No PolyType found:
+          validParams = new ArrayList<String>(type.params.keySet());
+        }
+        Collections.sort(validParams);
+        
+        StringBuilder sb = new StringBuilder();
+        for(int i=0;i<path.size();i++) {
+          if (i > 0) {
+            sb.append(" > ");
+          }
+          sb.append(path.get(i));
+        }
+
+        if (sb.length() != 0) {
+          sb.append(" > ");
+        }
+        sb.append(param);
+        return "param " + sb.toString() + " is unrecognized; valid params" + extra + " are: " + validParams.toString();
+      }
+    }
+      
+    // Recurse:
+    for(Map.Entry<String,Object> ent : r.entrySet()) {
+      Param param = type.params.get(ent.getKey());
+      if (param == null && polyEntry != null) {
+        param = polyEntry.type.params.get(ent.getKey());
+      }
+      if (param == null) {
+        param = type.params.get("*");
+      }
+      // nocommit go into array, poly too
+      // nocommit handle case where we expected object but
+      // didnt' get json object
+      if (param.type instanceof StructType && ent.getValue() instanceof JSONObject) {
+        path.add(param.name);
+        String details = findFirstWrongParam((StructType) param.type, (JSONObject) fullRequest.get(ent.getKey()), (JSONObject) ent.getValue(), path);
+        if (details != null) {
+          return details;
+        }
+        path.remove(path.size()-1);
+      }
+    }
+
+    return null;
+  }
+
+  private static void usage() {
+    System.out.println("\nUsage: java -cp <stuff> org.apache.lucene.server.Server [-port port] [-maxHTTPThreadCount count] [-stateDir /path/to/dir]\n\n");
+  }
+
+  /** Sole constructor. */
+  public Server(File globalStateDir) throws IOException {
+    globalState = new GlobalState(globalStateDir);
+    staticFileHandler = new HttpStaticFileServerHandler(new File(globalState.stateDir, "plugins"));
+  }
+
+  /** Runs the server. */
+  public void run(int port, int maxHTTPThreadCount, CountDownLatch ready) throws Exception {
+
+    globalState.loadIndexNames();
+
+    // nocommit use fixed thread pools, so we don't cycle
+    // threads through Lucene's CloseableThreadLocals!
+    ExecutorService bossThreads = new ThreadPoolExecutor(0, maxHTTPThreadCount, 60L,
+                                                         TimeUnit.SECONDS,
+                                                         new SynchronousQueue<Runnable>(),
+                                                         new NamedThreadFactory("LuceneServer-boss"));
+    ExecutorService workerThreads = new ThreadPoolExecutor(0, maxHTTPThreadCount, 60L,
+                                                           TimeUnit.SECONDS,
+                                                           new SynchronousQueue<Runnable>(),
+                                                           new
+                                                         NamedThreadFactory("LuceneServer-worker"));
+    
+    //ExecutorService bossThreads = Executors.newCachedThreadPool();
+    //ExecutorService workerThreads = Executors.newCachedThreadPool();
+
+    ServerBootstrap bootstrap = null;
+    try {
+
+      ChannelFactory factory = new NioServerSocketChannelFactory(bossThreads, workerThreads, maxHTTPThreadCount);
+      //ChannelFactory factory = new NioServerSocketChannelFactory(bossThreads, workerThreads);
+      bootstrap = new ServerBootstrap(factory);
+
+      bootstrap.setOption("reuseAddress", true);
+      bootstrap.setPipelineFactory(new ChannelPipelineFactory() {
+          @Override
+          public ChannelPipeline getPipeline() {
+            ChannelPipeline p = Channels.pipeline();
+            p.addLast("decoder", new HttpRequestDecoder());
+            p.addLast("encoder", new HttpResponseEncoder());
+            p.addLast("deflater", new HttpContentCompressor(1));
+            // nocommit deflater (HttpContentCompressor)?
+            p.addLast("handler", new Dispatcher());
+            return p;
+          }
+        });
+
+      bootstrap.setOption("child.tcpNoDelay", true);
+      bootstrap.setOption("child.keepAlive", true);
+        
+      // Bind and start to accept incoming connections.
+      Channel sc = bootstrap.bind(new InetSocketAddress(port));
+      this.actualPort = ((InetSocketAddress) sc.getLocalAddress()).getPort();
+
+      globalState.addHandler("addDocument", new AddDocumentHandler(globalState));
+      globalState.addHandler("addDocuments", new AddDocumentsHandler(globalState));
+      globalState.addHandler("analyze", new AnalysisHandler(globalState));
+      globalState.addHandler("buildSuggest", new BuildSuggestHandler(globalState));
+      globalState.addHandler("bulkAddDocument", new BulkAddDocumentHandler(globalState));
+      globalState.addHandler("bulkAddDocuments", new BulkAddDocumentsHandler(globalState));
+      globalState.addHandler("bulkUpdateDocument", new BulkUpdateDocumentHandler(globalState));
+      globalState.addHandler("bulkUpdateDocuments", new BulkUpdateDocumentsHandler(globalState));
+      globalState.addHandler("commit", new CommitHandler(globalState));
+      globalState.addHandler("createIndex", new CreateIndexHandler(globalState));
+      globalState.addHandler("createSnapshot", new CreateSnapshotHandler(globalState));
+      globalState.addHandler("deleteAllDocuments", new DeleteAllDocumentsHandler(globalState));
+      globalState.addHandler("deleteIndex", new DeleteIndexHandler(globalState));
+      globalState.addHandler("deleteDocuments", new DeleteDocumentsHandler(globalState));
+      globalState.addHandler("liveSettings", new LiveSettingsHandler(globalState));
+      globalState.addHandler("liveValues", new LiveValuesHandler(globalState));
+      globalState.addHandler("registerFields", new RegisterFieldHandler(globalState));
+      globalState.addHandler("releaseSnapshot", new ReleaseSnapshotHandler(globalState));
+      globalState.addHandler("search", new SearchHandler(globalState));
+      globalState.addHandler("settings", new SettingsHandler(globalState));
+      globalState.addHandler("shutdown", new ShutdownHandler(globalState));
+      globalState.addHandler("startIndex", new StartIndexHandler(globalState));
+      globalState.addHandler("stats", new StatsHandler(globalState));
+      globalState.addHandler("stopIndex", new StopIndexHandler(globalState));
+      globalState.addHandler("suggestLookup", new SuggestLookupHandler(globalState));
+      globalState.addHandler("updateSuggest", new UpdateSuggestHandler(globalState));
+      globalState.addHandler("updateDocument", new UpdateDocumentHandler(globalState));
+      globalState.addHandler("setCommitUserData", new SetCommitUserDataHandler(globalState));
+      globalState.addHandler("getCommitUserData", new GetCommitUserDataHandler(globalState));
+
+      globalState.loadPlugins();
+
+      System.out.println("SVR: listening on port " + actualPort + ".");
+
+      // Notify caller server is started:
+      ready.countDown();
+
+      // Await shutdown:
+      globalState.shutdownNow.await();
+
+      // Close everything:
+      sc.close().awaitUninterruptibly();
+
+      globalState.close();
+
+    } finally {
+      if (bootstrap != null) {
+        bootstrap.releaseExternalResources();
+      }
+      bossThreads.shutdown();
+      workerThreads.shutdown();
+    }
+  }
+
+  /** Command-line entry. */
+  public static void main(String[] args) throws Exception {
+    File stateDir = null;
+
+    int port = 4000;
+    int maxHTTPThreadCount = 2*Runtime.getRuntime().availableProcessors();
+    for(int i=0;i<args.length;i++) {
+      if (args[i].equals("-port")) {
+        if (args.length == i+1) {
+          throw new IllegalArgumentException("no value specified after -port");
+        }
+        port = Integer.parseInt(args[i+1]);
+        i++;
+      } else if (args[i].equals("-maxHTTPThreadCount")) {
+        if (args.length == i+1) {
+          throw new IllegalArgumentException("no value specified after -maxHTTPThreadCount");
+        }
+        maxHTTPThreadCount = Integer.parseInt(args[i+1]);
+        i++;
+      } else if (args[i].equals("-stateDir")) {
+        if (args.length == i+1) {
+          throw new IllegalArgumentException("no value specified after -stateDir");
+        }
+        stateDir = new File(args[i+1]);
+        i++;
+      } else {
+        usage();
+        System.exit(-1);
+      }
+    }
+
+    new Server(stateDir).run(port, maxHTTPThreadCount, new CountDownLatch(1));
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/ServerCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/ServerCodec.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/ServerCodec.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/ServerCodec.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,147 @@
+package org.apache.lucene.server;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46FieldInfosFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46SegmentInfoFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/** Implements per-index {@link Codec}. */
+
+public class ServerCodec extends Codec {
+
+  public final static String DEFAULT_POSTINGS_FORMAT = "Lucene41";
+  public final static String DEFAULT_DOC_VALUES_FORMAT = "Lucene45";
+
+  private final IndexState state;
+  private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
+  private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+
+  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+    @Override
+    public PostingsFormat getPostingsFormatForField(String field) {
+      String pf;
+      try {
+        pf = state.getField(field).postingsFormat;
+      } catch (IllegalArgumentException iae) {
+        // The indexed facets field will have drill-downs,
+        // which will pull the postings format:
+        if (state.internalFacetFieldNames.contains(field)) {
+          pf = DEFAULT_POSTINGS_FORMAT;
+        } else {
+          throw iae;
+        }
+      }
+      return PostingsFormat.forName(pf);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      String dvf;
+      try {
+        dvf = state.getField(field).docValuesFormat;
+      } catch (IllegalArgumentException iae) {
+        if (state.internalFacetFieldNames.contains(field)) {
+          dvf = DEFAULT_DOC_VALUES_FORMAT;
+        } else {
+          throw iae;
+        }
+      }
+      return DocValuesFormat.forName(dvf);
+    }
+  };
+
+  /** Sole constructor. */
+  public ServerCodec(IndexState state) {
+    super("ServerCodec");
+    this.state = state;
+  }
+
+  /** Default constructor. */
+  public ServerCodec() {
+    this(null);
+  }
+
+  @Override
+  public final StoredFieldsFormat storedFieldsFormat() {
+    return fieldsFormat;
+  }
+  
+  @Override
+  public final TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
+
+  @Override
+  public final PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+  
+  @Override
+  public final FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+  
+  @Override
+  public final SegmentInfoFormat segmentInfoFormat() {
+    return segmentInfosFormat;
+  }
+  
+  @Override
+  public final LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return docValuesFormat;
+  }
+
+  @Override
+  public final NormsFormat normsFormat() {
+    if (state == null) {
+      return new Lucene42NormsFormat();
+    } else if (state.normsFormat.equals("Lucene42")) {
+      return new Lucene42NormsFormat(state.normsAcceptableOverheadRatio);
+    } else if (state.normsFormat.equals("Lucene40")) {
+      return new Lucene40NormsFormat();
+    } else {
+      throw new AssertionError();
+    }
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/StringLiveFieldValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/StringLiveFieldValues.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/StringLiveFieldValues.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/StringLiveFieldValues.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,61 @@
+package org.apache.lucene.server;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collections;
+
+import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager.SearcherAndTaxonomy;
+import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager;
+import org.apache.lucene.index.StoredDocument;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.LiveFieldValues;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+
+// TODO: if there are more than one live field we could
+// "bundle" them so we only do one stored doc lookup
+// TODO: also be able to lookup from doc values
+
+/** Implements live field values, for string values. */
+public class StringLiveFieldValues extends LiveFieldValues<SearcherAndTaxonomy,String> {
+  private final String idFieldName;
+  private final String liveFieldName;
+
+  /** Sole constructor. */
+  public StringLiveFieldValues(SearcherTaxonomyManager mgr, String idFieldName, String liveFieldName) {
+    super(mgr, "");
+    this.idFieldName = idFieldName;
+    this.liveFieldName = liveFieldName;
+  }
+
+  @Override
+  protected String lookupFromSearcher(SearcherAndTaxonomy s, String id) throws IOException {
+
+    TermQuery q = new TermQuery(new Term(idFieldName, id));
+    TopDocs hits = s.searcher.search(q, 1);
+    if (hits.totalHits == 0) {
+      return null;
+    } else if (hits.totalHits > 1) {
+      throw new IllegalStateException("field \"" + idFieldName + "\"=\"" + id + "\" matches more than one document");
+    } else {
+      StoredDocument doc = s.searcher.doc(hits.scoreDocs[0].doc, Collections.singleton(liveFieldName));
+      return doc.get(liveFieldName);
+    }
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/WholeMVJSONPassageFormatter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/WholeMVJSONPassageFormatter.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/WholeMVJSONPassageFormatter.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/WholeMVJSONPassageFormatter.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,86 @@
+package org.apache.lucene.server;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.postingshighlight.Passage;
+import org.apache.lucene.search.postingshighlight.PassageFormatter;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+/** From a multi-valued field (indexed with the values
+ *  joined with INFO_SEP), highlight each value entirely. */
+public class WholeMVJSONPassageFormatter extends PassageFormatter {
+
+  private final int offsetGap;
+
+  /** Create this, with the specified offsetGap. */
+  public WholeMVJSONPassageFormatter(int offsetGap) {
+    this.offsetGap = offsetGap;
+  }
+
+  /** Carefully finds the field boundaries
+   *  (INFORMATION_SEPARATOR) in the content and builds a
+   *  JSONArray so that each original field value is
+   *  separated and highlighted. */
+  @Override
+  public JSONArray format(Passage[] passages, String content) {
+
+    // Caller must use WholeBreakIterator:
+    assert passages.length == 1;
+    Passage passage = passages[0];
+
+    String[] chunks = content.split(Constants.INFORMATION_SEP_REGEX);
+    JSONArray result = new JSONArray();
+    int matchUpto = 0;
+    int charOffset = 0;
+    for(String chunk : chunks) {
+      JSONArray part = new JSONArray();
+      result.add(part);
+      int pos = 0;
+      int posEnd = chunk.length();
+      while (matchUpto < passage.getNumMatches()) {
+        int start = passage.getMatchStarts()[matchUpto] - charOffset;
+        if (start >= posEnd) {
+          break;
+        }
+        if (start > pos) {
+          part.add(chunk.substring(pos, start));
+          pos = start;
+        }
+        JSONObject match = new JSONObject();
+        part.add(match);
+        int end = passage.getMatchEnds()[matchUpto] - charOffset;
+        match.put("text", chunk.substring(start, end));
+        match.put("term", passage.getMatchTerms()[matchUpto].utf8ToString());
+        pos = end;
+        matchUpto++;
+      }
+      if (pos < chunk.length()) {
+        part.add(chunk.substring(pos));
+        pos = chunk.length();
+      }
+
+      // nocommit we always join w/ INFO_SEP ... so it
+      //should just be 1?
+      //charOffset += chunk.length()+offsetGap;
+      charOffset += chunk.length()+1;
+    }
+
+    return result;
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AddDocumentHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AddDocumentHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AddDocumentHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AddDocumentHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,452 @@
+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.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.facet.FacetField;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.server.Constants;
+import org.apache.lucene.server.FieldDef;
+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.apache.lucene.util.BytesRef;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+import net.minidev.json.JSONValue;
+
+/** Handles {@code addDocument}, by delegating the single
+ *  document to {@link BulkAddDocumentHandler}. */
+public class AddDocumentHandler extends Handler {
+
+  /** Type for a document. */
+  public final static StructType DOCUMENT_TYPE = new StructType(
+                                       new Param("fields", "Fields to index into one document",
+                                           new StructType(
+                                                          new Param("anyFieldName", "A name/value pair for this document.  Multiple name/values can be specified, but each field name must already have been registered via @registerFields.  The type of the value must match how the field was registered.", new AnyType()))));
+
+  /** Parmeter type. */
+  final StructType TYPE = new StructType(
+                                     new Param("indexName", "Index name", new StringType()));
+
+  /** Sole constructor. */
+  public AddDocumentHandler(GlobalState state) {
+    super(state);
+    TYPE.params.putAll(DOCUMENT_TYPE.params);
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Adds one document to the index.  Returns the index generation (indexGen) that contains this added document.";
+  }
+
+  private static class MyField extends Field {
+    public MyField(String name, FieldType ft, Object value) {
+      super(name, ft);
+      fieldsData = value;
+    }
+  }
+
+  /** Parses value for one field. */
+  @SuppressWarnings({"unchecked"})
+  private static void parseOneNativeValue(FieldDef fd, Document doc, Object o, float boost) {
+
+    assert o != null;
+    assert fd != null;
+    
+    if (fd.fieldType.stored() || fd.fieldType.indexed() || fd.fieldType.docValueType() != null) {
+      if (fd.valueType.equals("text") || fd.valueType.equals("atom")) {
+        if (!(o instanceof String)) {
+          fail(fd.name, "expected String value but got " + o);
+        }
+      } else if (fd.valueType.equals("boolean")) {
+        if (!(o instanceof Boolean)) {
+          fail(fd.name, "expected Boolean value but got " + o.getClass());
+        }
+        // Turn boolean -> int now
+        if (o == Boolean.TRUE) {
+          o = Integer.valueOf(1);
+        } else {
+          o = Integer.valueOf(0);
+        }
+      } else if (fd.valueType.equals("float") || fd.valueType.equals("double")) {
+        if (!(o instanceof Number)) {
+          fail(fd.name, "for float or double field, expected Number value but got " + o);
+        }
+      } else {
+        // int or long
+        if (!(o instanceof Integer) && !(o instanceof Long)) {
+          fail(fd.name, "for int or long field, expected Integer or Long value but got " + o);
+        }
+      }
+    }
+
+    if (fd.faceted.equals("flat")) {
+
+      if (o instanceof List) { 
+        fail(fd.name, "value should be String when facet=flat; got JSONArray");
+      }
+        
+      doc.add(new FacetField(fd.name, o.toString()));
+    } else if (fd.faceted.equals("hierarchy")) {
+      if (o instanceof List) { 
+        List<String> values = (List<String>) o;
+        doc.add(new FacetField(fd.name, values.toArray(new String[values.size()])));
+      } else {
+        doc.add(new FacetField(fd.name, o.toString()));
+      }
+    } else if (fd.faceted.equals("sortedSetDocValues")) {
+      if (o instanceof List) { 
+        fail(fd.name, "value should be String when facet=sortedSetDocValues; got JSONArray");
+      }
+      doc.add(new SortedSetDocValuesFacetField(fd.name, o.toString()));
+    }
+
+    if (fd.highlighted) {
+      assert o instanceof String;
+      if (fd.multiValued && (((String) o).indexOf(Constants.INFORMATION_SEP) != -1)) {
+        // TODO: we could remove this restriction if it
+        // ever matters ... we can highlight multi-valued
+        // fields at search time without stealing a
+        // character:
+        fail(fd.name, "multiValued and hihglighted fields cannot contain INFORMATION_SEPARATOR (U+001F) character: this character is used internally when highlighting multi-valued fields");
+      }
+    }
+
+    // nocommit what about sorted set?
+
+    // Separately index doc values:
+    if (fd.fieldType.docValueType() == DocValuesType.BINARY ||
+        fd.fieldType.docValueType() == DocValuesType.SORTED) {
+      assert o instanceof String;
+      BytesRef br = new BytesRef((String) o);
+      if (fd.fieldType.docValueType() == DocValuesType.BINARY) {
+        doc.add(new BinaryDocValuesField(fd.name, br));
+      } else {
+        doc.add(new SortedDocValuesField(fd.name, br));
+      }
+    } else if (fd.fieldType.docValueType() == DocValuesType.NUMERIC) {
+      if (fd.valueType.equals("float")) {
+        //doc.add(new NumericDocValuesField(fd.name, Float.floatToRawIntBits(((Number) o).floatValue())));
+        doc.add(new FloatDocValuesField(fd.name, ((Number) o).floatValue()));
+      } else if (fd.valueType.equals("double")) {
+        //doc.add(new NumericDocValuesField(fd.name, Double.doubleToRawLongBits(((Number) o).doubleValue())));
+        doc.add(new DoubleDocValuesField(fd.name, ((Number) o).doubleValue()));
+      } else if (fd.valueType.equals("int")) {
+        doc.add(new NumericDocValuesField(fd.name, ((Number) o).intValue()));
+      } else if (fd.valueType.equals("long")) {
+        doc.add(new NumericDocValuesField(fd.name, ((Number) o).longValue()));
+      } else {
+        assert fd.valueType.equals("boolean");
+        doc.add(new NumericDocValuesField(fd.name, ((Integer) o).intValue()));
+      }
+    }
+
+    if (fd.fieldType.stored() || fd.fieldType.indexed()) {
+      // We use fieldTypeNoDV because we separately added
+      // (above) the doc values field:
+      Field f = new MyField(fd.name, fd.fieldTypeNoDV, o);
+      f.setBoost(boost);
+      doc.add(f);
+    }
+    //System.out.println("add doc: " + doc);
+  }
+
+  /** Used by plugins to process a document after it was
+   *  created from the JSON request. */
+  public interface PostHandle {
+    // nocommit need test coverage:
+    /** Invoke the handler, non-streaming. */
+    public void invoke(IndexState state, Request r, Document doc) throws IOException;
+    /** Invoke the handler, streaming. */
+    public boolean invoke(IndexState state, String fieldName, JsonParser p, Document doc) throws IOException;
+  }
+
+  final List<PostHandle> postHandlers = new CopyOnWriteArrayList<PostHandle>();
+  
+  /** Record a new {@link PostHandle}. */
+  public void addPostHandle(PostHandle handler) {
+    postHandlers.add(handler);
+  }
+
+  /** Parses the string value to the appropriate type. */
+  /*
+  public static Object fixType(FieldDef fd, String value) {
+    Object o;
+    if (fd.valueType.equals("int")) {
+      o = Integer.valueOf(Integer.parseInt(value));
+    } else if (fd.valueType.equals("long")) {
+      o = Long.valueOf(Long.parseLong(value));
+    } else if (fd.valueType.equals("float")) {
+      o = Float.valueOf(Float.parseFloat(value));
+    } else if (fd.valueType.equals("double")) {
+      o = Double.valueOf(Double.parseDouble(value));
+    } else {
+      o = value;
+    }
+    return o;
+  }
+  */
+
+  static void fail(String fieldName, String message) {
+    throw new IllegalArgumentException("field=" + fieldName + ": " + message);
+  }
+
+  /** Parses the fields, which should look like {field1:
+   *  ..., field2: ..., ...} */
+  void parseFields(IndexState state, Document doc, JsonParser p) throws IOException {
+    JsonToken token = p.nextToken();
+    if (token != JsonToken.START_OBJECT) {
+      throw new IllegalArgumentException("fields should be an object");
+    }
+    while (true) {
+      token = p.nextToken();
+      if (token == JsonToken.END_OBJECT) {
+        break;
+      }
+      assert token == JsonToken.FIELD_NAME;
+      parseOneField(p, state, doc, p.getText());
+    }
+  }
+
+  /** Parse a Document using Jackson's streaming parser
+   * API.  The document should look like {indexName: 'foo',
+   * fields: {..., ...}} */
+  Document parseDocument(IndexState state, JsonParser p) throws IOException {
+    //System.out.println("parseDocument: " + r);
+    JsonToken token = p.nextToken();
+    if (token == JsonToken.END_ARRAY) {
+      // nocommit hackish.. caller should tell us this means "end"?
+      return null;
+    } else if (token != JsonToken.START_OBJECT) {
+      throw new IllegalArgumentException("expected JSON Object");
+    }
+
+    final Document doc = new Document();
+    while (true) {
+      token = p.nextToken();
+      if (token == JsonToken.END_OBJECT) {
+        break;
+      }
+      assert token == JsonToken.FIELD_NAME: token;
+
+      String fieldName = p.getText();
+      if (fieldName.equals("fields")) {
+        parseFields(state, doc, p);
+      } else {
+        // Let a plugin handle it:
+        boolean handled = false;
+        for(PostHandle postHandle : postHandlers) {
+          if (postHandle.invoke(state, fieldName, p, doc)) {
+            handled = true;
+            break;
+          }
+        }
+
+        if (!handled) {
+          throw new IllegalArgumentException("unrecognized field " + p.getText());
+        }
+      }
+
+      // nocommit need test that same field name can't
+      // appear more than once?  app must put all values for
+      // a given field into an array (for a multi-valued
+      // field) 
+    }
+
+    return doc;
+  }
+
+  /** Parses a field's value, which is an array in the
+   * multi-valued case, or an object of the appropriate type
+   * in the single-valued case. */
+  private static void parseOneField(JsonParser p, IndexState state, Document doc, String name) throws IOException {
+
+    FieldDef fd = state.getField(name);
+
+    if (fd.multiValued) {
+      // Field is mutli-valued; parse an array
+      JsonToken token = p.nextToken();
+      if (token != JsonToken.START_ARRAY) {
+        fail(name, "field is multiValued; expected array but got " + token);
+      }
+      while (true) {
+        if (!parseOneValue(fd, p, doc)) {
+          break;
+        }
+      }
+    } else {
+      parseOneValue(fd, p, doc);
+    }
+  }
+
+  /** Parses the current json token into the corresponding
+   *  java object. */
+  private static Object getNativeValue(FieldDef fd, JsonToken token, JsonParser p) throws IOException {
+    Object o;
+    if (token == JsonToken.VALUE_STRING) {
+      o = p.getText();
+    } else if (token == JsonToken.VALUE_NUMBER_INT) {
+      o = Long.valueOf(p.getLongValue());
+    } else if (token == JsonToken.VALUE_NUMBER_FLOAT) {
+      o = Double.valueOf(p.getDoubleValue());
+    } else if (token == JsonToken.VALUE_TRUE) {
+      o = Boolean.TRUE;
+    } else if (token == JsonToken.VALUE_FALSE) {
+      o = Boolean.FALSE;
+    } else if (fd.faceted.equals("hierarchy") && token == JsonToken.START_ARRAY) {
+      List<String> values = new ArrayList<String>();
+      while (true) {
+        token = p.nextToken();
+        if (token == JsonToken.END_ARRAY) {
+          break;
+        } else if (token != JsonToken.VALUE_STRING) {
+          if (token == JsonToken.START_ARRAY) {
+            fail(fd.name, "expected array of strings, but saw array inside array");
+          } else {
+            fail(fd.name, "expected array of strings, but saw " + token + " inside array");
+          }
+        }
+        values.add(p.getText());
+      }
+      o = values;
+    } else {
+      String message;
+      if (token == JsonToken.VALUE_NULL) {
+        message = "null field value not supported; just omit this field from the document instead";
+      } else {
+        message = "value in inner object field value should be string, int/long, float/double or boolean; got " + token;
+      }
+
+      fail(fd.name, message);
+
+      // Dead code but compiler disagrees:
+      o = null;
+    }
+    return o;
+  }
+  
+  /** Parse one value for a field, which is either an
+   *  object matching the type of the field, or a {boost:
+   *  ..., value: ...}. */
+  private static boolean parseOneValue(FieldDef fd, JsonParser p, Document doc) throws IOException {
+
+    Object o = null;
+
+    JsonToken token = p.nextToken();
+    if (token == JsonToken.END_ARRAY) {
+      assert fd.multiValued;
+      return false;
+    }
+
+    float boost = 1.0f;
+    if (fd.fieldType.indexed() && token == JsonToken.START_OBJECT) {
+      // Parse a {boost: X, value: Y}
+      while(true) {
+        token = p.nextToken();
+        if (token == JsonToken.END_OBJECT) {
+          break;
+        }
+        assert token == JsonToken.FIELD_NAME;
+        String key = p.getText();
+        if (key.equals("boost")) {
+          token = p.nextToken(); 
+          if (token == JsonToken.VALUE_NUMBER_INT || token == JsonToken.VALUE_NUMBER_FLOAT) {
+            boost = p.getFloatValue();
+          } else {
+            fail(fd.name, "boost in inner object field value must have float or int value; got: " + token);
+          }
+        } else if (key.equals("value")) {
+          o = getNativeValue(fd, p.nextToken(), p);
+        } else {
+          fail(fd.name, "unrecognized json key \"" + key + "\" in inner object field value; must be boost or value");
+        }
+      }
+      if (o == null) {
+        fail(fd.name, "missing 'value' key");
+      }
+    } else {
+      // Parse a native value:
+      o = getNativeValue(fd, token, p);
+    }
+
+    parseOneNativeValue(fd, doc, o, boost);
+    return true;
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+
+    state.verifyStarted(r);
+
+    // NOTE: somewhat wasteful since we re-serialize to
+    // string only to re-parse the JSON, but this allows
+    // single-source (bulk) for parsing, and apps that care
+    // about performance will use bulk APIs:
+
+    JSONObject raw = r.getRawParams();
+    StringBuilder sb = new StringBuilder();
+    sb.append("{\"indexName\": \"");
+    sb.append(state.name);
+    sb.append("\", \"documents\": [");
+    sb.append(raw.toString());
+    sb.append("]}");
+    raw.clear();
+
+    final String bulkRequestString = sb.toString();
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws Exception {
+        String result = globalState.getHandler("bulkAddDocument").handleStreamed(new StringReader(bulkRequestString), null);
+        if (result.indexOf("errors") != -1) {
+          JSONObject o = (JSONObject) JSONValue.parseStrict(result);
+          if (o.containsKey("errors")) {
+            JSONObject err = (JSONObject) ((JSONArray) o.get("errors")).get(0);
+            throw new IllegalArgumentException((String) err.get("exception"));
+          }
+        }
+        return result;
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AddDocumentsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AddDocumentsHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AddDocumentsHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AddDocumentsHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,94 @@
+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.StringReader;
+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.JSONArray;
+import net.minidev.json.JSONObject;
+import net.minidev.json.JSONValue;
+
+/** Handles {@code addDocuments}, by delegating the single
+ *  document to {@link BulkAddDocumentsHandler} */
+public class AddDocumentsHandler extends Handler {
+
+  final static StructType TYPE = new StructType(
+                               new Param("indexName", "Index name", 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)));
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Adds one document block (= single parent and multiple children) to the index.  This can be used for block grouping and block joins.  Returns the index generation (indexGen) that contains this added document block.";
+  }
+
+  /** Sole constructor. */
+  public AddDocumentsHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+
+    state.verifyStarted(r);
+
+    // NOTE: somewhat wasteful since we re-serialize to
+    // string only to re-parse the JSON, but this allows
+    // single-source (bulk) for parsing, and apps that care
+    // about performance will use bulk APIs:
+
+    JSONObject raw = r.getRawParams();
+    StringBuilder sb = new StringBuilder();
+    sb.append("{\"indexName\": \"");
+    sb.append(state.name);
+    sb.append("\", \"documents\": [");
+    sb.append(raw.toString());
+    sb.append("]}");
+    raw.clear();
+
+    final String bulkRequestString = sb.toString();
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws Exception {
+        String result = globalState.getHandler("bulkAddDocuments").handleStreamed(new StringReader(bulkRequestString), null);
+        if (result.indexOf("errors") != -1) {
+          JSONObject o = (JSONObject) JSONValue.parseStrict(result);
+          if (o.containsKey("errors")) {
+            JSONObject err = (JSONObject) ((JSONArray) o.get("errors")).get(0);
+            throw new IllegalArgumentException((String) err.get("exception"));
+          }
+        }
+        return result;
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AnalysisHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AnalysisHandler.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AnalysisHandler.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/server/src/java/org/apache/lucene/server/handlers/AnalysisHandler.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,99 @@
+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.StringReader;
+import java.util.List;
+import java.util.Map;
+
+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.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+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 net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+
+/** Useful for debugging analyzers. */
+public class AnalysisHandler extends Handler {
+  private final static StructType TYPE = new StructType(
+                                                        new Param("indexName", "Index Name", new StringType()),
+                                                        new Param("text", "Text to analyze", new StringType()),
+                                                        new Param("analyzer", "Analyzer", RegisterFieldHandler.ANALYZER_TYPE));
+
+  /** Sole constructor. */
+  public AnalysisHandler(GlobalState state) {
+    super(state);
+  }
+
+  @Override
+  public StructType getType() {
+    return TYPE;
+  }
+
+  @Override
+  public String getTopDoc() {
+    return "Run an analyzer on text and see the resulting tokens.";
+  }
+
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+    final String text = r.getString("text");
+
+    // TODO: allow passing field name in, and we use its analyzer?
+    final Analyzer a = RegisterFieldHandler.getAnalyzer(state, r, "analyzer");
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws Exception {
+        TokenStream ts = a.tokenStream("field", new StringReader(text));
+        CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+        PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
+        PositionLengthAttribute posLenAtt = ts.addAttribute(PositionLengthAttribute.class);
+        OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
+        ts.reset();
+
+        JSONArray tokens = new JSONArray();
+        int pos = -1;
+        while(ts.incrementToken()) {
+          JSONObject o = new JSONObject();
+          tokens.add(o);
+          o.put("token", termAtt.toString());
+          pos += posIncAtt.getPositionIncrement();
+          o.put("position", pos);
+          o.put("positionLength", posLenAtt.getPositionLength());
+          o.put("startOffset", offsetAtt.startOffset());
+          o.put("endOffset", offsetAtt.endOffset());
+        }
+        ts.end();
+        ts.close();
+        JSONObject result = new JSONObject();
+        result.put("tokens", tokens);
+        return result.toString();
+      }
+    };
+  }
+}