You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by sh...@apache.org on 2009/04/22 10:53:29 UTC

svn commit: r767412 [1/2] - in /lucene/solr/trunk: ./ example/solr/conf/ src/common/org/apache/solr/common/params/ src/java/org/apache/solr/handler/ src/solrj/org/apache/solr/client/solrj/request/ src/solrj/org/apache/solr/client/solrj/response/ src/te...

Author: shalin
Date: Wed Apr 22 08:53:28 2009
New Revision: 767412

URL: http://svn.apache.org/viewvc?rev=767412&view=rev
Log:
SOLR-1099 -- A FieldAnalysisRequestHandler which provides the analysis functionality of the web admin page as a service. The AnalysisRequestHandler is renamed to DocumentAnalysisRequestHandler which is enhanced with query analysis and showMatch support. AnalysisRequestHandler is now deprecated. Support for both FieldAnalysisRequestHandler and DocumentAnalysisRequestHandler is also provided in the Solrj client.

Added:
    lucene/solr/trunk/src/common/org/apache/solr/common/params/AnalysisParams.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/FieldAnalysisRequestHandler.java   (with props)
    lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java   (with props)
    lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java   (with props)
    lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/AnalysisResponseBase.java   (with props)
    lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/DocumentAnalysisResponse.java   (with props)
    lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/FieldAnalysisResponse.java   (with props)
    lucene/solr/trunk/src/test/org/apache/solr/client/solrj/response/AnlysisResponseBaseTest.java   (with props)
    lucene/solr/trunk/src/test/org/apache/solr/client/solrj/response/DocumentAnalysisResponseTest.java   (with props)
    lucene/solr/trunk/src/test/org/apache/solr/client/solrj/response/FieldAnalysisResponseTest.java   (with props)
    lucene/solr/trunk/src/test/org/apache/solr/handler/AnalysisRequestHandlerTestBase.java   (with props)
    lucene/solr/trunk/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java   (with props)
    lucene/solr/trunk/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java   (with props)
Modified:
    lucene/solr/trunk/CHANGES.txt
    lucene/solr/trunk/example/solr/conf/solrconfig.xml
    lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandler.java

Modified: lucene/solr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/CHANGES.txt?rev=767412&r1=767411&r2=767412&view=diff
==============================================================================
--- lucene/solr/trunk/CHANGES.txt (original)
+++ lucene/solr/trunk/CHANGES.txt Wed Apr 22 08:53:28 2009
@@ -199,6 +199,12 @@
 
 36. SOLR-1115: <bool>on</bool> and <bool>yes</bool> work as expected in solrconfig.xml. (koji)
 
+37. SOLR-1099:  A FieldAnalysisRequestHandler which provides the analysis functionality of the web admin page as
+    a service. The AnalysisRequestHandler is renamed to DocumentAnalysisRequestHandler which is enhanced with
+    query analysis and showMatch support. AnalysisRequestHandler is now deprecated. Support for both
+    FieldAnalysisRequestHandler and DocumentAnalysisRequestHandler is also provided in the Solrj client.
+    (Uri Boness, shalin)
+
 Optimizations
 ----------------------
  1. SOLR-374: Use IndexReader.reopen to save resources by re-using parts of the

Modified: lucene/solr/trunk/example/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/example/solr/conf/solrconfig.xml?rev=767412&r1=767411&r2=767412&view=diff
==============================================================================
--- lucene/solr/trunk/example/solr/conf/solrconfig.xml (original)
+++ lucene/solr/trunk/example/solr/conf/solrconfig.xml Wed Apr 22 08:53:28 2009
@@ -680,10 +680,54 @@
   <requestHandler name="/update/javabin" class="solr.BinaryUpdateRequestHandler" />
 
   <!--
-   Analysis request handler.  Since Solr 1.3.  Use to returnhow a document is analyzed.  Useful
-   for debugging and as a token server for other types of applications
+   Analysis request handler.  Since Solr 1.3.  Use to return how a document is analyzed.  Useful
+   for debugging and as a token server for other types of applications.
+
+   This is deprecated in favor of the improved DocumentAnalysisRequestHandler and FieldAnalysisRequestHandler
+
+   <requestHandler name="/analysis" class="solr.AnalysisRequestHandler" />
+   -->
+
+  <!--
+    An analysis handler that provides a breakdown of the analysis process of provided docuemnts. This handler expects a
+    (single) content stream with the following format:
+
+    <docs>
+      <doc>
+        <field name="id">1</field>
+        <field name="name">The Name</field>
+        <field name="text">The Text Value</field>
+      <doc>
+      <doc>...</doc>
+      <doc>...</doc>
+      ...
+    </docs>
+
+    Note: Each document must contain a field which serves as the unique key. This key is used in the returned
+    response to assoicate an analysis breakdown to the analyzed document.
+
+    Like the FieldAnalysisRequestHandler, this handler also supports query analysis by
+    sending either an "analysis.query" or "q" request paraemter that holds the query text to be analyized. It also
+    supports the "analysis.showmatch" parameter which when set to true, all field tokens that match the query
+    tokens will be marked as a "match".
+  -->
+  <requestHandler name="/analysis/document" class="solr.DocumentAnalysisRequestHandler" />
+
+  <!--
+    RequestHandler that provides much the same functionality as analysis.jsp. Provides the ability
+    to specify multiple field types and field names in the same request and outputs index-time and
+    query-time analysis for each of them.
+
+    Request parameters are:
+    analysis.fieldname - The field name whose analyzers are to be used
+    analysis.fieldtype - The field type whose analyzers are to be used
+    analysis.fieldvalue - The text for index-time analysis
+    q (or analysis.q) - The text for query time analysis
+    analysis.showmatch (true|false) - When set to true and when query analysis is performed, the produced
+                                      tokens of the field value analysis will be marked as "matched" for every
+                                      token that is produces by the query analysis
    -->
-  <requestHandler name="/analysis" class="solr.AnalysisRequestHandler" />
+  <requestHandler name="/analysis/field" class="solr.FieldAnalysisRequestHandler" />
 
 
   <!-- CSV update handler, loaded on demand -->

Added: lucene/solr/trunk/src/common/org/apache/solr/common/params/AnalysisParams.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/common/org/apache/solr/common/params/AnalysisParams.java?rev=767412&view=auto
==============================================================================
--- lucene/solr/trunk/src/common/org/apache/solr/common/params/AnalysisParams.java (added)
+++ lucene/solr/trunk/src/common/org/apache/solr/common/params/AnalysisParams.java Wed Apr 22 08:53:28 2009
@@ -0,0 +1,60 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.common.params;
+
+/**
+ * Defines the request parameters used by all analysis request handlers.
+ *
+ * @version $Id$
+ * @since solr 1.4
+ */
+public interface AnalysisParams {
+
+  /**
+   * The prefix for all parameters.
+   */
+  static final String PREFIX = "analysis";
+
+  /**
+   * Holds the query to be analyzed.
+   */
+  static final String QUERY = PREFIX + ".query";
+
+  /**
+   * Set to {@code true} to indicate that the index tokens that match query tokens should be marked as "mateched".
+   */
+  static final String SHOW_MATCH = PREFIX + ".showmatch";
+
+
+  //===================================== FieldAnalysisRequestHandler Params =========================================
+
+  /**
+   * Holds the value of the field which should be analyzed.
+   */
+  static final String FIELD_NAME = PREFIX + ".fieldname";
+
+  /**
+   * Holds a comma-separated list of field types that the analysis should be peformed for.
+   */
+  static final String FIELD_TYPE = PREFIX + ".fieldtype";
+
+  /**
+   * Hodls a comma-separated list of field named that the analysis should be performed for.
+   */
+  static final String FIELD_VALUE = PREFIX + ".fieldvalue";
+}

Propchange: lucene/solr/trunk/src/common/org/apache/solr/common/params/AnalysisParams.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/common/org/apache/solr/common/params/AnalysisParams.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Modified: lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandler.java?rev=767412&r1=767411&r2=767412&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandler.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandler.java Wed Apr 22 08:53:28 2009
@@ -31,6 +31,8 @@
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.xml.stream.XMLInputFactory;
 import javax.xml.stream.XMLStreamConstants;
@@ -39,15 +41,11 @@
 import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  *
