You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/05/18 18:24:34 UTC

svn commit: r1124321 [5/5] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/db/bdb-je/ dev-tools/idea/lucene/contrib...

Modified: lucene/dev/branches/docvalues/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java Wed May 18 16:24:27 2011
@@ -18,9 +18,11 @@ package org.apache.solr.handler.clusteri
  */
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -37,6 +39,7 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.handler.clustering.SearchClusteringEngine;
 import org.apache.solr.handler.component.HighlightComponent;
 import org.apache.solr.highlight.SolrHighlighter;
@@ -52,9 +55,17 @@ import org.carrot2.core.ControllerFactor
 import org.carrot2.core.Document;
 import org.carrot2.core.IClusteringAlgorithm;
 import org.carrot2.core.attribute.AttributeNames;
+import org.carrot2.text.linguistic.DefaultLexicalDataFactoryDescriptor;
+import org.carrot2.text.preprocessing.pipeline.BasicPreprocessingPipelineDescriptor;
+import org.carrot2.util.resource.ClassLoaderLocator;
+import org.carrot2.util.resource.IResource;
+import org.carrot2.util.resource.IResourceLocator;
+import org.carrot2.util.resource.ResourceLookup;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 /**
@@ -64,19 +75,33 @@ import com.google.common.collect.Sets;
  *
  * @link http://project.carrot2.org
  */
-@SuppressWarnings("unchecked")
 public class CarrotClusteringEngine extends SearchClusteringEngine {
-  private transient static Logger log = LoggerFactory
+	private transient static Logger log = LoggerFactory
           .getLogger(CarrotClusteringEngine.class);
 
+	/**
+	 * The subdirectory in Solr config dir to read customized Carrot2 resources from.
+	 */
+	private static final String CARROT_RESOURCES_PREFIX = "clustering/carrot2";
+
+  /**
+   * Name of Carrot2 document's field containing Solr document's identifier.
+   */
+  private static final String SOLR_DOCUMENT_ID = "solrId";
+
+  /**
+   * Name of Solr document's field containing the document's identifier. To avoid
+   * repeating the content of documents in clusters on output, each cluster contains
+   * identifiers of documents it contains.
+   */
+  private String idFieldName;
+
   /**
    * Carrot2 controller that manages instances of clustering algorithms
    */
   private Controller controller = ControllerFactory.createPooling();
   private Class<? extends IClusteringAlgorithm> clusteringAlgorithmClass;
 
-  private String idFieldName;
-
   @Override
   @Deprecated
   public Object cluster(Query query, DocList docList, SolrQueryRequest sreq) {
@@ -101,6 +126,10 @@ public class CarrotClusteringEngine exte
       attributes.put(AttributeNames.DOCUMENTS, documents);
       attributes.put(AttributeNames.QUERY, query.toString());
 
+      // Pass the fields on which clustering runs to the
+      // SolrStopwordsCarrot2LexicalDataFactory
+      attributes.put("solrFieldNames", getFieldsForClustering(sreq));
+
       // Pass extra overriding attributes from the request, if any
       extractCarrotAttributes(sreq.getParams(), attributes);
 
@@ -113,22 +142,68 @@ public class CarrotClusteringEngine exte
     }
   }
 
-  @Override
+	@Override
+	@SuppressWarnings({ "unchecked", "rawtypes" })
   public String init(NamedList config, final SolrCore core) {
     String result = super.init(config, core);
-    SolrParams initParams = SolrParams.toSolrParams(config);
+    final SolrParams initParams = SolrParams.toSolrParams(config);
 
     // Initialize Carrot2 controller. Pass initialization attributes, if any.
     HashMap<String, Object> initAttributes = new HashMap<String, Object>();
     extractCarrotAttributes(initParams, initAttributes);
-    
-    // Customize the language model factory. The implementation we provide here
-    // is included in the code base of Solr, so that it's possible to refactor
-    // the Lucene APIs the factory relies on if needed.
-    initAttributes.put("PreprocessingPipeline.languageModelFactory",
-      LuceneLanguageModelFactory.class);
-    this.controller.init(initAttributes);
 
+    // Customize the stemmer and tokenizer factories. The implementations we provide here
+    // are included in the code base of Solr, so that it's possible to refactor
+    // the Lucene APIs the factories rely on if needed.
+    // Additionally, we set a custom lexical resource factory for Carrot2 that
+    // will use both Carrot2 default stop words as well as stop words from
+    // the StopFilter defined on the field.
+		BasicPreprocessingPipelineDescriptor.attributeBuilder(initAttributes)
+				.stemmerFactory(LuceneCarrot2StemmerFactory.class)
+				.tokenizerFactory(LuceneCarrot2TokenizerFactory.class)
+				.lexicalDataFactory(SolrStopwordsCarrot2LexicalDataFactory.class);
+
+		// Pass the schema to SolrStopwordsCarrot2LexicalDataFactory.
+		initAttributes.put("solrIndexSchema", core.getSchema());
+
+    // Customize Carrot2's resource lookup to first look for resources
+    // using Solr's resource loader. If that fails, try loading from the classpath.
+    DefaultLexicalDataFactoryDescriptor.attributeBuilder(initAttributes)
+        .resourceLookup(new ResourceLookup(new IResourceLocator() {
+          @Override
+          public IResource[] getAll(final String resource) {
+            final SolrResourceLoader resourceLoader = core.getResourceLoader();
+            final String carrot2ResourcesDir = resourceLoader.getConfigDir()
+                + initParams.get(CarrotParams.LEXICAL_RESOURCES_DIR, CARROT_RESOURCES_PREFIX);
+            try {
+              log.debug("Looking for " + resource + " in "
+                  + carrot2ResourcesDir);
+              final InputStream resourceStream = resourceLoader
+                  .openResource(carrot2ResourcesDir + "/" + resource);
+
+              log.info(resource + " loaded from " + carrot2ResourcesDir);
+              final IResource foundResource = new IResource() {
+                @Override
+                public InputStream open() throws IOException {
+                  return resourceStream;
+                }
+              };
+              return new IResource[] { foundResource };
+            } catch (RuntimeException e) {
+              // No way to distinguish if the resource was found but failed
+              // to load or wasn't found at all, so we simply fall back
+              // to Carrot2 defaults here by returning an empty locations array.
+              log.debug(resource + " not found in " + carrot2ResourcesDir
+                  + ". Using the default " + resource + " from Carrot JAR.");
+              return new IResource[] {};
+            }
+          }
+        },
+
+        // Using the class loader directly because this time we want to omit the prefix
+        new ClassLoaderLocator(core.getResourceLoader().getClassLoader())));
+
+    this.controller.init(initAttributes);
     this.idFieldName = core.getSchema().getUniqueKeyField().getName();
 
     // Make sure the requested Carrot2 clustering algorithm class is available
@@ -148,17 +223,29 @@ public class CarrotClusteringEngine exte
   protected Set<String> getFieldsToLoad(SolrQueryRequest sreq){
     SolrParams solrParams = sreq.getParams();
 
-    // Names of fields to deliver content for clustering
-    String urlField = solrParams.get(CarrotParams.URL_FIELD_NAME, "url");
+    HashSet<String> fields = Sets.newHashSet(getFieldsForClustering(sreq));
+    fields.add(idFieldName);
+    fields.add(solrParams.get(CarrotParams.URL_FIELD_NAME, "url"));
+		return fields;
+  }
+
+	/**
+	 * Returns the names of fields that will be delivering the actual
+	 * content for clustering. Currently, there are two such fields: document
+	 * title and document content.
+	 */
+	private Set<String> getFieldsForClustering(SolrQueryRequest sreq) {
+    SolrParams solrParams = sreq.getParams();
+
     String titleField = solrParams.get(CarrotParams.TITLE_FIELD_NAME, "title");
     String snippetField = solrParams.get(CarrotParams.SNIPPET_FIELD_NAME, titleField);
     if (StringUtils.isBlank(snippetField)) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, CarrotParams.SNIPPET_FIELD_NAME
               + " must not be blank.");
     }
-    return Sets.newHashSet(urlField, titleField, snippetField, idFieldName);
-  }
-  
+    return Sets.newHashSet(titleField, snippetField);
+	}
+
   /**
    * Prepares Carrot2 documents for clustering.
    */
@@ -180,7 +267,7 @@ public class CarrotClusteringEngine exte
     if (produceSummary == true) {
       highlighter = HighlightComponent.getHighlighter(core);
       if (highlighter != null){
-        Map args = new HashMap();
+        Map<String, Object> args = Maps.newHashMap();
         snippetFieldAry = new String[]{snippetField};
         args.put(HighlightParams.FIELDS, snippetFieldAry);
         args.put(HighlightParams.HIGHLIGHT, "true");
@@ -214,11 +301,12 @@ public class CarrotClusteringEngine exte
       if (produceSummary && docIds != null) {
         docsHolder[0] = docIds.get(sdoc).intValue();
         DocList docAsList = new DocSlice(0, 1, docsHolder, scores, 1, 1.0f);
-        NamedList highlights = highlighter.doHighlighting(docAsList, theQuery, req, snippetFieldAry);
+        NamedList<Object> highlights = highlighter.doHighlighting(docAsList, theQuery, req, snippetFieldAry);
         if (highlights != null && highlights.size() == 1) {//should only be one value given our setup
           //should only be one document with one field
-          NamedList tmp = (NamedList) highlights.getVal(0);
-          String [] highlt = (String[]) tmp.get(snippetField);
+          @SuppressWarnings("unchecked")
+					NamedList<String []> tmp = (NamedList<String[]>) highlights.getVal(0);
+          String [] highlt = tmp.get(snippetField);
           if (highlt != null && highlt.length == 1) {
             snippet = highlt[0];
           }
@@ -226,27 +314,13 @@ public class CarrotClusteringEngine exte
       }
       Document carrotDocument = new Document(getValue(sdoc, titleField),
               snippet, (String)sdoc.getFieldValue(urlField));
-      carrotDocument.setField("solrId", sdoc.getFieldValue(idFieldName));
+      carrotDocument.setField(SOLR_DOCUMENT_ID, sdoc.getFieldValue(idFieldName));
       result.add(carrotDocument);
     }
 
     return result;
   }
 
-  @Deprecated
-  protected String getValue(org.apache.lucene.document.Document doc,
-                            String field) {
-    StringBuilder result = new StringBuilder();
-    String[] vals = doc.getValues(field);
-    for (int i = 0; i < vals.length; i++) {
-      // Join multiple values with a period so that Carrot2 does not pick up
-      // phrases that cross field value boundaries (in most cases it would
-      // create useless phrases).
-      result.append(vals[i]).append(" . ");
-    }
-    return result.toString().trim();
-  }
-
   protected String getValue(SolrDocument sdoc, String field) {
     StringBuilder result = new StringBuilder();
     Collection<Object> vals = sdoc.getFieldValues(field);
@@ -261,9 +335,9 @@ public class CarrotClusteringEngine exte
     return result.toString().trim();
   }
 
-  private List clustersToNamedList(List<Cluster> carrotClusters,
+  private List<NamedList<Object>> clustersToNamedList(List<Cluster> carrotClusters,
                                    SolrParams solrParams) {
-    List result = new ArrayList();
+    List<NamedList<Object>> result = Lists.newArrayList();
     clustersToNamedList(carrotClusters, result, solrParams.getBool(
             CarrotParams.OUTPUT_SUB_CLUSTERS, true), solrParams.getInt(
             CarrotParams.NUM_DESCRIPTIONS, Integer.MAX_VALUE));
@@ -271,25 +345,40 @@ public class CarrotClusteringEngine exte
   }
 
   private void clustersToNamedList(List<Cluster> outputClusters,
-                                   List parent, boolean outputSubClusters, int maxLabels) {
+                                   List<NamedList<Object>> parent, boolean outputSubClusters, int maxLabels) {
     for (Cluster outCluster : outputClusters) {
-      NamedList cluster = new SimpleOrderedMap();
+      NamedList<Object> cluster = new SimpleOrderedMap<Object>();
       parent.add(cluster);
 
+      // Add labels
       List<String> labels = outCluster.getPhrases();
-      if (labels.size() > maxLabels)
+      if (labels.size() > maxLabels) {
         labels = labels.subList(0, maxLabels);
+      }
       cluster.add("labels", labels);
 
+      // Add cluster score
+      final Double score = outCluster.getScore();
+      if (score != null) {
+        cluster.add("score", score);
+      }
+
+      // Add other topics marker
+      if (outCluster.isOtherTopics()) {
+        cluster.add("other-topics", outCluster.isOtherTopics());
+      }
+
+      // Add documents
       List<Document> docs = outputSubClusters ? outCluster.getDocuments() : outCluster.getAllDocuments();
-      List docList = new ArrayList();
+      List<Object> docList = Lists.newArrayList();
       cluster.add("docs", docList);
       for (Document doc : docs) {
-        docList.add(doc.getField("solrId"));
+        docList.add(doc.getField(SOLR_DOCUMENT_ID));
       }
 
-      if (outputSubClusters) {
-        List subclusters = new ArrayList();
+      // Add subclusters
+      if (outputSubClusters && !outCluster.getSubclusters().isEmpty()) {
+        List<NamedList<Object>> subclusters = Lists.newArrayList();
         cluster.add("clusters", subclusters);
         clustersToNamedList(outCluster.getSubclusters(), subclusters,
                 outputSubClusters, maxLabels);

Modified: lucene/dev/branches/docvalues/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotParams.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotParams.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotParams.java Wed May 18 16:24:27 2011
@@ -35,6 +35,8 @@ public interface CarrotParams {
   String OUTPUT_SUB_CLUSTERS = CARROT_PREFIX + "outputSubClusters";
   String SUMMARY_FRAGSIZE = CARROT_PREFIX + "fragzise";
 
+  String LEXICAL_RESOURCES_DIR = CARROT_PREFIX + "lexicalResourcesDir";
+
   public static final Set<String> CARROT_PARAM_NAMES = ImmutableSet.of(
           ALGORITHM, TITLE_FIELD_NAME, URL_FIELD_NAME, SNIPPET_FIELD_NAME,
           PRODUCE_SUMMARY, NUM_DESCRIPTIONS, OUTPUT_SUB_CLUSTERS, SUMMARY_FRAGSIZE);

Modified: lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngineTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngineTest.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngineTest.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngineTest.java Wed May 18 16:24:27 2011
@@ -17,6 +17,11 @@ package org.apache.solr.handler.clusteri
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -37,15 +42,11 @@ import org.apache.solr.util.SolrPluginUt
 import org.carrot2.util.attribute.AttributeUtils;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import com.google.common.collect.ImmutableList;
 
 /**
  *
  */
-@SuppressWarnings("unchecked")
 public class CarrotClusteringEngineTest extends AbstractClusteringTestCase {
   @Test
   public void testCarrotLingo() throws Exception {
@@ -74,7 +75,7 @@ public class CarrotClusteringEngineTest 
 
   @Test
   public void testWithoutSubclusters() throws Exception {
-    checkClusters(checkEngine(getClusteringEngine("mock"), this.numberOfDocs),
+    checkClusters(checkEngine(getClusteringEngine("mock"), AbstractClusteringTestCase.numberOfDocs),
             1, 1, 0);
   }
 
@@ -82,7 +83,7 @@ public class CarrotClusteringEngineTest 
   public void testWithSubclusters() throws Exception {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(CarrotParams.OUTPUT_SUB_CLUSTERS, true);
-    checkClusters(checkEngine(getClusteringEngine("mock"), this.numberOfDocs), 1, 1, 2);
+    checkClusters(checkEngine(getClusteringEngine("mock"), AbstractClusteringTestCase.numberOfDocs), 1, 1, 2);
   }
 
   @Test
@@ -90,19 +91,107 @@ public class CarrotClusteringEngineTest 
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(AttributeUtils.getKey(MockClusteringAlgorithm.class, "labels"), 5);
     params.set(CarrotParams.NUM_DESCRIPTIONS, 3);
-    checkClusters(checkEngine(getClusteringEngine("mock"), this.numberOfDocs,
+    checkClusters(checkEngine(getClusteringEngine("mock"), AbstractClusteringTestCase.numberOfDocs,
             params), 1, 3, 0);
   }
 
   @Test
+  public void testClusterScores() throws Exception {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(AttributeUtils.getKey(MockClusteringAlgorithm.class, "depth"), 1);
+    List<NamedList<Object>> clusters = checkEngine(getClusteringEngine("mock"),
+        AbstractClusteringTestCase.numberOfDocs, params);
+    int i = 1;
+    for (NamedList<Object> cluster : clusters) {
+      final Double score = getScore(cluster);
+      assertNotNull(score);
+      assertEquals(0.25 * i++, score, 0);
+    }
+  }
+
+  @Test
+  public void testOtherTopics() throws Exception {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(AttributeUtils.getKey(MockClusteringAlgorithm.class, "depth"), 1);
+    params.set(AttributeUtils.getKey(MockClusteringAlgorithm.class, "otherTopicsModulo"), 2);
+    List<NamedList<Object>> clusters = checkEngine(getClusteringEngine("mock"),
+        AbstractClusteringTestCase.numberOfDocs, params);
+    int i = 1;
+    for (NamedList<Object> cluster : clusters) {
+      assertEquals(i++ % 2 == 0 ? true : null, isOtherTopics(cluster));
+    }
+  }
+
+  @Test
   public void testCarrotAttributePassing() throws Exception {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(AttributeUtils.getKey(MockClusteringAlgorithm.class, "depth"), 1);
     params.set(AttributeUtils.getKey(MockClusteringAlgorithm.class, "labels"), 3);
-    checkClusters(checkEngine(getClusteringEngine("mock"), this.numberOfDocs,
+    checkClusters(checkEngine(getClusteringEngine("mock"), AbstractClusteringTestCase.numberOfDocs,
             params), 1, 3, 0);
   }
 
+	@Test
+	public void testLexicalResourcesFromSolrConfigDefaultDir() throws Exception {
+		checkLexicalResourcesFromSolrConfig("lexical-resource-check",
+				"online,customsolrstopword,customsolrstoplabel");
+	}
+
+	@Test
+	public void testLexicalResourcesFromSolrConfigCustomDir() throws Exception {
+		checkLexicalResourcesFromSolrConfig("lexical-resource-check-custom-resource-dir",
+				"online,customsolrstopwordcustomdir,customsolrstoplabelcustomdir");
+	}
+
+	private void checkLexicalResourcesFromSolrConfig(String engineName, String wordsToCheck)
+			throws IOException {
+		ModifiableSolrParams params = new ModifiableSolrParams();
+		params.set("merge-resources", false);
+		params.set(AttributeUtils.getKey(
+				LexicalResourcesCheckClusteringAlgorithm.class, "wordsToCheck"),
+				wordsToCheck);
+
+		// "customsolrstopword" is in stopwords.en, "customsolrstoplabel" is in
+		// stoplabels.en, so we're expecting only one cluster with label "online".
+		final List<NamedList<Object>> clusters = checkEngine(
+				getClusteringEngine(engineName), 1, params);
+		assertEquals(getLabels(clusters.get(0)), ImmutableList.of("online"));
+	}
+
+	@Test
+	public void solrStopWordsUsedInCarrot2Clustering() throws Exception {
+		ModifiableSolrParams params = new ModifiableSolrParams();
+		params.set("merge-resources", false);
+		params.set(AttributeUtils.getKey(
+				LexicalResourcesCheckClusteringAlgorithm.class, "wordsToCheck"),
+		"online,solrownstopword");
+
+		// "solrownstopword" is in stopwords.txt, so we're expecting
+		// only one cluster with label "online".
+		final List<NamedList<Object>> clusters = checkEngine(
+				getClusteringEngine("lexical-resource-check"), 1, params);
+		assertEquals(getLabels(clusters.get(0)), ImmutableList.of("online"));
+	}
+
+	@Test
+	public void solrStopWordsNotDefinedOnAFieldForClustering() throws Exception {
+		ModifiableSolrParams params = new ModifiableSolrParams();
+		// Force string fields to be used for clustering. Does not make sense
+		// in a real word, but does the job in the test.
+		params.set(CarrotParams.TITLE_FIELD_NAME, "url");
+		params.set(CarrotParams.SNIPPET_FIELD_NAME, "url");
+		params.set("merge-resources", false);
+		params.set(AttributeUtils.getKey(
+				LexicalResourcesCheckClusteringAlgorithm.class, "wordsToCheck"),
+		"online,solrownstopword");
+
+		final List<NamedList<Object>> clusters = checkEngine(
+				getClusteringEngine("lexical-resource-check"), 2, params);
+		assertEquals(ImmutableList.of("online"), getLabels(clusters.get(0)));
+		assertEquals(ImmutableList.of("solrownstopword"),
+				getLabels(clusters.get(1)));
+	}
+
   private CarrotClusteringEngine getClusteringEngine(String engineName) {
     ClusteringComponent comp = (ClusteringComponent) h.getCore()
             .getSearchComponent("clustering");
@@ -114,18 +203,18 @@ public class CarrotClusteringEngineTest 
     return engine;
   }
 
-  private List checkEngine(CarrotClusteringEngine engine,
+  private List<NamedList<Object>> checkEngine(CarrotClusteringEngine engine,
                             int expectedNumClusters) throws IOException {
     return checkEngine(engine, numberOfDocs, expectedNumClusters, new MatchAllDocsQuery(), new ModifiableSolrParams());
   }
 
-  private List checkEngine(CarrotClusteringEngine engine,
+  private List<NamedList<Object>> checkEngine(CarrotClusteringEngine engine,
                             int expectedNumClusters, SolrParams clusteringParams) throws IOException {
     return checkEngine(engine, numberOfDocs, expectedNumClusters, new MatchAllDocsQuery(), clusteringParams);
   }
 
 
-  private List checkEngine(CarrotClusteringEngine engine, int expectedNumDocs,
+  private List<NamedList<Object>> checkEngine(CarrotClusteringEngine engine, int expectedNumDocs,
                            int expectedNumClusters, Query query, SolrParams clusteringParams) throws IOException {
     // Get all documents to cluster
     RefCounted<SolrIndexSearcher> ref = h.getCore().getSearcher();
@@ -145,7 +234,9 @@ public class CarrotClusteringEngineTest 
       LocalSolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), solrParams);
       Map<SolrDocument,Integer> docIds = new HashMap<SolrDocument, Integer>(docList.size());
       SolrDocumentList solrDocList = SolrPluginUtils.docListToSolrDocumentList( docList, searcher, engine.getFieldsToLoad(req), docIds );
-      List results = (List)engine.cluster(query, solrDocList, docIds, req);
+
+      @SuppressWarnings("unchecked")
+			List<NamedList<Object>> results = (List<NamedList<Object>>) engine.cluster(query, solrDocList, docIds, req);
       req.close();
       assertEquals("number of clusters: " + results, expectedNumClusters, results.size());
       checkClusters(results, false);
@@ -155,51 +246,74 @@ public class CarrotClusteringEngineTest 
     }
   }
 
-  private void checkClusters(List results, int expectedDocCount,
+  private void checkClusters(List<NamedList<Object>> results, int expectedDocCount,
                              int expectedLabelCount, int expectedSubclusterCount) {
     for (int i = 0; i < results.size(); i++) {
-      NamedList cluster = (NamedList) results.get(i);
+      NamedList<Object> cluster = results.get(i);
       checkCluster(cluster, expectedDocCount, expectedLabelCount,
               expectedSubclusterCount);
     }
   }
 
-  private void checkClusters(List results, boolean hasSubclusters) {
+  private void checkClusters(List<NamedList<Object>> results, boolean hasSubclusters) {
     for (int i = 0; i < results.size(); i++) {
-      checkCluster((NamedList) results.get(i), hasSubclusters);
+      checkCluster(results.get(i), hasSubclusters);
     }
   }
 
-  private void checkCluster(NamedList cluster, boolean hasSubclusters) {
-    List docs = (List) cluster.get("docs");
+  private void checkCluster(NamedList<Object> cluster, boolean hasSubclusters) {
+    List<Object> docs = getDocs(cluster);
     assertNotNull("docs is null and it shouldn't be", docs);
     for (int j = 0; j < docs.size(); j++) {
       String id = (String) docs.get(j);
       assertNotNull("id is null and it shouldn't be", id);
     }
 
-    List labels = (List) cluster.get("labels");
+    List<String> labels = getLabels(cluster);
     assertNotNull("labels is null but it shouldn't be", labels);
 
     if (hasSubclusters) {
-      List subclusters = (List) cluster.get("clusters");
+      List<NamedList<Object>> subclusters = getSubclusters(cluster);
       assertNotNull("subclusters is null but it shouldn't be", subclusters);
     }
   }
 
-  private void checkCluster(NamedList cluster, int expectedDocCount,
+  private void checkCluster(NamedList<Object> cluster, int expectedDocCount,
                             int expectedLabelCount, int expectedSubclusterCount) {
     checkCluster(cluster, expectedSubclusterCount > 0);
     assertEquals("number of docs in cluster", expectedDocCount,
-            ((List) cluster.get("docs")).size());
+            getDocs(cluster).size());
     assertEquals("number of labels in cluster", expectedLabelCount,
-            ((List) cluster.get("labels")).size());
+            getLabels(cluster).size());
 
     if (expectedSubclusterCount > 0) {
-      List subclusters = (List) cluster.get("clusters");
+      List<NamedList<Object>> subclusters = getSubclusters(cluster);
       assertEquals("numClusters", expectedSubclusterCount, subclusters.size());
       assertEquals("number of subclusters in cluster",
               expectedSubclusterCount, subclusters.size());
     }
   }
+
+	@SuppressWarnings("unchecked")
+	private List<NamedList<Object>> getSubclusters(NamedList<Object> cluster) {
+		return (List<NamedList<Object>>) cluster.get("clusters");
+	}
+
+	@SuppressWarnings("unchecked")
+	private List<String> getLabels(NamedList<Object> cluster) {
+		return (List<String>) cluster.get("labels");
+	}
+
+	private Double getScore(NamedList<Object> cluster) {
+	  return (Double) cluster.get("score");
+	}
+
+	private Boolean isOtherTopics(NamedList<Object> cluster) {
+	  return (Boolean)cluster.get("other-topics");
+	}
+
+	@SuppressWarnings("unchecked")
+	private List<Object> getDocs(NamedList<Object> cluster) {
+		return (List<Object>) cluster.get("docs");
+	}
 }

Modified: lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/carrot2/MockClusteringAlgorithm.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/carrot2/MockClusteringAlgorithm.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/carrot2/MockClusteringAlgorithm.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/carrot2/MockClusteringAlgorithm.java Wed May 18 16:24:27 2011
@@ -49,6 +49,11 @@ public class MockClusteringAlgorithm ext
   @IntRange(min = 1, max = 5)
   private int labels = 1;
 
+  @Input
+  @Processing
+  @Attribute
+  private int otherTopicsModulo = 0;
+
   @Override
   public void process() throws ProcessingException {
     clusters = Lists.newArrayList();
@@ -59,21 +64,26 @@ public class MockClusteringAlgorithm ext
     int documentIndex = 1;
     for (Document document : documents) {
       StringBuilder label = new StringBuilder("Cluster " + documentIndex);
-      Cluster cluster = createCluster(label.toString(), document);
+      Cluster cluster = createCluster(label.toString(), documentIndex, document);
       clusters.add(cluster);
       for (int i = 1; i <= depth; i++) {
         label.append(".");
         label.append(i);
-        Cluster newCluster = createCluster(label.toString(), document);
-        cluster.addSubclusters(createCluster(label.toString(), document), newCluster);
+        Cluster newCluster = createCluster(label.toString(), documentIndex, document);
+        cluster.addSubclusters(createCluster(label.toString(), documentIndex, document), newCluster);
         cluster = newCluster;
       }
       documentIndex++;
     }
   }
 
-  private Cluster createCluster(String labelBase, Document... documents) {
+  private Cluster createCluster(String labelBase, int documentIndex, Document... documents) {
     Cluster cluster = new Cluster();
+    cluster.setScore(documentIndex * 0.25);
+    if (otherTopicsModulo != 0 && documentIndex % otherTopicsModulo == 0)
+    {
+      cluster.setOtherTopics(true);
+    }
     for (int i = 0; i < labels; i++) {
       cluster.addPhrases(labelBase + "#" + (i + 1));
     }

Modified: lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/resources/solr-clustering/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/resources/solr-clustering/conf/solrconfig.xml?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/resources/solr-clustering/conf/solrconfig.xml (original)
+++ lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/resources/solr-clustering/conf/solrconfig.xml Wed May 18 16:24:27 2011
@@ -396,6 +396,15 @@
       <str name="name">mock</str>
       <str name="carrot.algorithm">org.apache.solr.handler.clustering.carrot2.MockClusteringAlgorithm</str>
     </lst>
+    <lst name="engine">
+      <str name="name">lexical-resource-check</str>
+      <str name="carrot.algorithm">org.apache.solr.handler.clustering.carrot2.LexicalResourcesCheckClusteringAlgorithm</str>
+    </lst>
+    <lst name="engine">
+      <str name="name">lexical-resource-check-custom-resource-dir</str>
+      <str name="carrot.algorithm">org.apache.solr.handler.clustering.carrot2.LexicalResourcesCheckClusteringAlgorithm</str>
+      <str name="carrot.lexicalResourcesDir">clustering/custom</str>
+    </lst>
   </searchComponent>
 
   <searchComponent class="org.apache.solr.handler.clustering.ClusteringComponent" name="doc-clustering">

Modified: lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/resources/solr-clustering/conf/stopwords.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/resources/solr-clustering/conf/stopwords.txt?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/resources/solr-clustering/conf/stopwords.txt (original)
+++ lucene/dev/branches/docvalues/solr/contrib/clustering/src/test/resources/solr-clustering/conf/stopwords.txt Wed May 18 16:24:27 2011
@@ -55,4 +55,5 @@ to
 was
 will
 with
+solrownstopword
 

Modified: lucene/dev/branches/docvalues/solr/contrib/extraction/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/extraction/CHANGES.txt?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/extraction/CHANGES.txt (original)
+++ lucene/dev/branches/docvalues/solr/contrib/extraction/CHANGES.txt Wed May 18 16:24:27 2011
@@ -22,7 +22,7 @@ to your Solr Home lib directory.  See ht
 
 Current Version: Tika 0.8 (released 11/07/2010)
 
-$Id:$
+$Id$
 
 ================== Release 4.0-dev ==================
 
@@ -30,7 +30,8 @@ $Id:$
 
 ================== Release 3.2-dev ==================
 
-(No Changes)
+* SOLR-2480: Add ignoreTikaException flag so that users can ignore TikaException but index
+  meta data. (Shinichiro Abe, koji)
 
 ================== Release 3.1-dev ==================
 

Modified: lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/ExtractingDocumentLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/ExtractingDocumentLoader.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/ExtractingDocumentLoader.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/ExtractingDocumentLoader.java Wed May 18 16:24:27 2011
@@ -16,20 +16,27 @@
  */
 package org.apache.solr.handler.extraction;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.util.Locale;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.handler.ContentStreamLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
-import org.apache.solr.handler.ContentStreamLoader;
 import org.apache.tika.config.TikaConfig;
+import org.apache.tika.exception.TikaException;
 import org.apache.tika.metadata.Metadata;
+import org.apache.tika.mime.MediaType;
 import org.apache.tika.parser.AutoDetectParser;
 import org.apache.tika.parser.ParseContext;
 import org.apache.tika.parser.Parser;
@@ -37,26 +44,24 @@ import org.apache.tika.sax.XHTMLContentH
 import org.apache.tika.sax.xpath.Matcher;
 import org.apache.tika.sax.xpath.MatchingContentHandler;
 import org.apache.tika.sax.xpath.XPathParser;
-import org.apache.tika.exception.TikaException;
-import org.apache.tika.mime.MediaType;
-import org.apache.xml.serialize.OutputFormat;
 import org.apache.xml.serialize.BaseMarkupSerializer;
-import org.apache.xml.serialize.XMLSerializer;
+import org.apache.xml.serialize.OutputFormat;
 import org.apache.xml.serialize.TextSerializer;
+import org.apache.xml.serialize.XMLSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.StringWriter;
-import java.util.Locale;
-
 
 /**
  * The class responsible for loading extracted content into Solr.
  *
  **/
 public class ExtractingDocumentLoader extends ContentStreamLoader {
+
+  private static final Logger log = LoggerFactory.getLogger(ExtractingDocumentLoader.class);
+
   /**
    * Extract Only supported format
    */
@@ -74,6 +79,7 @@ public class ExtractingDocumentLoader ex
   final IndexSchema schema;
   final SolrParams params;
   final UpdateRequestProcessor processor;
+  final boolean ignoreTikaException;
   protected AutoDetectParser autoDetectParser;
 
   private final AddUpdateCommand templateAdd;
@@ -95,6 +101,8 @@ public class ExtractingDocumentLoader ex
     //this is lightweight
     autoDetectParser = new AutoDetectParser(config);
     this.factory = factory;
+    
+    ignoreTikaException = params.getBool(ExtractingParams.IGNORE_TIKA_EXCEPTION, false);
   }
 
 
@@ -180,9 +188,17 @@ public class ExtractingDocumentLoader ex
           parsingHandler = new MatchingContentHandler(handler, matcher);
         } //else leave it as is
 
-        //potentially use a wrapper handler for parsing, but we still need the SolrContentHandler for getting the document.
-        ParseContext context = new ParseContext();//TODO: should we design a way to pass in parse context?
-        parser.parse(inputStream, parsingHandler, metadata, context);
+        try{
+          //potentially use a wrapper handler for parsing, but we still need the SolrContentHandler for getting the document.
+          ParseContext context = new ParseContext();//TODO: should we design a way to pass in parse context?
+          parser.parse(inputStream, parsingHandler, metadata, context);
+        } catch (TikaException e) {
+          if(ignoreTikaException)
+            log.warn(new StringBuilder("skip extracting text due to ").append(e.getLocalizedMessage())
+                .append(". metadata=").append(metadata.toString()).toString());
+          else
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+        }
         if (extractOnly == false) {
           addDoc(handler);
         } else {
@@ -202,8 +218,6 @@ public class ExtractingDocumentLoader ex
         }
       } catch (SAXException e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-      } catch (TikaException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
       } finally {
         IOUtils.closeQuietly(inputStream);
       }

Modified: lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/ExtractingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/ExtractingParams.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/ExtractingParams.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/ExtractingParams.java Wed May 18 16:24:27 2011
@@ -28,6 +28,11 @@ public interface ExtractingParams {
    */
   public static final String LOWERNAMES = "lowernames";
 
+  /**
+   * if true, ignore TikaException (give up to extract text but index meta data)
+   */
+  public static final String IGNORE_TIKA_EXCEPTION = "ignoreTikaException";
+
 
   /**
    * The param prefix for mapping Tika metadata to Solr fields.

Modified: lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/SolrContentHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/SolrContentHandler.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/SolrContentHandler.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/extraction/src/main/java/org/apache/solr/handler/extraction/SolrContentHandler.java Wed May 18 16:24:27 2011
@@ -47,23 +47,23 @@ import java.util.*;
  */
 public class SolrContentHandler extends DefaultHandler implements ExtractingParams {
   private transient static Logger log = LoggerFactory.getLogger(SolrContentHandler.class);
-  private SolrInputDocument document;
+  protected SolrInputDocument document;
 
-  private Collection<String> dateFormats = DateUtil.DEFAULT_DATE_FORMATS;
+  protected Collection<String> dateFormats = DateUtil.DEFAULT_DATE_FORMATS;
 
-  private Metadata metadata;
-  private SolrParams params;
-  private StringBuilder catchAllBuilder = new StringBuilder(2048);
-  private IndexSchema schema;
-  private Map<String, StringBuilder> fieldBuilders = Collections.emptyMap();
+  protected Metadata metadata;
+  protected SolrParams params;
+  protected StringBuilder catchAllBuilder = new StringBuilder(2048);
+  protected IndexSchema schema;
+  protected Map<String, StringBuilder> fieldBuilders = Collections.emptyMap();
   private LinkedList<StringBuilder> bldrStack = new LinkedList<StringBuilder>();
 
-  private boolean captureAttribs;
-  private boolean lowerNames;
-  private String contentFieldName = "content";
+  protected boolean captureAttribs;
+  protected boolean lowerNames;
+  protected String contentFieldName = "content";
 
-  private String unknownFieldPrefix = "";
-  private String defaultField = "";
+  protected String unknownFieldPrefix = "";
+  protected String defaultField = "";
 
   public SolrContentHandler(Metadata metadata, SolrParams params, IndexSchema schema) {
     this(metadata, params, schema, DateUtil.DEFAULT_DATE_FORMATS);
@@ -99,46 +99,82 @@ public class SolrContentHandler extends 
    * The base implementation adds the metadata as fields, allowing for potential remapping.
    *
    * @return The {@link org.apache.solr.common.SolrInputDocument}.
+   *
+   * @see #addMetadata()
+   * @see #addCapturedContent()
+   * @see #addContent()
+   * @see #addLiterals()
    */
   public SolrInputDocument newDocument() {
     float boost = 1.0f;
     //handle the metadata extracted from the document
-    for (String name : metadata.names()) {
-      String[] vals = metadata.getValues(name);
-      addField(name, null, vals);
-    }
+    addMetadata();
 
     //handle the literals from the params
-    Iterator<String> paramNames = params.getParameterNamesIterator();
-    while (paramNames.hasNext()) {
-      String pname = paramNames.next();
-      if (!pname.startsWith(LITERALS_PREFIX)) continue;
-
-      String name = pname.substring(LITERALS_PREFIX.length());
-      addField(name, null, params.getParams(pname));
-    }
+    addLiterals();
 
 
     //add in the content
-    addField(contentFieldName, catchAllBuilder.toString(), null);
+    addContent();
 
     //add in the captured content
+    addCapturedContent();
+
+    if (log.isDebugEnabled()) {
+      log.debug("Doc: {}", document);
+    }
+    return document;
+  }
+
+  /**
+   * Add the per field captured content to the Solr Document.  Default implementation uses the
+   * {@link #fieldBuilders} info
+   */
+  protected void addCapturedContent() {
     for (Map.Entry<String, StringBuilder> entry : fieldBuilders.entrySet()) {
       if (entry.getValue().length() > 0) {
         addField(entry.getKey(), entry.getValue().toString(), null);
       }
     }
-    if (log.isDebugEnabled()) {
-      log.debug("Doc: " + document);
+  }
+
+  /**
+   * Add in the catch all content to the field.  Default impl. uses the {@link #contentFieldName}
+   * and the {@link #catchAllBuilder}
+   */
+  protected void addContent() {
+    addField(contentFieldName, catchAllBuilder.toString(), null);
+  }
+
+  /**
+   * Add in the literals to the document using the {@link #params} and the {@link #LITERALS_PREFIX}.
+   */
+  protected void addLiterals() {
+    Iterator<String> paramNames = params.getParameterNamesIterator();
+    while (paramNames.hasNext()) {
+      String pname = paramNames.next();
+      if (!pname.startsWith(LITERALS_PREFIX)) continue;
+
+      String name = pname.substring(LITERALS_PREFIX.length());
+      addField(name, null, params.getParams(pname));
+    }
+  }
+
+  /**
+   * Add in any metadata using {@link #metadata} as the source.
+   */
+  protected void addMetadata() {
+    for (String name : metadata.names()) {
+      String[] vals = metadata.getValues(name);
+      addField(name, null, vals);
     }
-    return document;
   }
 
   // Naming rules:
   // 1) optionally map names to nicenames (lowercase+underscores)
   // 2) execute "map" commands
   // 3) if resulting field is unknown, map it to a common prefix
-  private void addField(String fname, String fval, String[] vals) {
+  protected void addField(String fname, String fval, String[] vals) {
     if (lowerNames) {
       StringBuilder sb = new StringBuilder();
       for (int i=0; i<fname.length(); i++) {

Modified: lucene/dev/branches/docvalues/solr/contrib/uima/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/uima/CHANGES.txt?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/uima/CHANGES.txt (original)
+++ lucene/dev/branches/docvalues/solr/contrib/uima/CHANGES.txt Wed May 18 16:24:27 2011
@@ -33,6 +33,9 @@ New Features
 
 * SOLR-2503: extend mapping function to map feature value to dynamicField. (koji)
 
+* SOLR-2512: add ignoreErrors flag so that users can ignore exceptions in AE.
+  (Tommaso Teofili, koji)
+
 Test Cases:
 ----------------------
 

Modified: lucene/dev/branches/docvalues/solr/contrib/uima/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/uima/README.txt?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/uima/README.txt (original)
+++ lucene/dev/branches/docvalues/solr/contrib/uima/README.txt Wed May 18 16:24:27 2011
@@ -30,6 +30,13 @@ To start using Solr UIMA Metadata Extrac
           <str name="oc_licenseID">VALID_OPENCALAIS_KEY</str>
         </lst>
         <str name="analysisEngine">/org/apache/uima/desc/OverridingParamsExtServicesAE.xml</str>
+        <!-- Set to true if you want to continue indexing even if text processing fails.
+             Default is false. That is, Solr throws RuntimeException and
+             never indexed documents entirely in your session. -->
+        <bool name="ignoreErrors">true</bool>
+        <!-- This is optional. It is used for logging when text processing fails.
+             Usually, set uniqueKey field name -->
+        <str name="logField">id</str>
         <lst name="analyzeFields">
           <bool name="merge">false</bool>
           <arr name="fields">

Modified: lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/SolrUIMAConfiguration.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/SolrUIMAConfiguration.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/SolrUIMAConfiguration.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/SolrUIMAConfiguration.java Wed May 18 16:24:27 2011
@@ -36,14 +36,20 @@ public class SolrUIMAConfiguration {
 
   private Map<String, Object> runtimeParameters;
 
+  private boolean ignoreErrors;
+  
+  private String logField;
+
   public SolrUIMAConfiguration(String aePath, String[] fieldsToAnalyze, boolean fieldsMerging,
           Map<String, Map<String, MapField>> typesFeaturesFieldsMapping,
-          Map<String, Object> runtimeParameters) {
+          Map<String, Object> runtimeParameters, boolean ignoreErrors, String logField) {
     this.aePath = aePath;
     this.fieldsToAnalyze = fieldsToAnalyze;
     this.fieldsMerging = fieldsMerging;
     this.runtimeParameters = runtimeParameters;
     this.typesFeaturesFieldsMapping = typesFeaturesFieldsMapping;
+    this.ignoreErrors = ignoreErrors;
+    this.logField = logField;
   }
 
   public String[] getFieldsToAnalyze() {
@@ -65,6 +71,14 @@ public class SolrUIMAConfiguration {
   public Map<String, Object> getRuntimeParameters() {
     return runtimeParameters;
   }
+
+  public boolean isIgnoreErrors() {
+    return ignoreErrors;
+  }
+  
+  public String getLogField(){
+    return logField;
+  }
   
   static final class MapField {
     

Modified: lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/SolrUIMAConfigurationReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/SolrUIMAConfigurationReader.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/SolrUIMAConfigurationReader.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/SolrUIMAConfigurationReader.java Wed May 18 16:24:27 2011
@@ -40,7 +40,8 @@ public class SolrUIMAConfigurationReader
 
   public SolrUIMAConfiguration readSolrUIMAConfiguration() {
     return new SolrUIMAConfiguration(readAEPath(), readFieldsToAnalyze(), readFieldsMerging(),
-            readTypesFeaturesFieldsMapping(), readAEOverridingParameters());
+            readTypesFeaturesFieldsMapping(), readAEOverridingParameters(), readIgnoreErrors(),
+            readLogField());
   }
 
   private String readAEPath() {
@@ -105,4 +106,12 @@ public class SolrUIMAConfigurationReader
     return runtimeParameters;
   }
 
+  private boolean readIgnoreErrors() {
+    Object ignoreErrors = args.get("ignoreErrors");
+    return ignoreErrors == null ? false : (Boolean)ignoreErrors;
+  }
+
+  private String readLogField() {
+    return (String)args.get("logField");
+  }
 }

Modified: lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessor.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessor.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/uima/src/main/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessor.java Wed May 18 16:24:27 2011
@@ -20,7 +20,9 @@ package org.apache.solr.uima.processor;
 import java.io.IOException;
 import java.util.Map;
 
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.uima.processor.SolrUIMAConfiguration.MapField;
 import org.apache.solr.uima.processor.ae.AEProvider;
@@ -58,12 +60,15 @@ public class UIMAUpdateRequestProcessor 
 
   @Override
   public void processAdd(AddUpdateCommand cmd) throws IOException {
+    String text = null;
     try {
       /* get Solr document */
       SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 
       /* get the fields to analyze */
-      for (String text : getTextsToAnalyze(solrInputDocument)) {
+      String[] texts = getTextsToAnalyze(solrInputDocument);
+      for (int i = 0; i < texts.length; i++) {
+        text = texts[i];
         if (text != null && !"".equals(text)) {
           /* process the text value */
           JCas jcas = processText(text);
@@ -79,7 +84,21 @@ public class UIMAUpdateRequestProcessor 
         }
       }
     } catch (UIMAException e) {
-      throw new RuntimeException(e);
+      String logField = solrUIMAConfiguration.getLogField();
+      String optionalFieldInfo = logField == null ? "." :
+        new StringBuilder(". ").append(logField).append("=")
+        .append((String)cmd.getSolrInputDocument().getField(logField).getValue())
+        .append(", ").toString();
+      if (solrUIMAConfiguration.isIgnoreErrors())
+        log.warn(new StringBuilder("skip the text processing due to ")
+          .append(e.getLocalizedMessage()).append(optionalFieldInfo)
+          .append(" text=\"").append(text.substring(0, 100)).append("...\"").toString());
+      else{
+        throw new SolrException(ErrorCode.SERVER_ERROR,
+            new StringBuilder("processing error: ")
+              .append(e.getLocalizedMessage()).append(optionalFieldInfo)
+              .append(" text=\"").append(text.substring(0, 100)).append("...\"").toString(), e);
+      }
     }
     super.processAdd(cmd);
   }

Modified: lucene/dev/branches/docvalues/solr/contrib/uima/src/test/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/uima/src/test/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorTest.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/uima/src/test/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorTest.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/uima/src/test/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorTest.java Wed May 18 16:24:27 2011
@@ -93,7 +93,7 @@ public class UIMAUpdateRequestProcessorT
   @Test
   public void testProcessing() throws Exception {
 
-    addDoc(adoc(
+    addDoc("uima", adoc(
             "id",
             "2312312321312",
             "text",
@@ -111,13 +111,13 @@ public class UIMAUpdateRequestProcessorT
   @Test
   public void testTwoUpdates() throws Exception {
 
-    addDoc(adoc("id", "1", "text", "The Apache Software Foundation is happy to announce "
+    addDoc("uima", adoc("id", "1", "text", "The Apache Software Foundation is happy to announce "
             + "BarCampApache Sydney, Australia, the first ASF-backed event in the Southern "
             + "Hemisphere!"));
     assertU(commit());
     assertQ(req("sentence:*"), "//*[@numFound='1']");
 
-    addDoc(adoc("id", "2", "text", "Taking place 11th December 2010 at the University "
+    addDoc("uima", adoc("id", "2", "text", "Taking place 11th December 2010 at the University "
             + "of Sydney's Darlington Centre, the BarCampApache \"unconference\" will be"
             + " attendee-driven, facilitated by members of the Apache community and will "
             + "focus on the Apache..."));
@@ -128,9 +128,41 @@ public class UIMAUpdateRequestProcessorT
     assertQ(req("ORGANIZATION_sm:Apache"), "//*[@numFound='2']");
   }
 
-  private void addDoc(String doc) throws Exception {
+  @Test
+  public void testErrorHandling() throws Exception {
+
+    try{
+      addDoc("uima-not-ignoreErrors", adoc(
+            "id",
+            "2312312321312",
+            "text",
+            "SpellCheckComponent got improvement related to recent Lucene changes. \n  "
+                    + "Add support for specifying Spelling SuggestWord Comparator to Lucene spell "
+                    + "checkers for SpellCheckComponent. Issue SOLR-2053 is already fixed, patch is"
+                    + " attached if you need it, but it is also committed to trunk and 3_x branch."
+                    + " Last Lucene European Conference has been held in Prague."));
+      fail("exception shouldn't be ignored");
+    }
+    catch(RuntimeException expected){}
+    assertU(commit());
+    assertQ(req("*:*"), "//*[@numFound='0']");
+
+    addDoc("uima-ignoreErrors", adoc(
+            "id",
+            "2312312321312",
+            "text",
+            "SpellCheckComponent got improvement related to recent Lucene changes. \n  "
+                    + "Add support for specifying Spelling SuggestWord Comparator to Lucene spell "
+                    + "checkers for SpellCheckComponent. Issue SOLR-2053 is already fixed, patch is"
+                    + " attached if you need it, but it is also committed to trunk and 3_x branch."
+                    + " Last Lucene European Conference has been held in Prague."));
+    assertU(commit());
+    assertQ(req("*:*"), "//*[@numFound='1']");
+  }
+
+  private void addDoc(String chain, String doc) throws Exception {
     Map<String, String[]> params = new HashMap<String, String[]>();
-    params.put(UpdateParams.UPDATE_CHAIN, new String[] { "uima" });
+    params.put(UpdateParams.UPDATE_CHAIN, new String[] { chain });
     MultiMapSolrParams mmparams = new MultiMapSolrParams(params);
     SolrQueryRequestBase req = new SolrQueryRequestBase(h.getCore(), (SolrParams) mmparams) {
     };

Modified: lucene/dev/branches/docvalues/solr/contrib/uima/src/test/resources/solr-uima/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/uima/src/test/resources/solr-uima/conf/solrconfig.xml?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/uima/src/test/resources/solr-uima/conf/solrconfig.xml (original)
+++ lucene/dev/branches/docvalues/solr/contrib/uima/src/test/resources/solr-uima/conf/solrconfig.xml Wed May 18 16:24:27 2011
@@ -1003,7 +1003,6 @@
         </lst>
       </lst>
     </processor>
-    <processor class="solr.LogUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory" />
   </updateRequestProcessorChain>
 
@@ -1037,6 +1036,48 @@
     </processor>
   </updateRequestProcessorChain>
 
+  <updateRequestProcessorChain name="uima-not-ignoreErrors">
+    <processor class="org.apache.solr.uima.processor.UIMAUpdateRequestProcessorFactory">
+      <lst name="uimaConfig">
+        <lst name="runtimeParameters">
+          <int name="ngramsize">3</int>
+        </lst>
+        <str name="analysisEngine">/TestExceptionAE.xml</str>
+        <bool name="ignoreErrors">false</bool>
+        <lst name="analyzeFields">
+          <bool name="merge">false</bool>
+          <arr name="fields">
+            <str>text</str>
+          </arr>
+        </lst>
+        <lst name="fieldMappings"/>
+      </lst>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="uima-ignoreErrors">
+    <processor class="org.apache.solr.uima.processor.UIMAUpdateRequestProcessorFactory">
+      <lst name="uimaConfig">
+        <lst name="runtimeParameters">
+          <int name="ngramsize">3</int>
+        </lst>
+        <str name="analysisEngine">/TestExceptionAE.xml</str>
+        <bool name="ignoreErrors">true</bool>
+        <!-- This is optional. It is used for logging when text processing fails. Usually, set uniqueKey field name -->
+        <str name="logField">id</str>
+        <lst name="analyzeFields">
+          <bool name="merge">false</bool>
+          <arr name="fields">
+            <str>text</str>
+          </arr>
+        </lst>
+        <lst name="fieldMappings"/>
+      </lst>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
   <!--
     queryResponseWriter plugins... query responses will be written using
     the writer specified by the 'wt' request parameter matching the name

Modified: lucene/dev/branches/docvalues/solr/example/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/example/solr/conf/solrconfig.xml?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/example/solr/conf/solrconfig.xml (original)
+++ lucene/dev/branches/docvalues/solr/example/solr/conf/solrconfig.xml Wed May 18 16:24:27 2011
@@ -1198,17 +1198,20 @@
     <lst name="engine">
       <!-- The name, only one can be named "default" -->
       <str name="name">default</str>
-      <!-- Class name of Carrot2 clustering algorithm. 
-           
+
+      <!-- Class name of Carrot2 clustering algorithm.
+
            Currently available algorithms are:
            
            * org.carrot2.clustering.lingo.LingoClusteringAlgorithm
            * org.carrot2.clustering.stc.STCClusteringAlgorithm
+           * org.carrot2.clustering.kmeans.BisectingKMeansClusteringAlgorithm
            
            See http://project.carrot2.org/algorithms.html for the
            algorithm's characteristics.
         -->
       <str name="carrot.algorithm">org.carrot2.clustering.lingo.LingoClusteringAlgorithm</str>
+
       <!-- Overriding values for Carrot2 default algorithm attributes.
 
            For a description of all available attributes, see:
@@ -1219,9 +1222,22 @@
            name and attribute value as parameter value.
         -->
       <str name="LingoClusteringAlgorithm.desiredClusterCountBase">20</str>
-      
+
+      <!-- Location of Carrot2 lexical resources.
+
+           A directory from which to load Carrot2-specific stop words
+           and stop labels. Absolute or relative to Solr config directory.
+           If a specific resource (e.g. stopwords.en) is present in the
+           specified dir, it will completely override the corresponding
+           default one that ships with Carrot2.
+
+           For an overview of Carrot2 lexical resources, see:
+           http://download.carrot2.org/head/manual/#chapter.lexical-resources
+        -->
+      <str name="carrot.lexicalResourcesDir">clustering/carrot2</str>
+
       <!-- The language to assume for the documents.
-           
+
            For a list of allowed values, see:
            http://download.carrot2.org/stable/manual/#section.attribute.lingo.MultilingualClustering.defaultLanguage
        -->

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/SolrConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/SolrConfig.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/SolrConfig.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/SolrConfig.java Wed May 18 16:24:27 2011
@@ -57,7 +57,6 @@ import java.util.regex.Pattern;
 import java.util.regex.Matcher;
 import java.io.FileFilter;
 import java.io.IOException;
-import java.io.InputStream;
 
 
 /**
@@ -130,12 +129,12 @@ public class SolrConfig extends Config {
   throws ParserConfigurationException, IOException, SAXException {
     super(loader, name, is, "/config/");
     initLibs();
+    luceneMatchVersion = getLuceneVersion("luceneMatchVersion");
     defaultIndexConfig = new SolrIndexConfig(this, null, null);
     mainIndexConfig = new SolrIndexConfig(this, "mainIndex", defaultIndexConfig);
     reopenReaders = getBool("mainIndex/reopenReaders", true);
     
     booleanQueryMaxClauseCount = getInt("query/maxBooleanClauses", BooleanQuery.getMaxClauseCount());
-    luceneMatchVersion = getLuceneVersion("luceneMatchVersion");
     log.info("Using Lucene MatchVersion: " + luceneMatchVersion);
 
     filtOptEnabled = getBool("query/boolTofilterOptimizer/@enabled", false);

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/JSONResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/JSONResponseWriter.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/JSONResponseWriter.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/JSONResponseWriter.java Wed May 18 16:24:27 2011
@@ -442,7 +442,7 @@ class JSONWriter extends TextResponseWri
 
       for (int i=0; i<val.length(); i++) {
         char ch = val.charAt(i);
-        if ((ch > '#' && ch != '\\' && ch !=  '\u2028') || ch==' ') { // fast path
+        if ((ch > '#' && ch != '\\' && ch < '\u2028') || ch == ' ') { // fast path
           writer.write(ch);
           continue;
         }
@@ -457,7 +457,10 @@ class JSONWriter extends TextResponseWri
           case '\t': writer.write('\\'); writer.write('t'); break;
           case '\b': writer.write('\\'); writer.write('b'); break;
           case '\f': writer.write('\\'); writer.write('f'); break;
-          case '\u2028': unicodeEscape(writer,ch); break;
+          case '\u2028': // fallthrough
+          case '\u2029':
+            unicodeEscape(writer,ch);
+            break;
           // case '/':
           default: {
             if (ch <= 0x1F) {

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/JoinQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/JoinQParserPlugin.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/JoinQParserPlugin.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/JoinQParserPlugin.java Wed May 18 16:24:27 2011
@@ -457,7 +457,7 @@ class JoinQuery extends Query {
         return resultList.get(0);
       }
 
-      int sz = resultList.size();
+      int sz = 0;
 
       for (DocSet set : resultList)
         sz += set.size();

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java Wed May 18 16:24:27 2011
@@ -811,7 +811,7 @@ public class SolrIndexSearcher extends I
       bitsSet += upto;
       result = new BitDocSet(obs, bitsSet);
     } else {
-      result = new SortedIntDocSet(Arrays.copyOf(docs, upto));
+      result = upto==0 ? DocSet.EMPTY : new SortedIntDocSet(Arrays.copyOf(docs, upto));
     }
 
     if (useCache) {

Modified: lucene/dev/branches/docvalues/solr/src/test-framework/org/apache/solr/JSONTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test-framework/org/apache/solr/JSONTestUtil.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test-framework/org/apache/solr/JSONTestUtil.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test-framework/org/apache/solr/JSONTestUtil.java Wed May 18 16:24:27 2011
@@ -25,29 +25,69 @@ import java.util.*;
 
 public class JSONTestUtil {
 
+  /**
+   * Default delta used in numeric equality comparisons for floats and doubles.
+   */
+  public final static double DEFAULT_DELTA = 1e-5;
+
+  /** 
+   * comparison using default delta
+   * @see #DEFAULT_DELTA
+   * @see #match(String,String,double)
+   */
   public static String match(String input, String pathAndExpected) throws Exception {
+    return match(input, pathAndExpected, DEFAULT_DELTA);
+  }
+
+  /** 
+   * comparison using default delta
+   * @see #DEFAULT_DELTA
+   * @see #match(String,String,String,double)
+   */
+  public static String match(String path, String input, String expected) throws Exception {
+    return match(path, input, expected, DEFAULT_DELTA);
+  }
+
+  /**
+   * comparison using default delta
+   * @see #DEFAULT_DELTA
+   * @see #matchObj(String,Object,Object,double)
+   */
+  public static String matchObj(String path, Object input, Object expected) throws Exception {
+    return matchObj(path,input,expected, DEFAULT_DELTA);
+  }
+
+  /**
+   * @param input JSON Structure to parse and test against
+   * @param pathAndExpected JSON path expression + '==' + expected value
+   * @param delta tollerance allowed in comparing float/double values
+   */
+  public static String match(String input, String pathAndExpected, double delta) throws Exception {
     int pos = pathAndExpected.indexOf("==");
     String path = pos>=0 ? pathAndExpected.substring(0,pos) : null;
     String expected = pos>=0 ? pathAndExpected.substring(pos+2) : pathAndExpected;
-    return match(path, input, expected);
+    return match(path, input, expected, delta);
   }
 
-  public static String match(String path, String input, String expected) throws Exception {
+  /**
+   * @param path JSON path expression
+   * @param input JSON Structure to parse and test against
+   * @param expected expected value of path
+   * @param delta tollerance allowed in comparing float/double values
+   */
+  public static String match(String path, String input, String expected, double delta) throws Exception {
     Object inputObj = ObjectBuilder.fromJSON(input);
     Object expectObj = ObjectBuilder.fromJSON(expected);
     return matchObj(path, inputObj, expectObj);
   }
-
-  /**
-  public static Object fromJSON(String json) {
-    try {
-      Object out = ObjectBuilder.fromJSON(json);
-    } finally {
-
-  }
-  **/
   
-  public static String matchObj(String path, Object input, Object expected) throws Exception {
+  /**
+   * @param path JSON path expression
+   * @param input JSON Structure
+   * @param expected expected JSON Object
+   * @param delta tollerance allowed in comparing float/double values
+   */
+  public static String matchObj(String path, Object input, Object expected, double delta) throws Exception {
     CollectionTester tester = new CollectionTester(input);
     boolean reversed = path.startsWith("!");
     String positivePath = reversed ? path.substring(1) : path;
@@ -68,14 +108,19 @@ class CollectionTester {
   public Object val;
   public Object expectedRoot;
   public Object expected;
+  public double delta;
   public List<Object> path;
   public String err;
 
-  public CollectionTester(Object val) {
+  public CollectionTester(Object val, double delta) {
     this.val = val;
     this.valRoot = val;
+    this.delta = delta;
     path = new ArrayList<Object>();
   }
+  public CollectionTester(Object val) {
+    this(val, JSONTestUtil.DEFAULT_DELTA);
+  }
 
   public String getPath() {
     StringBuilder sb = new StringBuilder();
@@ -143,7 +188,7 @@ class CollectionTester {
         double a = ((Number)expected).doubleValue();
         double b = ((Number)val).doubleValue();
         if (Double.compare(a,b) == 0) return true;
-        if (Math.abs(a-b) < 1e-5) return true;
+        if (Math.abs(a-b) < delta) return true;
         return false;
       } else {
         setErr("mismatch: '" + expected + "'!='" + val + "'");

Modified: lucene/dev/branches/docvalues/solr/src/test-framework/org/apache/solr/SolrTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test-framework/org/apache/solr/SolrTestCaseJ4.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test-framework/org/apache/solr/SolrTestCaseJ4.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test-framework/org/apache/solr/SolrTestCaseJ4.java Wed May 18 16:24:27 2011
@@ -36,8 +36,12 @@ import org.apache.solr.handler.JsonUpdat
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.ResultContext;
+import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocList;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.servlet.DirectSolrConnection;
 import org.apache.solr.util.TestHarness;
@@ -374,15 +378,29 @@ public abstract class SolrTestCaseJ4 ext
     }
   }
 
-  /** Validates a query matches some JSON test expressions and closes the query.
-   * The text expression is of the form path:JSON.  To facilitate easy embedding
-   * in Java strings, the JSON can have double quotes replaced with single quotes.
-   *
-   * Please use this with care: this makes it easy to match complete structures, but doing so
-   * can result in fragile tests if you are matching more than what you want to test.
-   *
-   **/
+  /** 
+   * Validates a query matches some JSON test expressions using the default double delta tollerance.
+   * @see JSONTestUtil#DEFAULT_DELTA
+   * @see #assertJQ(SolrQueryRequest,double,String...)
+   */
   public static void assertJQ(SolrQueryRequest req, String... tests) throws Exception {
+    assertJQ(req, JSONTestUtil.DEFAULT_DELTA, tests);
+  }
+  /** 
+   * Validates a query matches some JSON test expressions and closes the 
+   * query. The text expression is of the form path:JSON.  To facilitate 
+   * easy embedding in Java strings, the JSON can have double quotes 
+   * replaced with single quotes.
+   * <p>
+   * Please use this with care: this makes it easy to match complete 
+   * structures, but doing so can result in fragile tests if you are 
+   * matching more than what you want to test.
+   * </p>
+   * @param req Solr request to execute
+   * @param delta tollerance allowed in comparing float/double values
+   * @param tests JSON path expression + '==' + expected value
+   */
+  public static void assertJQ(SolrQueryRequest req, double delta, String... tests) throws Exception {
     SolrParams params =  null;
     try {
       params = req.getParams();
@@ -409,7 +427,7 @@ public abstract class SolrTestCaseJ4 ext
 
         try {
           failed = true;
-          String err = JSONTestUtil.match(response, testJSON);
+          String err = JSONTestUtil.match(response, testJSON, delta);
           failed = false;
           if (err != null) {
             log.error("query failed JSON validation. error=" + err +

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/TestJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/TestJoin.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/TestJoin.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/TestJoin.java Wed May 18 16:24:27 2011
@@ -101,6 +101,14 @@ public class TestJoin extends SolrTestCa
     int indexIter=50 * RANDOM_MULTIPLIER;
     int queryIter=50 * RANDOM_MULTIPLIER;
 
+    // groups of fields that have any chance of matching... used to
+    // increase test effectiveness by avoiding 0 resultsets much of the time.
+    String[][] compat = new String[][] {
+        {"small_s","small2_s","small2_ss","small3_ss"},
+        {"small_i","small2_i","small2_is","small3_is"}
+    };
+
+
     while (--indexIter >= 0) {
       int indexSize = random.nextInt(20 * RANDOM_MULTIPLIER);
 
@@ -121,8 +129,19 @@ public class TestJoin extends SolrTestCa
       Map<String, Map<Comparable, Set<Comparable>>> pivots = new HashMap<String, Map<Comparable, Set<Comparable>>>();
 
       for (int qiter=0; qiter<queryIter; qiter++) {
-        String fromField = types.get(random.nextInt(types.size())).fname;
-        String toField = types.get(random.nextInt(types.size())).fname;
+        String fromField;
+        String toField;
+        if (random.nextInt(100) < 5) {
+          // pick random fields 5% of the time
+          fromField = types.get(random.nextInt(types.size())).fname;
+          // pick the same field 50% of the time we pick a random field (since other fields won't match anything)
+          toField = (random.nextInt(100) < 50) ? fromField : types.get(random.nextInt(types.size())).fname;
+        } else {
+          // otherwise, pick compatible fields that have a chance of matching indexed tokens
+          String[] group = compat[random.nextInt(compat.length)];
+          fromField = group[random.nextInt(group.length)];
+          toField = group[random.nextInt(group.length)];
+        }
 
         Map<Comparable, Set<Comparable>> pivot = pivots.get(fromField+"/"+toField);
         if (pivot == null) {
@@ -146,7 +165,7 @@ public class TestJoin extends SolrTestCa
         resultSet.put("start", 0);
         resultSet.put("docs", sortedDocs);
 
-        // todo: use filters
+        // todo: use different join queries for better coverage
 
         SolrQueryRequest req = req("wt","json","indent","true", "echoParams","all",
             "q","{!join from="+fromField+" to="+toField
@@ -159,7 +178,7 @@ public class TestJoin extends SolrTestCa
         Object realResponse = ObjectBuilder.fromJSON(strResponse);
         String err = JSONTestUtil.matchObj("/response", realResponse, resultSet);
         if (err != null) {
-          log.error("GROUPING MISMATCH: " + err
+          log.error("JOIN MISMATCH: " + err
            + "\n\trequest="+req
            + "\n\tresult="+strResponse
            + "\n\texpected="+ JSONUtil.toJSON(resultSet)

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/request/JSONWriterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/request/JSONWriterTest.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/request/JSONWriterTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/request/JSONWriterTest.java Wed May 18 16:24:27 2011
@@ -73,12 +73,12 @@ public class JSONWriterTest extends Solr
 
     StringWriter buf = new StringWriter();
     NamedList nl = new NamedList();
-    nl.add("data1", "hello");
+    nl.add("data1", "he\u2028llo\u2029!");       // make sure that 2028 and 2029 are both escaped (they are illegal in javascript)
     nl.add(null, 42);
     rsp.add("nl", nl);
 
     w.write(buf, req, rsp);
-    assertEquals(buf.toString(), "{\"nl\":[[\"data1\",\"hello\"],[null,42]]}");
+    assertEquals("{\"nl\":[[\"data1\",\"he\\u2028llo\\u2029!\"],[null,42]]}", buf.toString());
     req.close();
   }
   

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java?rev=1124321&r1=1124320&r2=1124321&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java Wed May 18 16:24:27 2011
@@ -76,39 +76,74 @@ public class DistanceFunctionTest extend
     assertU(adoc("id", "100", "store", "1,2"));
     assertU(commit());
    
-    assertJQ(req("defType","func", "q","geodist(1,2,3,4)","fq","id:100","fl","id,score")
-      ,"/response/docs/[0]/score==314.40338"
-    );
+    assertJQ(req("defType","func", 
+                 "q","geodist(1,2,3,4)",
+                 "fq","id:100",
+                 "fl","id,score")
+             , 1e-5
+             , "/response/docs/[0]/score==314.40338"
+             );
 
     // throw in some decimal points
-    assertJQ(req("defType","func", "q","geodist(1.0,2,3,4.0)","fq","id:100","fl","id,score")
-      ,"/response/docs/[0]/score==314.40338"
-    );
+    assertJQ(req("defType","func", 
+                 "q","geodist(1.0,2,3,4.0)",
+                 "fq","id:100",
+                 "fl","id,score")
+             , 1e-5
+             , "/response/docs/[0]/score==314.40338"
+             );
 
     // default to reading pt
-    assertJQ(req("defType","func", "q","geodist(1,2)","pt","3,4", "fq","id:100","fl","id,score")
-      ,"/response/docs/[0]/score==314.40338"
-    );
+    assertJQ(req("defType","func", 
+                 "q","geodist(1,2)",
+                 "pt","3,4", 
+                 "fq","id:100",
+                 "fl","id,score")
+             , 1e-5
+             , "/response/docs/[0]/score==314.40338"
+             );
 
     // default to reading pt first
-    assertJQ(req("defType","func", "q","geodist(1,2)","pt","3,4", "sfield","store", "fq","id:100","fl","id,score")
-      ,"/response/docs/[0]/score==314.40338"
-    );
+    assertJQ(req("defType","func", 
+                 "q","geodist(1,2)",
+                 "pt","3,4", 
+                 "sfield","store", 
+                 "fq","id:100",
+                 "fl","id,score")
+             , 1e-5
+             , "/response/docs/[0]/score==314.40338"
+             );
 
     // if pt missing, use sfield
-    assertJQ(req("defType","func", "q","geodist(3,4)","sfield","store", "fq","id:100","fl","id,score")
-      ,"/response/docs/[0]/score==314.40338"
-    );
-
+    assertJQ(req("defType","func", 
+                 "q","geodist(3,4)",
+                 "sfield","store", 
+                 "fq","id:100",
+                 "fl","id,score")
+             , 1e-5
+             ,"/response/docs/[0]/score==314.40338"
+             );
+    
     // read both pt and sfield
-    assertJQ(req("defType","func", "q","geodist()","pt","3,4","sfield","store", "fq","id:100","fl","id,score")
-      ,"/response/docs/[0]/score==314.40338"
-    );
+    assertJQ(req("defType","func", 
+                 "q","geodist()","pt","3,4",
+                 "sfield","store", 
+                 "fq","id:100",
+                 "fl","id,score")
+             , 1e-5
+             ,"/response/docs/[0]/score==314.40338"
+             );
 
     // param substitution
-    assertJQ(req("defType","func", "q","geodist($a,$b)","a","3,4","b","store", "fq","id:100","fl","id,score")
-      ,"/response/docs/[0]/score==314.40338"
-    );
+    assertJQ(req("defType","func", 
+                 "q","geodist($a,$b)",
+                 "a","3,4",
+                 "b","store", 
+                 "fq","id:100",
+                 "fl","id,score")
+             , 1e-5
+             ,"/response/docs/[0]/score==314.40338"
+             );
 
   }