- *
+ * @deprecated Use {@link org.apache.solr.handler.DocumentAnalysisRequestHandler} instead.
  **/
 public class AnalysisRequestHandler extends RequestHandlerBase {
 

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java?rev=767412&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java Wed Apr 22 08:53:28 2009
@@ -0,0 +1,283 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.handler;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.solr.analysis.TokenFilterFactory;
+import org.apache.solr.analysis.TokenizerChain;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryResponse;
+import org.apache.solr.schema.FieldType;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.*;
+
+/**
+ * A base class for all analysis request handlers.
+ *
+ * @version $Id$
+ * @since solr 1.4
+ */
+public abstract class AnalysisRequestHandlerBase extends RequestHandlerBase {
+
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    rsp.add("analysis", doAnalysis(req));
+  }
+
+  /**
+   * Performs the analysis based on the given solr request and returns the analysis result as a named list.
+   *
+   * @param req The solr request.
+   *
+   * @return The analysis result as a named list.
+   *
+   * @throws Exception When analysis fails.
+   */
+  protected abstract NamedList doAnalysis(SolrQueryRequest req) throws Exception;
+
+  /**
+   * Analyzes the given value using the given Analyzer.
+   *
+   * @param value   Value to analyze
+   * @param context The {@link AnalysisContext analysis context}.
+   *
+   * @return NamedList containing the tokens produced by analyzing the given value
+   */
+  protected NamedList<List<NamedList>> analyzeValue(String value, AnalysisContext context) {
+
+    Analyzer analyzer = context.getAnalyzer();
+
+    if (!TokenizerChain.class.isInstance(analyzer)) {
+      TokenStream tokenStream = analyzer.tokenStream(context.getFieldName(), new StringReader(value));
+      NamedList<List<NamedList>> namedList = new SimpleOrderedMap<List<NamedList>>();
+      namedList.add(tokenStream.getClass().getName(), convertTokensToNamedLists(analyzeTokenStream(tokenStream), context));
+      return namedList;
+    }
+
+    TokenizerChain tokenizerChain = (TokenizerChain) analyzer;
+
+    NamedList<List<NamedList>> namedList = new SimpleOrderedMap<List<NamedList>>();
+
+    TokenStream tokenStream = tokenizerChain.getTokenizerFactory().create(new StringReader(value));
+    List<Token> tokens = analyzeTokenStream(tokenStream);
+
+    namedList.add(tokenStream.getClass().getName(), convertTokensToNamedLists(tokens, context));
+
+    ListBasedTokenStream listBasedTokenStream = new ListBasedTokenStream(tokens);
+
+    for (TokenFilterFactory tokenFilterFactory : tokenizerChain.getTokenFilterFactories()) {
+      tokenStream = tokenFilterFactory.create(listBasedTokenStream);
+      List<Token> tokenList = analyzeTokenStream(tokenStream);
+      namedList.add(tokenStream.getClass().getName(), convertTokensToNamedLists(tokenList, context));
+      listBasedTokenStream = new ListBasedTokenStream(tokenList);
+    }
+
+    return namedList;
+  }
+
+  /**
+   * Analyzes the given text using the given analyzer and returns the produced tokens.
+   *
+   * @param value    The value to analyze.
+   * @param analyzer The analyzer to use.
+   *
+   * @return The produces token list.
+   */
+  protected List<Token> analyzeValue(String value, Analyzer analyzer) {
+    TokenStream tokenStream = analyzer.tokenStream("", new StringReader(value));
+    return analyzeTokenStream(tokenStream);
+  }
+
+  /**
+   * Analyzes the given TokenStream, collecting the Tokens it produces.
+   *
+   * @param tokenStream TokenStream to analyze
+   *
+   * @return List of tokens produced from the TokenStream
+   */
+  private List<Token> analyzeTokenStream(TokenStream tokenStream) {
+    List<Token> tokens = new ArrayList<Token>();
+    Token reusableToken = new Token();
+    Token token = null;
+
+    try {
+      while ((token = tokenStream.next(reusableToken)) != null) {
+        tokens.add((Token) token.clone());
+      }
+    } catch (IOException ioe) {
+      throw new RuntimeException("Error occured while iterating over tokenstream", ioe);
+    }
+
+    return tokens;
+  }
+
+  /**
+   * Converts the list of Tokens to a list of NamedLists representing the tokens.
+   *
+   * @param tokens  Tokens to convert
+   * @param context The analysis context
+   *
+   * @return List of NamedLists containing the relevant information taken from the tokens
+   */
+  private List<NamedList> convertTokensToNamedLists(List<Token> tokens, AnalysisContext context) {
+    List<NamedList> tokensNamedLists = new ArrayList<NamedList>();
+
+    Collections.sort(tokens, new Comparator<Token>() {
+      public int compare(Token o1, Token o2) {
+        return o1.endOffset() - o2.endOffset();
+      }
+    });
+
+    int position = 0;
+
+    FieldType fieldType = context.getFieldType();
+
+    for (Token token : tokens) {
+      NamedList<Object> tokenNamedList = new SimpleOrderedMap<Object>();
+
+      String text = fieldType.indexedToReadable(token.term());
+      tokenNamedList.add("text", text);
+      if (!text.equals(token.term())) {
+        tokenNamedList.add("raw_text", token.term());
+      }
+      tokenNamedList.add("type", token.type());
+      tokenNamedList.add("start", token.startOffset());
+      tokenNamedList.add("end", token.endOffset());
+
+      position += token.getPositionIncrement();
+      tokenNamedList.add("position", position);
+
+      if (context.getTermsToMatch().contains(token.term())) {
+        tokenNamedList.add("match", true);
+      }
+
+      if (token.getPayload() != null) {
+        tokenNamedList.add("payload", token.getPayload());
+      }
+
+      tokensNamedLists.add(tokenNamedList);
+    }
+
+    return tokensNamedLists;
+  }
+
+
+  // ================================================= Inner classes =================================================
+
+  /**
+   * TokenStream that iterates over a list of pre-existing Tokens
+   */
+  protected static class ListBasedTokenStream extends TokenStream {
+
+    private final Iterator<Token> tokenIterator;
+
+    /**
+     * Creates a new ListBasedTokenStream which uses the given tokens as its token source.
+     *
+     * @param tokens Source of tokens to be used
+     */
+    ListBasedTokenStream(List<Token> tokens) {
+      tokenIterator = tokens.iterator();
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Token next(Token token) throws IOException {
+      return (tokenIterator.hasNext()) ? tokenIterator.next() : null;
+    }
+  }
+
+  /**
+   * Serves as the context of an analysis process. This context contains the following constructs
+   */
+  protected static class AnalysisContext {
+
+    private final String fieldName;
+    private final FieldType fieldType;
+    private final Analyzer analyzer;
+    private final Set<String> termsToMatch;
+
+    /**
+     * Constructs a new AnalysisContext with a given field tpe, analyzer and termsToMatch. By default the field name in
+     * this context will be {@code null}. During the analysis processs, The produced tokens will be compaired to the
+     * termes in the {@code termsToMatch} set. When found, these tokens will be marked as a match.
+     *
+     * @param fieldType    The type of the field the analysis is performed on.
+     * @param analyzer     The analyzer to be used.
+     * @param termsToMatch Holds all the terms that should match during the analysis process.
+     */
+    public AnalysisContext(FieldType fieldType, Analyzer analyzer, Set<String> termsToMatch) {
+      this(null, fieldType, analyzer, termsToMatch);
+    }
+
+    /**
+     * Constructs an AnalysisContext with a given field name, field type and analyzer. By default this context will hold
+     * no terms to match
+     *
+     * @param fieldName The name of the field the analysis is performed on (may be {@code nuill}).
+     * @param fieldType The type of the field the analysis is performed on.
+     * @param analyzer  The analyzer to be used during the analysis process.
+     *
+     * @see #AnalysisContext(String, org.apache.solr.schema.FieldType, org.apache.lucene.analysis.Analyzer,
+     *      java.util.Set)
+     */
+    public AnalysisContext(String fieldName, FieldType fieldType, Analyzer analyzer) {
+      this(fieldName, fieldType, analyzer, Collections.EMPTY_SET);
+    }
+
+    /**
+     * Constructs a new AnalysisContext with a given field tpe, analyzer and termsToMatch. During the analysis processs,
+     * The produced tokens will be compaired to the termes in the {@codce termsToMatch} set. When found, these tokens
+     * will be marked as a match.
+     *
+     * @param fieldName    The name of the field the analysis is performed on (may be {@code null}).
+     * @param fieldType    The type of the field the analysis is performed on.
+     * @param analyzer     The analyzer to be used.
+     * @param termsToMatch Holds all the terms that should match during the analysis process.
+     */
+    public AnalysisContext(String fieldName, FieldType fieldType, Analyzer analyzer, Set<String> termsToMatch) {
+      this.fieldName = fieldName;
+      this.fieldType = fieldType;
+      this.analyzer = analyzer;
+      this.termsToMatch = termsToMatch;
+    }
+
+    public String getFieldName() {
+      return fieldName;
+    }
+
+    public FieldType getFieldType() {
+      return fieldType;
+    }
+
+    public Analyzer getAnalyzer() {
+      return analyzer;
+    }
+
+    public Set<String> getTermsToMatch() {
+      return termsToMatch;
+    }
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java?rev=767412&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java Wed Apr 22 08:53:28 2009
@@ -0,0 +1,348 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.handler;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.Token;
+import org.apache.solr.client.solrj.request.DocumentAnalysisRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.AnalysisParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamConstants;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
+import java.io.IOException;
+import java.io.Reader;
+import java.util.*;
+
+/**
+ * An analysis handler that provides a breakdown of the analysis process of provided docuemnts. This handler expects a
+ * (single) content stream of the following format:
+ * <p/>
+ * <pre><code>
+ *  &lt;docs&gt;
+ *      &lt;doc&gt;
+ *          &lt;field name="id"&gt;1&lt;/field&gt;
+ *          &lt;field name="name"&gt;The Name&lt;/field&gt;
+ *          &lt;field name="text"&gt;The Text Value&lt;/field&gt;
+ *      &lt;doc&gt;
+ *      &lt;doc&gt;...&lt;/doc&gt;
+ *      &lt;doc&gt;...&lt;/doc&gt;
+ *      ...
+ *  &lt;/docs&gt;
+ * </code></pre>
+ * <p/>
+ * <em><b>Note: Each document must contain a field which serves as the unique key. This key is used in the returned
+ * response to assoicate an analysis breakdown to the analyzed document.</b></em>
+ * <p/>
+ * <p/>
+ * <p/>
+ * Like the {@link org.apache.solr.handler.FieldAnalysisRequestHandler}, this handler also supports query analysis by
+ * sending either an "analysis.query" or "q" request paraemter that holds the query text to be analyized. It also
+ * supports the "analysis.showmatch" parameter which when set to {@code true}, all field tokens that match the query
+ * tokens will be marked as a "match".
+ *
+ * @version $Id$
+ * @since solr 1.4
+ */
+public class DocumentAnalysisRequestHandler extends AnalysisRequestHandlerBase {
+
+  public static final Logger log = LoggerFactory.getLogger(DocumentAnalysisRequestHandler.class);
+
+  private static final float DEFAULT_BOOST = 1.0f;
+
+  private XMLInputFactory inputFactory;
+
+  @Override
+  public void init(NamedList args) {
+    super.init(args);
+
+    inputFactory = XMLInputFactory.newInstance();
+    try {
+      // The java 1.6 bundled stax parser (sjsxp) does not currently have a thread-safe
+      // XMLInputFactory, as that implementation tries to cache and reuse the
+      // XMLStreamReader.  Setting the parser-specific "reuse-instance" property to false
+      // prevents this.
+      // All other known open-source stax parsers (and the bea ref impl)
+      // have thread-safe factories.
+      inputFactory.setProperty("reuse-instance", Boolean.FALSE);
+    } catch (IllegalArgumentException ex) {
+      // Other implementations will likely throw this exception since "reuse-instance"
+      // isimplementation specific.
+      log.debug("Unable to set the 'reuse-instance' property for the input factory: " + inputFactory);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  protected NamedList doAnalysis(SolrQueryRequest req) throws Exception {
+    DocumentAnalysisRequest analysisRequest = resolveAnalysisRequest(req);
+    return handleAnalysisRequest(analysisRequest, req.getSchema());
+  }
+
+  @Override
+  public String getDescription() {
+    return "Provides a breakdown of the analysis process of provided documents";
+  }
+
+  @Override
+  public String getVersion() {
+    return "$Revision$";
+  }
+
+  @Override
+  public String getSourceId() {
+    return "$Id$";
+  }
+
+  @Override
+  public String getSource() {
+    return "$URL$";
+  }
+
+
+  //================================================ Helper Methods ==================================================
+
+  /**
+   * Resolves the {@link DocumentAnalysisRequest} from the given solr request.
+   *
+   * @param req The solr request.
+   *
+   * @return The resolved document analysis request.
+   *
+   * @throws IOException        Thrown when reading/parsing the content stream of the request fails.
+   * @throws XMLStreamException Thrown when reading/parsing the content stream of the request fails.
+   */
+  DocumentAnalysisRequest resolveAnalysisRequest(SolrQueryRequest req) throws IOException, XMLStreamException {
+
+    DocumentAnalysisRequest request = new DocumentAnalysisRequest();
+
+    SolrParams params = req.getParams();
+
+    String query = params.get(AnalysisParams.QUERY, params.get(CommonParams.Q, null));
+    request.setQuery(query);
+
+    boolean showMatch = params.getBool(AnalysisParams.SHOW_MATCH, false);
+    request.setShowMatch(showMatch);
+
+    ContentStream stream = extractSingleContentStream(req);
+    Reader reader = stream.getReader();
+    XMLStreamReader parser = inputFactory.createXMLStreamReader(reader);
+
+    try {
+
+      while (true) {
+        int event = parser.next();
+        switch (event) {
+          case XMLStreamConstants.END_DOCUMENT: {
+            parser.close();
+            return request;
+          }
+          case XMLStreamConstants.START_ELEMENT: {
+            String currTag = parser.getLocalName();
+            if ("doc".equals(currTag)) {
+              log.trace("Reading doc...");
+              SolrInputDocument document = readDocument(parser, req.getSchema());
+              request.addDocument(document);
+            }
+            break;
+          }
+        }
+      }
+
+    } finally {
+      parser.close();
+      IOUtils.closeQuietly(reader);
+    }
+  }
+
+  /**
+   * Handles the resolved {@link DocumentAnalysisRequest} and returns the analysis response as a named list.
+   *
+   * @param request The {@link DocumentAnalysisRequest} to be handled.
+   * @param schema  The index schema.
+   *
+   * @return The analysis response as a named list.
+   */
+  NamedList<Object> handleAnalysisRequest(DocumentAnalysisRequest request, IndexSchema schema) {
+
+    SchemaField uniqueKeyField = schema.getUniqueKeyField();
+    NamedList<Object> result = new SimpleOrderedMap<Object>();
+
+    for (SolrInputDocument document : request.getDocuments()) {
+
+      NamedList<NamedList> theTokens = new SimpleOrderedMap<NamedList>();
+      result.add(document.getFieldValue(uniqueKeyField.getName()).toString(), theTokens);
+      for (String name : document.getFieldNames()) {
+
+        // there's no point of providing analysis to unindexed fields.
+        SchemaField field = schema.getField(name);
+        if (!field.indexed()) {
+          continue;
+        }
+
+        NamedList<Object> fieldTokens = new SimpleOrderedMap<Object>();
+        theTokens.add(name, fieldTokens);
+
+        FieldType fieldType = schema.getFieldType(name);
+
+        Set<String> termsToMatch = new HashSet<String>();
+        if (request.getQuery() != null && request.isShowMatch()) {
+          List<Token> tokens = analyzeValue(request.getQuery(), fieldType.getQueryAnalyzer());
+          for (Token token : tokens) {
+            termsToMatch.add(token.term());
+          }
+        }
+
+        if (request.getQuery() != null) {
+          AnalysisContext analysisContext = new AnalysisContext(fieldType, fieldType.getQueryAnalyzer(), Collections.EMPTY_SET);
+          NamedList<List<NamedList>> tokens = analyzeValue(request.getQuery(), analysisContext);
+          fieldTokens.add("query", tokens);
+        }
+
+        Analyzer analyzer = fieldType.getAnalyzer();
+        AnalysisContext analysisContext = new AnalysisContext(fieldType, analyzer, termsToMatch);
+        Collection<Object> fieldValues = document.getFieldValues(name);
+        NamedList<NamedList<List<NamedList>>> indexTokens = new SimpleOrderedMap<NamedList<List<NamedList>>>();
+        for (Object fieldValue : fieldValues) {
+          NamedList<List<NamedList>> tokens = analyzeValue(fieldValue.toString(), analysisContext);
+          indexTokens.add(String.valueOf(fieldValue), tokens);
+        }
+        fieldTokens.add("index", indexTokens);
+      }
+    }
+
+    return result;
+  }
+
+  /**
+   * Reads the document from the given xml stream reader. The following document format is expected:
+   * <p/>
+   * <pre><code>
+   * &lt;doc&gt;
+   *    &lt;field name="id"&gt;1&lt;/field&gt;
+   *    &lt;field name="name"&gt;The Name&lt;/field&gt;
+   *    &lt;field name="text"&gt;The Text Value&lt;/field&gt;
+   * &lt;/doc&gt;
+   * </code></pre>
+   * <p/>
+   * <p/>
+   * <em>NOTE: each read document is expected to have at least one field which serves as the unique key.</em>
+   *
+   * @param reader The {@link XMLStreamReader} from which the document will be read.
+   * @param schema The index schema. The schema is used to validate that the read document has a unique key field.
+   *
+   * @return The read document.
+   *
+   * @throws XMLStreamException When reading of the document fails.
+   */
+  SolrInputDocument readDocument(XMLStreamReader reader, IndexSchema schema) throws XMLStreamException {
+    SolrInputDocument doc = new SolrInputDocument();
+
+    String uniqueKeyField = schema.getUniqueKeyField().getName();
+
+    StringBuilder text = new StringBuilder();
+    String fieldName = null;
+    boolean hasId = false;
+
+    while (true) {
+      int event = reader.next();
+      switch (event) {
+        // Add everything to the text
+        case XMLStreamConstants.SPACE:
+        case XMLStreamConstants.CDATA:
+        case XMLStreamConstants.CHARACTERS:
+          text.append(reader.getText());
+          break;
+
+        case XMLStreamConstants.END_ELEMENT:
+          if ("doc".equals(reader.getLocalName())) {
+            if (!hasId) {
+              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                      "All documents must contain a unique key value: '" + doc.toString() + "'");
+            }
+            return doc;
+          } else if ("field".equals(reader.getLocalName())) {
+            doc.addField(fieldName, text.toString(), DEFAULT_BOOST);
+            if (uniqueKeyField.equals(fieldName)) {
+              hasId = true;
+            }
+          }
+          break;
+
+        case XMLStreamConstants.START_ELEMENT:
+          text.setLength(0);
+          String localName = reader.getLocalName();
+          if (!"field".equals(localName)) {
+            log.warn("unexpected XML tag doc/" + localName);
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "unexpected XML tag doc/" + localName);
+          }
+
+          for (int i = 0; i < reader.getAttributeCount(); i++) {
+            String attrName = reader.getAttributeLocalName(i);
+            if ("name".equals(attrName)) {
+              fieldName = reader.getAttributeValue(i);
+            }
+          }
+          break;
+      }
+    }
+  }
+
+  /**
+   * Extracts the only content stream from the request. {@link org.apache.solr.common.SolrException.ErrorCode#BAD_REQUEST}
+   * error is thrown if the request doesn't hold any content stream or holds more than one.
+   *
+   * @param req The solr request.
+   *
+   * @return The single content stream which holds the documents to be analyzed.
+   */
+  private ContentStream extractSingleContentStream(SolrQueryRequest req) {
+    Iterable<ContentStream> streams = req.getContentStreams();
+    if (streams == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "DocumentAnlysisRequestHandler expects a single content stream with documents to analys");
+    }
+    Iterator<ContentStream> iter = streams.iterator();
+    if (!iter.hasNext()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "DocumentAnlysisRequestHandler expects a single content stream with documents to analys");
+    }
+    ContentStream stream = iter.next();
+    if (iter.hasNext()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "DocumentAnlysisRequestHandler expects a single content stream with documents to analys");
+    }
+    return stream;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/FieldAnalysisRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/FieldAnalysisRequestHandler.java?rev=767412&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/FieldAnalysisRequestHandler.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/FieldAnalysisRequestHandler.java Wed Apr 22 08:53:28 2009
@@ -0,0 +1,210 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.handler;
+
+import org.apache.lucene.analysis.Token;
+import org.apache.solr.client.solrj.request.FieldAnalysisRequest;
+import org.apache.solr.common.params.AnalysisParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * RequestHandler that provides much the same functionality as analysis.jsp.  Provides the ability to specify multiple
+ * field types and field names in the same request. Expected parameters:
+ * <table border="1">
+ * <tr>
+ * <th align="left">Name</th>
+ * <th align="left">Type</th>
+ * <th align="left">required</th>
+ * <th align="left">Description</th>
+ * <th align="left">Multi-valued</th>
+ * </tr>
+ * <tr>
+ * <td>analysis.fieldname</td>
+ * <td>string</td>
+ * <td>no</td>
+ * <td>When present, the text will be analyzed based on the type of this field name.</td>
+ * <td>Yes, this parameter may hold a comma-separated list of values and the analysis will be performed for each of the specified fields</td>
+ * </tr>
+ * <tr>
+ * <td>analysis.fieldtype</td>
+ * <td>string</td>
+ * <td>no</td>
+ * <td>When present, the text will be analyzed based on the specified type</td>
+ * <td>Yes, this parameter may hold a comma-separated list of values and the analysis will be performed for each of the specified field types</td>
+ * </tr>
+ * <tr>
+ * <td>analysis.fieldvalue</td>
+ * <td>string</td>
+ * <td>yes</td>
+ * <td>The text that will be analyzed. The analysis will mimic the index-time analysis.</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>{@code analysis.query} OR {@code q}</td>
+ * <td>string</td>
+ * <td>no</td>
+ * <td>When present, the text that will be analyzed. The analysis will mimic the query-time analysis. Note that the
+ * {@code analysis.query} parameter as precedes the {@code q} parameters.</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>analysis.showmatch</td>
+ * <td>boolean</td>
+ * <td>no</td>
+ * <td>When set to {@code true} and when query analysis is performed, the produced tokens of the field value
+ * analysis will be marked as "matched" for every token that is produces by the query analysis</td>
+ * <td>No</td>
+ * </tr>
+ * </table>
+ *
+ * @version $Id$
+ * @since solr 1.4 
+ */
+public class FieldAnalysisRequestHandler extends AnalysisRequestHandlerBase {
+
+  /**
+   * {@inheritDoc}
+   */
+  protected NamedList doAnalysis(SolrQueryRequest req) throws Exception {
+    FieldAnalysisRequest analysisRequest = resolveAnalysisRequest(req.getParams());
+    IndexSchema indexSchema = req.getCore().getSchema();
+    return handleAnalysisRequest(analysisRequest, indexSchema);
+  }
+
+  @Override
+  public String getDescription() {
+    return "Provide a breakdown of the analysis process of field/query text";
+  }
+
+  @Override
+  public String getVersion() {
+    return "$Revision$";
+  }
+
+  @Override
+  public String getSourceId() {
+    return "$Id$";
+  }
+
+  @Override
+  public String getSource() {
+    return "$URL$";
+  }
+
+  // ================================================= Helper methods ================================================
+
+  /**
+   * Resolves the AnalysisRequest based on the parameters in the given SolrParams.
+   *
+   * @param solrParams SolrParams taken from request
+   *
+   * @return AnalysisRequest containing all the information about what needs to be analyzed, and using what
+   *         fields/types
+   */
+  FieldAnalysisRequest resolveAnalysisRequest(SolrParams solrParams) {
+    FieldAnalysisRequest analysisRequest = new FieldAnalysisRequest();
+
+    if (solrParams.get(AnalysisParams.FIELD_TYPE) != null) {
+      analysisRequest.setFieldTypes(Arrays.asList(solrParams.get(AnalysisParams.FIELD_TYPE).split(",")));
+    }
+    if (solrParams.get(AnalysisParams.FIELD_NAME) != null) {
+      analysisRequest.setFieldNames(Arrays.asList(solrParams.get(AnalysisParams.FIELD_NAME).split(",")));
+    }
+    analysisRequest.setQuery(solrParams.get(AnalysisParams.QUERY, solrParams.get(CommonParams.Q)));
+    analysisRequest.setFieldValue(solrParams.get(AnalysisParams.FIELD_VALUE));
+    analysisRequest.setShowMatch(solrParams.getBool(AnalysisParams.SHOW_MATCH, false));
+    return analysisRequest;
+  }
+
+  /**
+   * Handles the resolved analysis request and returns the analysis breakdown response as a named list.
+   *
+   * @param request The request to handle.
+   * @param schema  The index schema.
+   *
+   * @return The analysis breakdown as a named list.
+   */
+  protected NamedList<NamedList> handleAnalysisRequest(FieldAnalysisRequest request, IndexSchema schema) {
+    NamedList<NamedList> analysisResults = new SimpleOrderedMap<NamedList>();
+
+    NamedList<NamedList> fieldTypeAnalysisResults = new SimpleOrderedMap<NamedList>();
+    for (String fieldTypeName : request.getFieldTypes()) {
+      FieldType fieldType = schema.getFieldTypes().get(fieldTypeName);
+      fieldTypeAnalysisResults.add(fieldTypeName, analyzeValues(request, fieldType, null));
+    }
+
+    NamedList<NamedList> fieldNameAnalysisResults = new SimpleOrderedMap<NamedList>();
+    for (String fieldName : request.getFieldNames()) {
+      FieldType fieldType = schema.getFieldType(fieldName);
+      fieldNameAnalysisResults.add(fieldName, analyzeValues(request, fieldType, fieldName));
+    }
+
+    analysisResults.add("field_types", fieldTypeAnalysisResults);
+    analysisResults.add("field_names", fieldNameAnalysisResults);
+
+    return analysisResults;
+  }
+
+  /**
+   * Analyzes the index value (if it exists) and the query value (if it exists) in the given AnalysisRequest, using
+   * the Analyzers of the given field type.
+   *
+   * @param analysisRequest AnalysisRequest from where the index and query values will be taken
+   * @param fieldType       Type of field whose analyzers will be used
+   * @param fieldName       Name of the field to be analyzed.  Can be {@code null}
+   *
+   * @return NamedList containing the tokens produced by the analyzers of the given field, separated into an index and
+   *         a query group
+   */
+  private NamedList<NamedList> analyzeValues(FieldAnalysisRequest analysisRequest, FieldType fieldType, String fieldName) {
+
+    Set<String> termsToMatch = new HashSet<String>();
+    String queryValue = analysisRequest.getQuery();
+    if (queryValue != null && analysisRequest.isShowMatch()) {
+      List<Token> tokens = analyzeValue(queryValue, fieldType.getQueryAnalyzer());
+      for (Token token : tokens) {
+        termsToMatch.add(token.term());
+      }
+    }
+
+    NamedList<NamedList> analyzeResults = new SimpleOrderedMap<NamedList>();
+    if (analysisRequest.getFieldValue() != null) {
+      AnalysisContext context = new AnalysisContext(fieldName, fieldType, fieldType.getAnalyzer(), termsToMatch);
+      NamedList analyzedTokens = analyzeValue(analysisRequest.getFieldValue(), context);
+      analyzeResults.add("index", analyzedTokens);
+    }
+    if (analysisRequest.getQuery() != null) {
+      AnalysisContext context = new AnalysisContext(fieldName, fieldType, fieldType.getQueryAnalyzer());
+      NamedList analyzedTokens = analyzeValue(analysisRequest.getQuery(), context);
+      analyzeResults.add("query", analyzedTokens);
+    }
+
+    return analyzeResults;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/FieldAnalysisRequestHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/FieldAnalysisRequestHandler.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java?rev=767412&view=auto
==============================================================================
--- lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java (added)
+++ lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java Wed Apr 22 08:53:28 2009
@@ -0,0 +1,207 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.client.solrj.request;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServer;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.DocumentAnalysisResponse;
+import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.AnalysisParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ContentStream;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * A request for the org.apache.solr.handler.DocumentAnalysisRequestHandler.
+ *
+ * @version $Id$
+ * @since solr 1.4
+ */
+public class DocumentAnalysisRequest extends SolrRequest {
+
+  private List<SolrInputDocument> documents = new ArrayList<SolrInputDocument>();
+  private String query;
+  private boolean showMatch = false;
+
+  /**
+   * Constructs a new request with a default uri of "/documentanalysis".
+   */
+  public DocumentAnalysisRequest() {
+    super(METHOD.POST, "/analysis/document");
+  }
+
+  /**
+   * Constructs a new request with the given request handler uri.
+   *
+   * @param uri The of the request handler.
+   */
+  public DocumentAnalysisRequest(String uri) {
+    super(METHOD.POST, uri);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Collection<ContentStream> getContentStreams() throws IOException {
+    return ClientUtils.toContentStreams(getXML(), ClientUtils.TEXT_XML);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public ModifiableSolrParams getParams() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    if (query != null) {
+      params.add(AnalysisParams.QUERY, query);
+      params.add(AnalysisParams.SHOW_MATCH, String.valueOf(showMatch));
+    }
+    return params;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public DocumentAnalysisResponse process(SolrServer server) throws SolrServerException, IOException {
+    long startTime = System.currentTimeMillis();
+    DocumentAnalysisResponse res = new DocumentAnalysisResponse();
+    res.setResponse(server.request(this));
+    res.setElapsedTime(System.currentTimeMillis() - startTime);
+    return res;
+  }
+
+  //================================================ Helper Methods ==================================================
+
+  /**
+   * Returns the xml be be set as the request body.
+   *
+   * @return The xml be be set as the request body.
+   *
+   * @throws IOException When constructing the xml fails
+   */
+  String getXML() throws IOException {
+    StringWriter writer = new StringWriter();
+    writer.write("<docs>");
+    for (SolrInputDocument document : documents) {
+      ClientUtils.writeXML(document, writer);
+    }
+    writer.write("</docs>");
+    writer.flush();
+
+    String xml = writer.toString();
+    return (xml.length() > 0) ? xml : null;
+  }
+
+
+  //============================================ Setter/Getter Methods ===============================================
+
+  /**
+   * Adds a document to be analyzed.
+   *
+   * @param doc The document to be analyzed.
+   *
+   * @return This DocumentAnalysisRequest (fluent interface support).
+   */
+  public DocumentAnalysisRequest addDocument(SolrInputDocument doc) {
+    documents.add(doc);
+    return this;
+  }
+
+  /**
+   * Adds a collection of documents to be analyzed.
+   *
+   * @param docs The documents to be analyzed.
+   *
+   * @return This DocumentAnalysisRequest (fluent interface support).
+   *
+   * @see #addDocument(org.apache.solr.common.SolrInputDocument)
+   */
+  public DocumentAnalysisRequest addDocuments(Collection<SolrInputDocument> docs) {
+    documents.addAll(docs);
+    return this;
+  }
+
+  /**
+   * Sets the query to be analyzed. By default the query is set to null, meaning no query analysis will be performed.
+   *
+   * @param query The query to be analyzed.
+   *
+   * @return This DocumentAnalysisRequest (fluent interface support).
+   */
+  public DocumentAnalysisRequest setQuery(String query) {
+    this.query = query;
+    return this;
+  }
+
+  /**
+   * Sets whether index time tokens that match query time tokens should be marked as a "match". By default this is set
+   * to {@code false}. Obviously, this flag is ignored if when the query is set to {@code null}.
+   *
+   * @param showMatch Sets whether index time tokens that match query time tokens should be marked as a "match".
+   *
+   * @return This DocumentAnalysisRequest (fluent interface support).
+   */
+  public DocumentAnalysisRequest setShowMatch(boolean showMatch) {
+    this.showMatch = showMatch;
+    return this;
+  }
+
+  /**
+   * Returns all documents that will be analyzed when processing the request.
+   *
+   * @return All documents that will be analyzed when processing the request.
+   *
+   * @see #addDocument(org.apache.solr.common.SolrInputDocument)
+   */
+  public List<SolrInputDocument> getDocuments() {
+    return documents;
+  }
+
+  /**
+   * Returns the query that will be analyzed when processing the request. May return {@code null} indicating that no
+   * query time analysis is taking place.
+   *
+   * @return The query that will be analyzed when processing the request.
+   *
+   * @see #setQuery(String)
+   */
+  public String getQuery() {
+    return query;
+  }
+
+  /**
+   * Returns whether index time tokens that match query time tokens will be marked as a "match".
+   *
+   * @return Whether index time tokens that match query time tokens will be marked as a "match".
+   *
+   * @see #setShowMatch(boolean)
+   */
+  public boolean isShowMatch() {
+    return showMatch;
+  }
+
+}

Propchange: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java?rev=767412&view=auto
==============================================================================
--- lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java (added)
+++ lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java Wed Apr 22 08:53:28 2009
@@ -0,0 +1,279 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.client.solrj.request;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServer;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.FieldAnalysisResponse;
+import org.apache.solr.common.params.AnalysisParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * A request for the org.apache.solr.handler.DocumentAnalysisRequestHandler.
+ *
+ * @version $Id$
+ * @since solr.14
+ */
+public class FieldAnalysisRequest extends SolrRequest {
+
+  private String fieldValue;
+  private String query;
+  private boolean showMatch;
+  private List<String> fieldNames;
+  private List<String> fieldTypes;
+
+  /**
+   * Constructs a new FieldAnalysisRequest with a default uri of "/fieldanalysis".
+   */
+  public FieldAnalysisRequest() {
+    super(METHOD.GET, "/analysis/field");
+  }
+
+  /**
+   * Constructs a new FieldAnalysisRequest with a given uri.
+   *
+   * @param uri the uri of the request handler.
+   */
+  public FieldAnalysisRequest(String uri) {
+    super(METHOD.GET, uri);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Collection<ContentStream> getContentStreams() throws IOException {
+    return null;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public SolrParams getParams() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(AnalysisParams.FIELD_VALUE, fieldValue);
+    if (query != null) {
+      params.add(AnalysisParams.QUERY, query);
+      params.add(AnalysisParams.SHOW_MATCH, String.valueOf(showMatch));
+    }
+    if (fieldNames != null) {
+      String fieldNameValue = listToCommaDelimitedString(fieldNames);
+      params.add(AnalysisParams.FIELD_NAME, fieldNameValue);
+    }
+    if (fieldTypes != null) {
+      String fieldTypeValue = listToCommaDelimitedString(fieldTypes);
+      params.add(AnalysisParams.FIELD_TYPE, fieldTypeValue);
+    }
+    return params;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public FieldAnalysisResponse process(SolrServer server) throws SolrServerException, IOException {
+    if (fieldTypes == null || fieldNames == null) {
+      throw new IllegalStateException("A list one field type or field name need to be specified");
+    }
+    if (fieldValue == null) {
+      throw new IllegalStateException("The field value must be set");
+    }
+    long startTime = System.currentTimeMillis();
+    FieldAnalysisResponse res = new FieldAnalysisResponse();
+    res.setResponse(server.request(this));
+    res.setElapsedTime(System.currentTimeMillis() - startTime);
+    return res;
+  }
+
+
+  //================================================ Helper Methods ==================================================
+
+  /**
+   * Convers the given list of string to a comma-separated string.
+   *
+   * @param list The list of string.
+   *
+   * @return The comma-separated string.
+   */
+  static String listToCommaDelimitedString(List<String> list) {
+    StringBuilder result = new StringBuilder();
+    for (String str : list) {
+      if (result.length() > 0) {
+        result.append(",");
+      }
+      result.append(str);
+    }
+    return result.toString();
+  }
+
+
+  //============================================ Setter/Getter Methods ===============================================
+
+  /**
+   * Sets the field value to be analyzed.
+   *
+   * @param fieldValue The field value to be analyzed.
+   *
+   * @return This FieldAnalysisRequest (fluent interface support).
+   */
+  public FieldAnalysisRequest setFieldValue(String fieldValue) {
+    this.fieldValue = fieldValue;
+    return this;
+  }
+
+  /**
+   * Returns the field value that will be analyzed when this request is processed.
+   *
+   * @return The field value that will be analyzed when this request is processed.
+   */
+  public String getFieldValue() {
+    return fieldValue;
+  }
+
+  /**
+   * Sets the query to be analyzed. May be {@code null} indicated that no query analysis should take place.
+   *
+   * @param query The query to be analyzed.
+   *
+   * @return This FieldAnalysisRequest (fluent interface support).
+   */
+  public FieldAnalysisRequest setQuery(String query) {
+    this.query = query;
+    return this;
+  }
+
+  /**
+   * Returns the query that will be analyzed. May return {@code null} indicating that no query analysis will be
+   * performed.
+   *
+   * @return The query that will be analyzed. May return {@code null} indicating that no query analysis will be
+   *         performed.
+   */
+  public String getQuery() {
+    return query;
+  }
+
+  /**
+   * Sets whether index time tokens that match query time tokens should be marked as a "match". By default this is set
+   * to {@code false}. Obviously, this flag is ignored if when the query is set to {@code null}.
+   *
+   * @param showMatch Sets whether index time tokens that match query time tokens should be marked as a "match".
+   *
+   * @return This FieldAnalysisRequest (fluent interface support).
+   */
+  public FieldAnalysisRequest setShowMatch(boolean showMatch) {
+    this.showMatch = showMatch;
+    return this;
+  }
+
+  /**
+   * Returns whether index time tokens that match query time tokens should be marked as a "match".
+   *
+   * @return Whether index time tokens that match query time tokens should be marked as a "match".
+   *
+   * @see #setShowMatch(boolean)
+   */
+  public boolean isShowMatch() {
+    return showMatch;
+  }
+
+  /**
+   * Adds the given field name for analysis.
+   *
+   * @param fieldName A field name on which the analysis should be performed.
+   *
+   * @return this FieldAnalysisRequest (fluent interface support).
+   */
+  public FieldAnalysisRequest addFieldName(String fieldName) {
+    if (fieldNames == null) {
+      fieldNames = new LinkedList<String>();
+    }
+    fieldNames.add(fieldName);
+    return this;
+  }
+
+  /**
+     * Sets the field names on which the analysis should be performed.
+     *
+     * @param fieldNames The field names on which the analysis should be performed.
+     *
+     * @return this FieldAnalysisRequest (fluent interface support).
+     */
+  public FieldAnalysisRequest setFieldNames(List<String> fieldNames) {
+    this.fieldNames = fieldNames;
+    return this;
+  }
+
+  /**
+   * Returns a list of field names the analysis should be performed on. May return {@code null} indicating that no
+   * analysis will be performed on field names.
+   *
+   * @return The field names the analysis should be performed on.
+   */
+  public List<String> getFieldNames() {
+    return fieldNames;
+  }
+
+  /**
+   * Adds the given field type for analysis.
+   *
+   * @param fieldTypeName A field type name on which analysis should be performed.
+   *
+   * @return This FieldAnalysisRequest (fluent interface support).
+   */
+  public FieldAnalysisRequest addFieldType(String fieldTypeName) {
+    if (fieldTypes == null) {
+      fieldTypes = new LinkedList<String>();
+    }
+    fieldTypes.add(fieldTypeName);
+    return this;
+  }
+
+/**
+   * Sets the field types on which analysis should be performed.
+   *
+   * @param fieldTypes The field type names on which analysis should be performed.
+   *
+   * @return This FieldAnalysisRequest (fluent interface support).
+   */
+  public FieldAnalysisRequest setFieldTypes(List<String> fieldTypes) {
+    this.fieldTypes = fieldTypes;
+    return this;
+  }
+
+
+  /**
+   * Returns a list of field types the analysis should be performed on. May return {@code null} indicating that no
+   * analysis will be peformed on field types.
+   *
+   * @return The field types the analysis should be performed on.
+   */
+  public List<String> getFieldTypes() {
+    return fieldTypes;
+  }
+
+}

Propchange: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/AnalysisResponseBase.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/AnalysisResponseBase.java?rev=767412&view=auto
==============================================================================
--- lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/AnalysisResponseBase.java (added)
+++ lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/AnalysisResponseBase.java Wed Apr 22 08:53:28 2009
@@ -0,0 +1,252 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.client.solrj.response;
+
+import org.apache.solr.common.util.NamedList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A base class for all analysis responses.
+ *
+ * @version $Id$
+ * @since solr 1.4
+ */
+public class AnalysisResponseBase extends SolrResponseBase {
+
+  /**
+   * Parses the given named list and builds a list of analysis phases form it. Expects a named list of the form:
+   * <p/>
+   * <pre><code>
+   *  &lt;lst name="index"&gt;
+   *      &lt;arr name="Tokenizer"&gt;
+   *          &lt;str name="text"&gt;the_text&lt;/str&gt;
+   *          &lt;str name="rawText"&gt;the_raw_text&lt;/str&gt; (optional)
+   *          &lt;str name="type"&gt;the_type&lt;/str&gt;
+   *          &lt;int name="start"&gt;1&lt;/str&gt;
+   *          &lt;int name="end"&gt;3&lt;/str&gt;
+   *          &lt;int name="position"&gt;1&lt;/str&gt;
+   *          &lt;bool name="match"&gt;true | false&lt;/bool&gt; (optional)
+   *      &lt;/arr&gt;
+   *      &lt;arr name="Filter1"&gt;
+   *          &lt;str name="text"&gt;the_text&lt;/str&gt;
+   *          &lt;str name="rawText"&gt;the_raw_text&lt;/str&gt; (optional)
+   *          &lt;str name="type"&gt;the_type&lt;/str&gt;
+   *          &lt;int name="start"&gt;1&lt;/str&gt;
+   *          &lt;int name="end"&gt;3&lt;/str&gt;
+   *          &lt;int name="position"&gt;1&lt;/str&gt;
+   *          &lt;bool name="match"&gt;true | false&lt;/bool&gt; (optional)
+   *      &lt;/arr&gt;
+   *      ...
+   *  &lt;/lst&gt;
+   * </code></pre>
+   *
+   * @param phaseNL The names list to parse.
+   *
+   * @return The built analysis phases list.
+   */
+  protected List<AnalysisPhase> buildPhases(NamedList<Object> phaseNL) {
+    List<AnalysisPhase> phases = new ArrayList<AnalysisPhase>(phaseNL.size());
+    for (Map.Entry<String, Object> phaseEntry : phaseNL) {
+      AnalysisPhase phase = new AnalysisPhase(phaseEntry.getKey());
+      List<NamedList> tokens = (List<NamedList>) phaseEntry.getValue();
+      for (NamedList token : tokens) {
+        TokenInfo tokenInfo = buildTokenInfo(token);
+        phase.addTokenInfo(tokenInfo);
+      }
+      phases.add(phase);
+    }
+    return phases;
+  }
+
+  /**
+   * Parses the given named list and builds a token infoform it. Expects a named list of the form:
+   * <p/>
+   * <pre><code>
+   *  &lt;arr name="Tokenizer"&gt;
+   *      &lt;str name="text"&gt;the_text&lt;/str&gt;
+   *      &lt;str name="rawText"&gt;the_raw_text&lt;/str&gt; (optional)
+   *      &lt;str name="type"&gt;the_type&lt;/str&gt;
+   *      &lt;int name="start"&gt;1&lt;/str&gt;
+   *      &lt;int name="end"&gt;3&lt;/str&gt;
+   *      &lt;int name="position"&gt;1&lt;/str&gt;
+   *      &lt;bool name="match"&gt;true | false&lt;/bool&gt; (optional)
+   *  &lt;/arr&gt;
+   * </code></pre>
+   *
+   * @param tokenNL The named list to parse.
+   *
+   * @return The built token info.
+   */
+  protected TokenInfo buildTokenInfo(NamedList tokenNL) {
+    String text = (String) tokenNL.get("text");
+    String rawText = (String) tokenNL.get("rawText");
+    String type = (String) tokenNL.get("type");
+    int start = (Integer) tokenNL.get("start");
+    int end = (Integer) tokenNL.get("end");
+    int position = (Integer) tokenNL.get("position");
+    Boolean match = (Boolean) tokenNL.get("match");
+    return new TokenInfo(text, rawText, type, start, end, position, (match == null ? false : match));
+  }
+
+
+  //================================================= Inner Classes ==================================================
+
+  /**
+   * A phase in the analysis process. The phase holds the tokens produced in this phase and the name of the class that
+   * produced them.
+   */
+  public static class AnalysisPhase {
+
+    private final String className;
+    private List<TokenInfo> tokens = new ArrayList<TokenInfo>();
+
+    AnalysisPhase(String className) {
+      this.className = className;
+    }
+
+    /**
+     * The name of the class (analyzer, tokenzier, or filter) that produced the token stream for this phase.
+     *
+     * @return The name of the class that produced the token stream for this phase.
+     */
+    public String getClassName() {
+      return className;
+    }
+
+    private void addTokenInfo(TokenInfo tokenInfo) {
+      tokens.add(tokenInfo);
+    }
+
+    /**
+     * Returns a list of tokens which represent the token stream produced in this phase.
+     *
+     * @return A list of tokens which represent the token stream produced in this phase.
+     */
+    public List<TokenInfo> getTokens() {
+      return tokens;
+    }
+
+  }
+
+  /**
+   * Holds all information of a token as part of an analysis phase.
+   */
+  public static class TokenInfo {
+
+    private final String text;
+    private final String rawText;
+    private final String type;
+    private final int start;
+    private final int end;
+    private final int position;
+    private final boolean match;
+
+    /**
+     * Constructs a new TokenInfo.
+     *
+     * @param text     The text of the token
+     * @param rawText  The raw text of the token. If the token is stored in the index in a special format (e.g.
+     *                 dates or padded numbers) this argument should hold this value. If the token is stored as is,
+     *                 then this value should be {@code null}.
+     * @param type     The type fo the token (typically either {@code word} or {@code <ALPHANUM>} though it depends
+     *                 on the tokenizer/filter used).
+     * @param start    The start position of the token in the original text where it was extracted from.
+     * @param end      The end position of the token in the original text where it was extracted from.
+     * @param position The position of the token within the token stream.
+     * @param match    Indicates whether this token matches one of the the query tokens.
+     */
+    TokenInfo(String text, String rawText, String type, int start, int end, int position, boolean match) {
+      this.text = text;
+      this.rawText = rawText;
+      this.type = type;
+      this.start = start;
+      this.end = end;
+      this.position = position;
+      this.match = match;
+    }
+
+    /**
+     * Returns the text of the token.
+     *
+     * @return The text of the token.
+     */
+    public String getText() {
+      return text;
+    }
+
+    /**
+     * Returns the raw text of the token. If the token is index in a special format (e.g. date or paddded numbers)
+     * it will be returned as the raw text. Returns {@code null} if the token is indexed as is.
+     *
+     * @return Returns the raw text of the token.
+     */
+    public String getRawText() {
+      return rawText;
+    }
+
+    /**
+     * Returns the type of the token. Typically this will be {@code word} or {@code <ALPHANUM>}, but it really
+     * depends on the tokenizer and filters that are used.
+     *
+     * @return The type of the token.
+     */
+    public String getType() {
+      return type;
+    }
+
+    /**
+     * Returns the start position of this token within the text it was originally extracted from.
+     *
+     * @return The start position of this token within the text it was originally extracted from.
+     */
+    public int getStart() {
+      return start;
+    }
+
+    /**
+     * Returns the end position of this token within the text it was originally extracted from.
+     *
+     * @return The end position of this token within the text it was originally extracted from.
+     */
+    public int getEnd() {
+      return end;
+    }
+
+    /**
+     * Returns the position of this token within the produced token stream.
+     *
+     * @return The position of this token within the produced token stream.
+     */
+    public int getPosition() {
+      return position;
+    }
+
+    /**
+     * Returns whether this token matches one of the query tokens (if query analysis is performed).
+     *
+     * @return Whether this token matches one of the query tokens (if query analysis is performed).
+     */
+    public boolean isMatch() {
+      return match;
+    }
+  }
+
+}

Propchange: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/AnalysisResponseBase.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/AnalysisResponseBase.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/DocumentAnalysisResponse.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/DocumentAnalysisResponse.java?rev=767412&view=auto
==============================================================================
--- lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/DocumentAnalysisResponse.java (added)
+++ lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/DocumentAnalysisResponse.java Wed Apr 22 08:53:28 2009
@@ -0,0 +1,250 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.client.solrj.response;
+
+import org.apache.solr.common.util.NamedList;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A response that is returned by processing the {@link org.apache.solr.client.solrj.request.DocumentAnalysisRequest}.
+ * Holds a map of {@link DocumentAnalysis} objects by a document id (unique key).
+ *
+ * @version $Id$
+ * @since solr 1.4
+ */
+public class DocumentAnalysisResponse extends AnalysisResponseBase implements Iterable<Map.Entry<String, DocumentAnalysisResponse.DocumentAnalysis>> {
+
+  private final Map<String, DocumentAnalysis> documentAnalysisByKey = new HashMap<String, DocumentAnalysis>();
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void setResponse(NamedList<Object> response) {
+    super.setResponse(response);
+
+    NamedList<Object> analysis = (NamedList<Object>) response.get("analysis");
+    for (Map.Entry<String, Object> documentEntry : analysis) {
+      DocumentAnalysis documentAnalysis = new DocumentAnalysis(documentEntry.getKey());
+      NamedList<Object> document = (NamedList<Object>) documentEntry.getValue();
+      for (Map.Entry<String, Object> fieldEntry : document) {
+        FieldAnalysis fieldAnalysis = new FieldAnalysis(fieldEntry.getKey());
+        NamedList field = (NamedList) fieldEntry.getValue();
+
+        NamedList<Object> query = (NamedList<Object>) field.get("query");
+        if (query != null) {
+          List<AnalysisPhase> phases = buildPhases(query);
+          fieldAnalysis.setQueryPhases(phases);
+        }
+
+        NamedList<Object> index = (NamedList<Object>) field.get("index");
+        for (Map.Entry<String, Object> valueEntry : index) {
+          String fieldValue = valueEntry.getKey();
+          NamedList<Object> valueNL = (NamedList<Object>) valueEntry.getValue();
+          List<AnalysisPhase> phases = buildPhases(valueNL);
+          fieldAnalysis.setIndexPhases(fieldValue, phases);
+        }
+
+        documentAnalysis.addFieldAnalysis(fieldAnalysis);
+      }
+
+      documentAnalysisByKey.put(documentAnalysis.getDocumentKey(), documentAnalysis);
+    }
+  }
+
+  /**
+   * Returns the number of document analyses in this response.
+   *
+   * @return The number of document analyses in this response.
+   */
+  public int getDocumentAnalysesCount() {
+    return documentAnalysisByKey.size();
+  }
+
+  /**
+   * Returns the document analysis for the document associated with the given unique key (id), {@code null} if no such
+   * association exists.
+   *
+   * @param documentKey The document unique key.
+   *
+   * @return The document analysis for the document associated with the given unique key (id).
+   */
+  public DocumentAnalysis getDocumentAnalysis(String documentKey) {
+    return documentAnalysisByKey.get(documentKey);
+  }
+
+  /**
+   * Returns an iterator over the document analyses map.
+   *
+   * @return An iterator over the document analyses map.
+   */
+  public Iterator<Map.Entry<String, DocumentAnalysis>> iterator() {
+    return documentAnalysisByKey.entrySet().iterator();
+  }
+
+  //================================================= Inner Classes ==================================================
+
+  /**
+   * An analysis process breakdown of a document. Holds a map of field analyses by the field name.
+   */
+  public static class DocumentAnalysis implements Iterable<Map.Entry<String, FieldAnalysis>> {
+
+    private final String documentKey;
+    private Map<String, FieldAnalysis> fieldAnalysisByFieldName = new HashMap<String, FieldAnalysis>();
+
+    private DocumentAnalysis(String documentKey) {
+      this.documentKey = documentKey;
+    }
+
+    private void addFieldAnalysis(FieldAnalysis fieldAnalysis) {
+      fieldAnalysisByFieldName.put(fieldAnalysis.getFieldName(), fieldAnalysis);
+    }
+
+    /**
+     * Returns the unique key of the analyzed document.
+     *
+     * @return The unique key of the analyzed document.
+     */
+    public String getDocumentKey() {
+      return documentKey;
+    }
+
+    /**
+     * Returns the number of field analyses for the documents.
+     *
+     * @return The number of field analyses for the documents.
+     */
+    public int getFieldAnalysesCount() {
+      return fieldAnalysisByFieldName.size();
+    }
+
+    public FieldAnalysis getFieldAnalysis(String fieldName) {
+      return fieldAnalysisByFieldName.get(fieldName);
+    }
+
+    /**
+     * Returns an iterator over the field analyses map.
+     *
+     * @return An iterator over the field analyses map.
+     */
+    public Iterator<Map.Entry<String, FieldAnalysis>> iterator() {
+      return fieldAnalysisByFieldName.entrySet().iterator();
+    }
+  }
+
+  /**
+   * An analysis process breakdown for a specific field. Holds a list of query time analysis phases (that is, if a
+   * query analysis was requested in the first place) and a list of index time analysis phases for each field value (a
+   * field can be multi-valued).
+   */
+  public static class FieldAnalysis {
+
+    private final String fieldName;
+    private List<AnalysisPhase> queryPhases;
+    private Map<String, List<AnalysisPhase>> indexPhasesByFieldValue = new HashMap<String, List<AnalysisPhase>>();
+
+    private FieldAnalysis(String fieldName) {
+      this.fieldName = fieldName;
+    }
+
+    public void setQueryPhases(List<AnalysisPhase> queryPhases) {
+      this.queryPhases = queryPhases;
+    }
+
+    public void setIndexPhases(String fieldValue, List<AnalysisPhase> indexPhases) {
+      indexPhasesByFieldValue.put(fieldValue, indexPhases);
+    }
+
+    /**
+     * Returns the field name.
+     *
+     * @return The name of the field.
+     */
+    public String getFieldName() {
+      return fieldName;
+    }
+
+    /**
+     * Returns the number of query time analysis phases or {@code -1) if this field analysis doesn't hold a query
+     * time analysis.
+     *
+     * @return Returns the number of query time analysis phases or {@code -1) if this field analysis doesn't hold a
+     *         query time analysis.
+     */
+    public int getQueryPhasesCount() {
+      return queryPhases == null ? -1 : queryPhases.size();
+    }
+
+    /**
+     * Returns the query time analysis phases for the field or {@code null} if this field doesn't hold a query time
+     * analysis.
+     *
+     * @return Returns the query time analysis phases for the field or {@code null} if this field doesn't hold a
+     *         query time analysis.
+     */
+    public Iterable<AnalysisPhase> getQueryPhases() {
+      return queryPhases;
+    }
+
+    /**
+     * Returns the number of values the field has.
+     *
+     * @return The number of values the field has.
+     */
+    public int getValueCount() {
+      return indexPhasesByFieldValue.entrySet().size();
+    }
+
+    /**
+     * Returns the number of index time analysis phases the given field value has.
+     *
+     * @param fieldValue The field value.
+     *
+     * @return The number of index time analysis phases the given field value has.
+     */
+    public int getIndexPhasesCount(String fieldValue) {
+      return indexPhasesByFieldValue.get(fieldValue).size();
+    }
+
+    /**
+     * Returns the index time analysis phases for the given field value.
+     *
+     * @param fieldValue The field value.
+     *
+     * @return The index time analysis phases for the given field value.
+     */
+    public Iterable<AnalysisPhase> getIndexPhases(String fieldValue) {
+      return indexPhasesByFieldValue.get(fieldValue);
+    }
+
+    /**
+     * Returns the index time analysis phases for all field values.
+     *
+     * @return Returns the index time analysis phases for all field value.
+     */
+    public Iterable<Map.Entry<String, List<AnalysisPhase>>> getIndexPhasesByFieldValue() {
+      return indexPhasesByFieldValue.entrySet();
+    }
+
+  }
+
+}

Propchange: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/DocumentAnalysisResponse.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/solrj/org/apache/solr/client/solrj/response/DocumentAnalysisResponse.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL