You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2011/12/28 22:03:35 UTC

svn commit: r1225335 [6/7] - in /lucene/dev/branches/solrcloud: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ dev-tools/maven/ dev-tools/maven/modules/queryparser/ lucene/ lucene/contrib/ lucene/contrib/memory/src/test/org/...

Modified: lucene/dev/branches/solrcloud/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java (original)
+++ lucene/dev/branches/solrcloud/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java Wed Dec 28 21:03:23 2011
@@ -16,14 +16,12 @@
  */
 package org.apache.solr.handler.dataimport;
 
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
+import java.util.*;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -32,118 +30,126 @@ import org.slf4j.LoggerFactory;
  * Unit test of SolrEntityProcessor. A very basic test outside of the DIH.
  */
 public class TestSolrEntityProcessorUnit extends AbstractDataImportHandlerTestCase {
-  
+
   private static final Logger LOG = LoggerFactory.getLogger(TestSolrEntityProcessorUnit.class);
   private static final String ID = "id";
-  
+
   public void testQuery() {
-    String[][][] docs = generateDocs(2);
-    
+    List<Doc> docs = generateUniqueDocs(2);
+
     MockSolrEntityProcessor processor = new MockSolrEntityProcessor(docs);
-    
+
     assertExpectedDocs(docs, processor);
     assertEquals(1, processor.getQueryCount());
   }
-  
+
   public void testNumDocsGreaterThanRows() {
-    String[][][] docs = generateDocs(44);
-    
+    List<Doc> docs = generateUniqueDocs(44);
+
     MockSolrEntityProcessor processor = new MockSolrEntityProcessor(docs, 10);
     assertExpectedDocs(docs, processor);
     assertEquals(5, processor.getQueryCount());
   }
-  
+
   public void testMultiValuedFields() {
-    String[][][] docs = new String[1][2][2];
-    String[][] doc = new String[][] { {"id", "1"}, {"multi", "multi1"},
-        {"multi", "multi2"}, {"multi", "multi3"}};
-    docs[0] = doc;
-    
+    List<Doc> docs = new ArrayList<Doc>();
+    List<FldType> types = new ArrayList<FldType>();
+    types.add(new FldType(ID, ONE_ONE, new SVal('A', 'Z', 4, 4)));
+    types.add(new FldType("description", new IRange(3, 3), new SVal('a', 'c', 1, 1)));
+    Doc testDoc = createDoc(types);
+    docs.add(testDoc);
+
     MockSolrEntityProcessor processor = new MockSolrEntityProcessor(docs);
-    
-    Map<String,Object> next = processor.nextRow();
+    Map<String, Object> next = processor.nextRow();
     assertNotNull(next);
-    assertEquals(doc[0][1], next.get(doc[0][0]));
-    
-    String[] multiValued = {"multi1", "multi2", "multi3"};
-    assertEquals(Arrays.asList(multiValued), next.get(doc[1][0]));
+
+    @SuppressWarnings("unchecked")
+    List<Comparable> multiField = (List<Comparable>) next.get("description");
+    assertEquals(testDoc.getValues("description").size(), multiField.size());
+    assertEquals(testDoc.getValues("description"), multiField);
     assertEquals(1, processor.getQueryCount());
     assertNull(processor.nextRow());
-    
   }
-  
-  public void testMultiThread() {
+
+  public void testMultiThread() throws Exception {
     int numThreads = 5;
     int numDocs = 40;
-    String[][][] docs = generateDocs(numDocs);
+    List<Doc> docs = generateUniqueDocs(numDocs);
     final MockSolrEntityProcessor entityProcessor = new MockSolrEntityProcessor(docs, 25);
-    
-    final Map<String,Map<String,Object>> rowList = new HashMap<String,Map<String,Object>>();
+    ThreadPoolExecutor executor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 5, TimeUnit.SECONDS,
+          new SynchronousQueue<Runnable>());
+
+    final Map<String, Map<String, Object>> rowList = new LinkedHashMap<String, Map<String, Object>>();
     final CountDownLatch latch = new CountDownLatch(numThreads);
+    final AtomicInteger errorCount = new AtomicInteger();
     for (int i = 0; i < numThreads; i++) {
       Runnable runnable = new Runnable() {
         public void run() {
           try {
             while (true) {
-              Map<String,Object> row;
               synchronized (entityProcessor) {
-                row = entityProcessor.nextRow();
+                Map<String, Object> row = entityProcessor.nextRow();
+                if (row == null) {
+                  break;
+                }
+                rowList.put(row.get(ID).toString(), row);
               }
-              if (row == null) {
-                break;
-              }
-              rowList.put(row.get(ID).toString(), row);
             }
+          } catch (Throwable t) {
+            errorCount.incrementAndGet();
+            LOG.error("Error in thread", t);
           } finally {
             latch.countDown();
           }
         }
       };
-      
-      new ThreadPoolExecutor(0, Integer.MAX_VALUE, 5, TimeUnit.SECONDS,
-          new SynchronousQueue<Runnable>()).execute(runnable);
-    }
-    
-    try {
-      latch.await();
-    } catch (InterruptedException e) {
-      LOG.error(e.getMessage(), e);
+      executor.execute(runnable);
     }
-    
+
+    latch.await();
+    assertEquals(0, errorCount.get());
     assertEquals(numDocs, rowList.size());
-    
-    for (String[][] expectedDoc : docs) {
-      Map<String,Object> row = rowList.get(expectedDoc[0][1]);
-      assertNotNull(row);
-      int i = 0;
-      for (Entry<String,Object> entry : row.entrySet()) {
-        assertEquals(expectedDoc[i][0], entry.getKey());
-        assertEquals(expectedDoc[i][1], entry.getValue());
-        i++;
-      }
-      rowList.remove(expectedDoc[0][1]);
+
+    for (Doc expectedDoc : docs) {
+      String id = (String) expectedDoc.getFirstValue("id");
+      Map<String, Object> row = rowList.get(id);
+      assertNotNull(id + " shouldn't yield null", row);
+      assertEquals(2, row.size());
+      assertEquals(expectedDoc.id, row.get("id"));
+      assertEquals(expectedDoc.getValues("description"), row.get("description"));
+      rowList.remove(id);
     }
-    
+
     assertEquals(0, rowList.size());
-    
+    executor.shutdown();
   }
-  
-  private static String[][][] generateDocs(int numDocs) {
-    String[][][] docs = new String[numDocs][2][2];
+
+  private List<Doc> generateUniqueDocs(int numDocs) {
+    List<FldType> types = new ArrayList<FldType>();
+    types.add(new FldType(ID, ONE_ONE, new SVal('A', 'Z', 4, 40)));
+    types.add(new FldType("description", new IRange(1, 3), new SVal('a', 'c', 1, 1)));
+
+    Set<Comparable> previousIds = new HashSet<Comparable>();
+    List<Doc> docs = new ArrayList<Doc>(numDocs);
     for (int i = 0; i < numDocs; i++) {
-      docs[i] = new String[][] { {"id", Integer.toString(i+1)},
-          {"description", "Description" + Integer.toString(i+1)}};
+      Doc doc = createDoc(types);
+      while (previousIds.contains(doc.id)) {
+        doc = createDoc(types);
+      }
+      previousIds.add(doc.id);
+      docs.add(doc);
     }
     return docs;
   }
-  
-  private static void assertExpectedDocs(String[][][] expectedDocs, SolrEntityProcessor processor) {
-    for (String[][] expectedDoc : expectedDocs) {
+
+  private static void assertExpectedDocs(List<Doc> expectedDocs, SolrEntityProcessor processor) {
+    for (Doc expectedDoc : expectedDocs) {
       Map<String, Object> next = processor.nextRow();
       assertNotNull(next);
-      assertEquals(expectedDoc[0][1], next.get(expectedDoc[0][0]));
-      assertEquals(expectedDoc[1][1], next.get(expectedDoc[1][0]));
+      assertEquals(expectedDoc.id, next.get("id"));
+      assertEquals(expectedDoc.getValues("description"), next.get("description"));
     }
     assertNull(processor.nextRow());
   }
+
 }

Modified: lucene/dev/branches/solrcloud/solr/contrib/extraction/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/contrib/extraction/CHANGES.txt?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/contrib/extraction/CHANGES.txt (original)
+++ lucene/dev/branches/solrcloud/solr/contrib/extraction/CHANGES.txt Wed Dec 28 21:03:23 2011
@@ -30,7 +30,9 @@ $Id$
 
 ================== Release 3.6.0 ==================
 
-(No Changes)
+* SOLR-2346: Add a chance to set content encoding explicitly via content type of stream.
+  This is convenient when Tika's auto detector cannot detect encoding, especially
+  the text file is too short to detect encoding. (koji)
 
 ================== Release 3.5.0 ==================
 

Modified: lucene/dev/branches/solrcloud/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingDocumentLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingDocumentLoader.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingDocumentLoader.java (original)
+++ lucene/dev/branches/solrcloud/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingDocumentLoader.java Wed Dec 28 21:03:23 2011
@@ -26,6 +26,7 @@ import org.apache.solr.common.SolrExcept
 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.ContentStreamBase;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.handler.ContentStreamLoader;
 import org.apache.solr.request.SolrQueryRequest;
@@ -158,6 +159,12 @@ public class ExtractingDocumentLoader ex
         metadata.add(ExtractingMetadataConstants.STREAM_SOURCE_INFO, stream.getSourceInfo());
         metadata.add(ExtractingMetadataConstants.STREAM_SIZE, String.valueOf(stream.getSize()));
         metadata.add(ExtractingMetadataConstants.STREAM_CONTENT_TYPE, stream.getContentType());
+        // HtmlParser and TXTParser regard Metadata.CONTENT_ENCODING in metadata
+        String charset = ContentStreamBase.getCharsetFromContentType(stream.getContentType());
+        if(charset != null){
+          metadata.add(Metadata.CONTENT_ENCODING, charset);
+        }
+
         String xpathExpr = params.get(ExtractingParams.XPATH_EXPRESSION);
         boolean extractOnly = params.getBool(ExtractingParams.EXTRACT_ONLY, false);
         SolrContentHandler handler = factory.createSolrContentHandler(metadata, params, schema);

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CodecFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CodecFactory.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CodecFactory.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CodecFactory.java Wed Dec 28 21:03:23 2011
@@ -17,7 +17,7 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.codecs.Codec;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/DefaultCodecFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/DefaultCodecFactory.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/DefaultCodecFactory.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/DefaultCodecFactory.java Wed Dec 28 21:03:23 2011
@@ -17,9 +17,9 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.PostingsFormat;
-import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40Codec;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/SolrCore.java Wed Dec 28 21:03:23 2011
@@ -17,10 +17,10 @@
 
 package org.apache.solr.core;
 
+import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.common.SolrException;

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java Wed Dec 28 21:03:23 2011
@@ -17,68 +17,67 @@
 
 package org.apache.solr.handler.component;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.StringReader;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.*;
-
-import org.apache.solr.common.params.QueryElevationParams;
-import org.apache.solr.response.transform.EditorialMarkerFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-import javax.xml.xpath.XPathFactory;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.*;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
-import org.apache.solr.cloud.ZkController;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.SentinelIntSet;
+import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.QueryElevationParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.DOMUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.Config;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.schema.StrField;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.transform.ElevatedMarkerFactory;
+import org.apache.solr.response.transform.ExcludedMarkerFactory;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.SortSpec;
 import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.util.VersionedFile;
+import org.apache.solr.search.SortSpec;
 import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.VersionedFile;
 import org.apache.solr.util.plugin.SolrCoreAware;
-import org.apache.solr.request.SolrQueryRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathFactory;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.*;
+
 /**
  * A component to elevate some documents to the top of the result set.
- * 
  *
  * @since solr 1.3
  */
-public class QueryElevationComponent extends SearchComponent implements SolrCoreAware
-{
+public class QueryElevationComponent extends SearchComponent implements SolrCoreAware {
   private static Logger log = LoggerFactory.getLogger(QueryElevationComponent.class);
-  
+
   // Constants used in solrconfig.xml
   static final String FIELD_TYPE = "queryFieldType";
   static final String CONFIG_FILE = "config-file";
   static final String EXCLUDE = "exclude";
-  
+  public static final String BOOSTED = "BOOSTED";
+  public static final String EXCLUDED = "EXCLUDED";
+
   // Runtime param -- should be in common?
 
   private SolrParams initArgs = null;
@@ -91,238 +90,235 @@ public class QueryElevationComponent ext
   // When the configuration is loaded from the data directory.
   // The key is null if loaded from the config directory, and
   // is never re-loaded.
-  final Map<IndexReader,Map<String, ElevationObj>> elevationCache =
-    new WeakHashMap<IndexReader, Map<String,ElevationObj>>();
+  final Map<IndexReader, Map<String, ElevationObj>> elevationCache =
+      new WeakHashMap<IndexReader, Map<String, ElevationObj>>();
 
   class ElevationObj {
     final String text;
     final String analyzed;
-    final BooleanClause[] exclude;
+    final TermQuery [] exclude;//just keep the term query, b/c we will not always explicitly exclude the item based on markExcludes query time param
     final BooleanQuery include;
-    final Map<BytesRef,Integer> priority;
+    final Map<BytesRef, Integer> priority;
     final Set<String> ids;
-    
-    // use singletons so hashCode/equals on Sort will just work
-    final FieldComparatorSource comparatorSource;
+    final Set<String> excludeIds;
 
-    ElevationObj( String qstr, List<String> elevate, List<String> exclude ) throws IOException
-    {
+    ElevationObj(String qstr, List<String> elevate, List<String> exclude) throws IOException {
       this.text = qstr;
-      this.analyzed = getAnalyzedQuery( this.text );
+      this.analyzed = getAnalyzedQuery(this.text);
       this.ids = new HashSet<String>();
-      
+      this.excludeIds = new HashSet<String>();
+
       this.include = new BooleanQuery();
-      this.include.setBoost( 0 );
+      this.include.setBoost(0);
       this.priority = new HashMap<BytesRef, Integer>();
-      int max = elevate.size()+5;
-      for( String id : elevate ) {
+      int max = elevate.size() + 5;
+      for (String id : elevate) {
         id = idSchemaFT.readableToIndexed(id);
         ids.add(id);
-        TermQuery tq = new TermQuery( new Term( idField, id ) );
-        include.add( tq, BooleanClause.Occur.SHOULD );
-        this.priority.put( new BytesRef(id), max-- );
+        TermQuery tq = new TermQuery(new Term(idField, id));
+        include.add(tq, BooleanClause.Occur.SHOULD);
+        this.priority.put(new BytesRef(id), max--);
       }
-      
-      if( exclude == null || exclude.isEmpty() ) {
+
+      if (exclude == null || exclude.isEmpty()) {
         this.exclude = null;
-      }
-      else {
-        this.exclude = new BooleanClause[exclude.size()];
-        for( int i=0; i<exclude.size(); i++ ) {
-          TermQuery tq = new TermQuery( new Term( idField, idSchemaFT.readableToIndexed(exclude.get(i)) ) );
-          this.exclude[i] = new BooleanClause( tq, BooleanClause.Occur.MUST_NOT );
+      } else {
+        this.exclude = new TermQuery[exclude.size()];
+        for (int i = 0; i < exclude.size(); i++) {
+          String id = idSchemaFT.readableToIndexed(exclude.get(i));
+          excludeIds.add(id);
+          this.exclude[i] = new TermQuery(new Term(idField, id));
         }
       }
-
-      this.comparatorSource = new ElevationComparatorSource(priority);
     }
   }
-  
+
   @Override
-  public void init( NamedList args )
-  {
-    this.initArgs = SolrParams.toSolrParams( args );
-  }
-  
-  public void inform(SolrCore core)
-  {
-    String a = initArgs.get( FIELD_TYPE );
-    if( a != null ) {
-      FieldType ft = core.getSchema().getFieldTypes().get( a );
-      if( ft == null ) {
-        throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-            "Unknown FieldType: '"+a+"' used in QueryElevationComponent" );
+  public void init(NamedList args) {
+    this.initArgs = SolrParams.toSolrParams(args);
+  }
+
+  public void inform(SolrCore core) {
+    String a = initArgs.get(FIELD_TYPE);
+    if (a != null) {
+      FieldType ft = core.getSchema().getFieldTypes().get(a);
+      if (ft == null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Unknown FieldType: '" + a + "' used in QueryElevationComponent");
       }
       analyzer = ft.getQueryAnalyzer();
     }
 
     SchemaField sf = core.getSchema().getUniqueKeyField();
-    if( sf == null || sf.getType().isTokenized() == true) {
+    if( sf == null) {
       throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, 
-          "QueryElevationComponent requires the schema to have a uniqueKeyField implemented using a non-tokenized field" );
+          "QueryElevationComponent requires the schema to have a uniqueKeyField." );
     }
     idSchemaFT = sf.getType();
     idField = sf.getName();
     //register the EditorialMarkerFactory
-    EditorialMarkerFactory factory = new EditorialMarkerFactory();
+    String excludeName = initArgs.get(QueryElevationParams.EXCLUDE_MARKER_FIELD_NAME, "excluded");
+    if (excludeName == null || excludeName.equals("") == true){
+      excludeName = "excluded";
+    }
+    ExcludedMarkerFactory excludedMarkerFactory = new ExcludedMarkerFactory();
+    core.addTransformerFactory(excludeName, excludedMarkerFactory);
+    ElevatedMarkerFactory elevatedMarkerFactory = new ElevatedMarkerFactory();
     String markerName = initArgs.get(QueryElevationParams.EDITORIAL_MARKER_FIELD_NAME, "elevated");
-    if (markerName == null || markerName.equals("") == true){
+    if (markerName == null || markerName.equals("") == true) {
       markerName = "elevated";
     }
-    core.addTransformerFactory(markerName, factory);
-    forceElevation = initArgs.getBool( QueryElevationParams.FORCE_ELEVATION, forceElevation );
+    core.addTransformerFactory(markerName, elevatedMarkerFactory);
+    forceElevation = initArgs.getBool(QueryElevationParams.FORCE_ELEVATION, forceElevation);
     try {
-      synchronized( elevationCache ) {
+      synchronized (elevationCache) {
         elevationCache.clear();
-        String f = initArgs.get( CONFIG_FILE );
-        if( f == null ) {
-          throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-              "QueryElevationComponent must specify argument: '"+CONFIG_FILE
-              +"' -- path to elevate.xml" );
+        String f = initArgs.get(CONFIG_FILE);
+        if (f == null) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "QueryElevationComponent must specify argument: '" + CONFIG_FILE
+                  + "' -- path to elevate.xml");
         }
         boolean exists = false;
 
         // check if using ZooKeeper
         ZkController zkController = core.getCoreDescriptor().getCoreContainer().getZkController();
-        if(zkController != null) {
+        if (zkController != null) {
           // TODO : shouldn't have to keep reading the config name when it has been read before
           exists = zkController.configFileExists(zkController.readConfigName(core.getCoreDescriptor().getCloudDescriptor().getCollectionName()), f);
         } else {
-          File fC = new File( core.getResourceLoader().getConfigDir(), f );
-          File fD = new File( core.getDataDir(), f );
-          if( fC.exists() == fD.exists() ) {
-            throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-                "QueryElevationComponent missing config file: '"+f + "\n"
-                +"either: "+fC.getAbsolutePath() + " or " + fD.getAbsolutePath() + " must exist, but not both." );
+          File fC = new File(core.getResourceLoader().getConfigDir(), f);
+          File fD = new File(core.getDataDir(), f);
+          if (fC.exists() == fD.exists()) {
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                "QueryElevationComponent missing config file: '" + f + "\n"
+                    + "either: " + fC.getAbsolutePath() + " or " + fD.getAbsolutePath() + " must exist, but not both.");
           }
-          if( fC.exists() ) {
+          if (fC.exists()) {
             exists = true;
-            log.info( "Loading QueryElevation from: "+ fC.getAbsolutePath() );
-            Config cfg = new Config( core.getResourceLoader(), f );
-            elevationCache.put(null, loadElevationMap( cfg ));
-          } 
+            log.info("Loading QueryElevation from: " + fC.getAbsolutePath());
+            Config cfg = new Config(core.getResourceLoader(), f);
+            elevationCache.put(null, loadElevationMap(cfg));
+          }
         }
         //in other words, we think this is in the data dir, not the conf dir
-        if (!exists){
+        if (!exists) {
           // preload the first data
           RefCounted<SolrIndexSearcher> searchHolder = null;
           try {
             searchHolder = core.getNewestSearcher(false);
             IndexReader reader = searchHolder.get().getIndexReader();
-            getElevationMap( reader, core );
+            getElevationMap(reader, core);
           } finally {
             if (searchHolder != null) searchHolder.decref();
           }
         }
       }
-    }
-    catch( Exception ex ) {
-      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-          "Error initializing QueryElevationComponent.", ex, false );
+    } catch (Exception ex) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Error initializing QueryElevationComponent.", ex, false);
     }
   }
+
   //get the elevation map from the data dir
-  Map<String, ElevationObj> getElevationMap( IndexReader reader, SolrCore core ) throws Exception
-  {
-    synchronized( elevationCache ) {
-      Map<String, ElevationObj> map = elevationCache.get( null );
+  Map<String, ElevationObj> getElevationMap(IndexReader reader, SolrCore core) throws Exception {
+    synchronized (elevationCache) {
+      Map<String, ElevationObj> map = elevationCache.get(null);
       if (map != null) return map;
 
-      map = elevationCache.get( reader );
-      if( map == null ) {
-        String f = initArgs.get( CONFIG_FILE );
-        if( f == null ) {
-          throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-                  "QueryElevationComponent must specify argument: "+CONFIG_FILE );
-        }
-        log.info( "Loading QueryElevation from data dir: "+f );
-
-        InputStream is = VersionedFile.getLatestFile( core.getDataDir(), f );
-        Config cfg = new Config( core.getResourceLoader(), f, new InputSource(is), null );
-        map = loadElevationMap( cfg );
-        elevationCache.put( reader, map );
+      map = elevationCache.get(reader);
+      if (map == null) {
+        String f = initArgs.get(CONFIG_FILE);
+        if (f == null) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "QueryElevationComponent must specify argument: " + CONFIG_FILE);
+        }
+        log.info("Loading QueryElevation from data dir: " + f);
+
+        InputStream is = VersionedFile.getLatestFile(core.getDataDir(), f);
+        Config cfg = new Config(core.getResourceLoader(), f, new InputSource(is), null);
+        map = loadElevationMap(cfg);
+        elevationCache.put(reader, map);
       }
       return map;
     }
   }
+
   //load up the elevation map
-  private Map<String, ElevationObj> loadElevationMap( Config cfg ) throws IOException
-  {
+  private Map<String, ElevationObj> loadElevationMap(Config cfg) throws IOException {
     XPath xpath = XPathFactory.newInstance().newXPath();
     Map<String, ElevationObj> map = new HashMap<String, ElevationObj>();
-    NodeList nodes = (NodeList)cfg.evaluate( "elevate/query", XPathConstants.NODESET );
-    for (int i=0; i<nodes.getLength(); i++) {
-      Node node = nodes.item( i );
-      String qstr = DOMUtil.getAttr( node, "text", "missing query 'text'" );
-      
+    NodeList nodes = (NodeList) cfg.evaluate("elevate/query", XPathConstants.NODESET);
+    for (int i = 0; i < nodes.getLength(); i++) {
+      Node node = nodes.item(i);
+      String qstr = DOMUtil.getAttr(node, "text", "missing query 'text'");
+
       NodeList children = null;
       try {
-        children = (NodeList)xpath.evaluate("doc", node, XPathConstants.NODESET);
-      } 
-      catch (XPathExpressionException e) {
-        throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, 
-            "query requires '<doc .../>' child" );
+        children = (NodeList) xpath.evaluate("doc", node, XPathConstants.NODESET);
+      } catch (XPathExpressionException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "query requires '<doc .../>' child");
       }
 
       ArrayList<String> include = new ArrayList<String>();
       ArrayList<String> exclude = new ArrayList<String>();
-      for (int j=0; j<children.getLength(); j++) {
+      for (int j = 0; j < children.getLength(); j++) {
         Node child = children.item(j);
-        String id = DOMUtil.getAttr( child, "id", "missing 'id'" );
-        String e = DOMUtil.getAttr( child, EXCLUDE, null );
-        if( e != null ) {
-          if( Boolean.valueOf( e ) ) {
-            exclude.add( id );
+        String id = DOMUtil.getAttr(child, "id", "missing 'id'");
+        String e = DOMUtil.getAttr(child, EXCLUDE, null);
+        if (e != null) {
+          if (Boolean.valueOf(e)) {
+            exclude.add(id);
             continue;
           }
         }
-        include.add( id );
+        include.add(id);
       }
-      
-      ElevationObj elev = new ElevationObj( qstr, include, exclude );
-      if( map.containsKey( elev.analyzed ) ) {
-        throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, 
-            "Boosting query defined twice for query: '"+elev.text+"' ("+elev.analyzed+"')" );
+
+      ElevationObj elev = new ElevationObj(qstr, include, exclude);
+      if (map.containsKey(elev.analyzed)) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Boosting query defined twice for query: '" + elev.text + "' (" + elev.analyzed + "')");
       }
-      map.put( elev.analyzed, elev );
+      map.put(elev.analyzed, elev);
     }
     return map;
   }
-  
+
   /**
    * Helpful for testing without loading config.xml
-   * @throws IOException 
+   *
+   * @throws IOException
    */
-  void setTopQueryResults( IndexReader reader, String query, String[] ids, String[] ex ) throws IOException
-  {
-    if( ids == null ) {
+  void setTopQueryResults(IndexReader reader, String query, String[] ids, String[] ex) throws IOException {
+    if (ids == null) {
       ids = new String[0];
     }
-    if( ex == null ) {
+    if (ex == null) {
       ex = new String[0];
     }
-    
-    Map<String,ElevationObj> elev = elevationCache.get( reader );
-    if( elev == null ) {
+
+    Map<String, ElevationObj> elev = elevationCache.get(reader);
+    if (elev == null) {
       elev = new HashMap<String, ElevationObj>();
-      elevationCache.put( reader, elev );
+      elevationCache.put(reader, elev);
     }
-    ElevationObj obj = new ElevationObj( query, Arrays.asList(ids), Arrays.asList(ex) );
-    elev.put( obj.analyzed, obj );
+    ElevationObj obj = new ElevationObj(query, Arrays.asList(ids), Arrays.asList(ex));
+    elev.put(obj.analyzed, obj);
   }
-  
-  String getAnalyzedQuery( String query ) throws IOException
-  {
-    if( analyzer == null ) {
+
+  String getAnalyzedQuery(String query) throws IOException {
+    if (analyzer == null) {
       return query;
     }
     StringBuilder norm = new StringBuilder();
     TokenStream tokens = analyzer.tokenStream("", new StringReader(query));
     tokens.reset();
-    
+
     CharTermAttribute termAtt = tokens.addAttribute(CharTermAttribute.class);
-    while( tokens.incrementToken() ) {
-      norm.append( termAtt.buffer(), 0, termAtt.length() );
+    while (tokens.incrementToken()) {
+      norm.append(termAtt.buffer(), 0, termAtt.length());
     }
     tokens.end();
     tokens.close();
@@ -332,24 +328,23 @@ public class QueryElevationComponent ext
   //---------------------------------------------------------------------------------
   // SearchComponent
   //---------------------------------------------------------------------------------
-  
+
   @Override
-  public void prepare(ResponseBuilder rb) throws IOException
-  {
+  public void prepare(ResponseBuilder rb) throws IOException {
     SolrQueryRequest req = rb.req;
     SolrParams params = req.getParams();
     // A runtime param can skip 
-    if( !params.getBool( QueryElevationParams.ENABLE, true ) ) {
+    if (!params.getBool(QueryElevationParams.ENABLE, true)) {
       return;
     }
 
     boolean exclusive = params.getBool(QueryElevationParams.EXCLUSIVE, false);
     // A runtime parameter can alter the config value for forceElevation
-    boolean force = params.getBool( QueryElevationParams.FORCE_ELEVATION, forceElevation );
-    
+    boolean force = params.getBool(QueryElevationParams.FORCE_ELEVATION, forceElevation);
+    boolean markExcludes = params.getBool(QueryElevationParams.MARK_EXCLUDES, false);
     Query query = rb.getQuery();
     String qstr = rb.getQueryString();
-    if( query == null || qstr == null) {
+    if (query == null || qstr == null) {
       return;
     }
 
@@ -357,81 +352,89 @@ public class QueryElevationComponent ext
     IndexReader reader = req.getSearcher().getIndexReader();
     ElevationObj booster = null;
     try {
-      booster = getElevationMap( reader, req.getCore() ).get( qstr );
+      booster = getElevationMap(reader, req.getCore()).get(qstr);
+    } catch (Exception ex) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Error loading elevation", ex);
     }
-    catch( Exception ex ) {
-      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-          "Error loading elevation", ex );      
-    }
-    
-    if( booster != null ) {
-      rb.req.getContext().put("BOOSTED", booster.ids);
-      
+
+    if (booster != null) {
+      rb.req.getContext().put(BOOSTED, booster.ids);
+
       // Change the query to insert forced documents
-      if (exclusive == true){
+      if (exclusive == true) {
         //we only want these results
         rb.setQuery(booster.include);
       } else {
-        BooleanQuery newq = new BooleanQuery( true );
-        newq.add( query, BooleanClause.Occur.SHOULD );
-        newq.add( booster.include, BooleanClause.Occur.SHOULD );
-        if( booster.exclude != null ) {
-          for( BooleanClause bq : booster.exclude ) {
-            newq.add( bq );
+        BooleanQuery newq = new BooleanQuery(true);
+        newq.add(query, BooleanClause.Occur.SHOULD);
+        newq.add(booster.include, BooleanClause.Occur.SHOULD);
+        if (booster.exclude != null) {
+          if (markExcludes == false) {
+            for (TermQuery tq : booster.exclude) {
+              newq.add(new BooleanClause(tq, BooleanClause.Occur.MUST_NOT));
+            }
+          } else {
+            //we are only going to mark items as excluded, not actually exclude them.  This works
+            //with the EditorialMarkerFactory
+            rb.req.getContext().put(EXCLUDED, booster.excludeIds);
+            for (TermQuery tq : booster.exclude) {
+              newq.add(new BooleanClause(tq, BooleanClause.Occur.SHOULD));
+            }
           }
         }
-        rb.setQuery( newq );
+        rb.setQuery(newq);
       }
 
-      
+      ElevationComparatorSource comparator = new ElevationComparatorSource(booster);
       // if the sort is 'score desc' use a custom sorting method to 
       // insert documents in their proper place 
       SortSpec sortSpec = rb.getSortSpec();
-      if( sortSpec.getSort() == null ) {
-        sortSpec.setSort( new Sort( 
-            new SortField(idField, booster.comparatorSource, false ),
-            new SortField(null, SortField.Type.SCORE, false)));
-      }
-      else {
+      if (sortSpec.getSort() == null) {
+        sortSpec.setSort(new Sort(new SortField[]{
+            new SortField(idField, comparator, false),
+            new SortField(null, SortField.Type.SCORE, false)
+        }));
+      } else {
         // Check if the sort is based on score
         boolean modify = false;
         SortField[] current = sortSpec.getSort().getSort();
-        ArrayList<SortField> sorts = new ArrayList<SortField>( current.length + 1 );
+        ArrayList<SortField> sorts = new ArrayList<SortField>(current.length + 1);
         // Perhaps force it to always sort by score
-        if( force && current[0].getType() != SortField.Type.SCORE ) {
-          sorts.add( new SortField(idField, booster.comparatorSource, false ) );
+        if (force && current[0].getType() != SortField.Type.SCORE) {
+          sorts.add(new SortField(idField, comparator, false));
           modify = true;
         }
-        for( SortField sf : current ) {
-          if( sf.getType() == SortField.Type.SCORE ) {
-            sorts.add( new SortField(idField, booster.comparatorSource, sf.getReverse() ) );
+        for (SortField sf : current) {
+          if (sf.getType() == SortField.Type.SCORE) {
+            sorts.add(new SortField(idField, comparator, sf.getReverse()));
             modify = true;
           }
-          sorts.add( sf );
+          sorts.add(sf);
         }
-        if( modify ) {
-          sortSpec.setSort( new Sort( sorts.toArray( new SortField[sorts.size()] ) ) );
+        if (modify) {
+          sortSpec.setSort(new Sort(sorts.toArray(new SortField[sorts.size()])));
         }
       }
     }
-    
+
     // Add debugging information
-    if( rb.isDebug() ) {
+    if (rb.isDebug()) {
       List<String> match = null;
-      if( booster != null ) {
+      if (booster != null) {
         // Extract the elevated terms into a list
         match = new ArrayList<String>(booster.priority.size());
-        for( Object o : booster.include.clauses() ) {
-          TermQuery tq = (TermQuery)((BooleanClause)o).getQuery();
-          match.add( tq.getTerm().text() );
+        for (Object o : booster.include.clauses()) {
+          TermQuery tq = (TermQuery) ((BooleanClause) o).getQuery();
+          match.add(tq.getTerm().text());
         }
       }
-      
+
       SimpleOrderedMap<Object> dbg = new SimpleOrderedMap<Object>();
-      dbg.add( "q", qstr );
-      dbg.add( "match", match );
+      dbg.add("q", qstr);
+      dbg.add("match", match);
       if (rb.isDebugQuery()) {
-        rb.addDebugInfo("queryBoosting", dbg );
+        rb.addDebugInfo("queryBoosting", dbg);
       }
     }
   }
@@ -440,7 +443,7 @@ public class QueryElevationComponent ext
   public void process(ResponseBuilder rb) throws IOException {
     // Do nothing -- the real work is modifying the input query
   }
-    
+
   //---------------------------------------------------------------------------------
   // SolrInfoMBean
   //---------------------------------------------------------------------------------
@@ -468,31 +471,33 @@ public class QueryElevationComponent ext
   @Override
   public URL[] getDocs() {
     try {
-      return new URL[] {
-        new URL("http://wiki.apache.org/solr/QueryElevationComponent")
+      return new URL[]{
+          new URL("http://wiki.apache.org/solr/QueryElevationComponent")
       };
-    } 
-    catch (MalformedURLException e) {
-      throw new RuntimeException( e );
+    } catch (MalformedURLException e) {
+      throw new RuntimeException(e);
     }
   }
-}
-
-class ElevationComparatorSource extends FieldComparatorSource {
-  private final Map<BytesRef,Integer> priority;
-
-  public ElevationComparatorSource( final Map<BytesRef,Integer> boosts) {
-    this.priority = boosts;
+  class ElevationComparatorSource extends FieldComparatorSource {
+  private QueryElevationComponent.ElevationObj elevations;
+  private SentinelIntSet ordSet; //the key half of the map
+  private BytesRef[] termValues;//the value half of the map
+
+  public ElevationComparatorSource(final QueryElevationComponent.ElevationObj elevations) throws IOException {
+    this.elevations = elevations;
+    int size = elevations.ids.size();
+    ordSet = new SentinelIntSet(size, -1);
+    termValues = new BytesRef[ordSet.keys.length];
   }
 
   @Override
   public FieldComparator<Integer> newComparator(final String fieldname, final int numHits, int sortPos, boolean reversed) throws IOException {
     return new FieldComparator<Integer>() {
-      
-      FieldCache.DocTermsIndex idIndex;
       private final int[] values = new int[numHits];
-      int bottomVal;
-      private final BytesRef tempBR = new BytesRef();
+      private int bottomVal;
+      private TermsEnum termsEnum;
+      private DocsEnum docsEnum;
+      Set<String> seen = new HashSet<String>(elevations.ids.size());
 
       @Override
       public int compare(int slot1, int slot2) {
@@ -505,9 +510,15 @@ class ElevationComparatorSource extends 
       }
 
       private int docVal(int doc) throws IOException {
-        BytesRef id = idIndex.getTerm(doc, tempBR);
-        Integer prio = priority.get(id);
-        return prio == null ? 0 : prio.intValue();
+        if (ordSet.size() > 0) {
+          int slot = ordSet.find(doc);
+          if (slot >= 0) {
+            BytesRef id = termValues[slot];
+            Integer prio = elevations.priority.get(id);
+            return prio == null ? 0 : prio.intValue();
+          }
+        }
+        return 0;
       }
 
       @Override
@@ -522,7 +533,29 @@ class ElevationComparatorSource extends 
 
       @Override
       public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
-        idIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, fieldname);
+        //convert the ids to Lucene doc ids, the ordSet and termValues needs to be the same size as the number of elevation docs we have
+        ordSet.clear();
+        Fields fields = context.reader.fields();
+        if (fields == null) return this;
+        Terms terms = fields.terms(fieldname);
+        if (terms == null) return this;
+        termsEnum = terms.iterator(termsEnum);
+        BytesRef term = new BytesRef();
+        Bits liveDocs = context.reader.getLiveDocs();
+
+        for (String id : elevations.ids) {
+          term.copyChars(id);
+          if (seen.contains(id) == false  && termsEnum.seekExact(term, false)) {
+            docsEnum = termsEnum.docs(liveDocs, docsEnum, false);
+            if (docsEnum != null) {
+              int docId = docsEnum.nextDoc();
+              if (docId == DocIdSetIterator.NO_MORE_DOCS ) continue;  // must have been deleted
+              termValues[ordSet.put(docId)] = BytesRef.deepCopyOf(term);
+              seen.add(id);
+              assert docsEnum.nextDoc() == DocIdSetIterator.NO_MORE_DOCS;
+            }
+          }
+        }
         return this;
       }
 
@@ -533,3 +566,6 @@ class ElevationComparatorSource extends 
     };
   }
 }
+}
+
+

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java Wed Dec 28 21:03:23 2011
@@ -34,7 +34,9 @@ import java.io.IOException;
  * <li>It's OK to have some documents without a keyField in the file (defVal is used as the default)</li>
  * <li>It's OK for a keyField value to point to multiple documents (no uniqueness requirement)</li>
  * </ul>
- * <code>valType</code> is a reference to another fieldType to define the value type of this field (must currently be FloatField (float))
+ * <code>valType</code> is a reference to another fieldType to define the value type of this field
+ * (must currently be TrieFloatField or FloatField (valType="pfloat|float|tfloat") if used).
+ * This parameter has never been implemented. As of Solr 3.6/4.0 it is optional and can be omitted.
  *
  * The format of the external file is simply newline separated keyFieldValue=floatValue.
  * <br/>Example:
@@ -60,18 +62,21 @@ public class ExternalFileField extends F
   private float defVal;
 
   @Override
-  protected void init(IndexSchema schema, Map<String,String> args) {
+  protected void init(IndexSchema schema, Map<String, String> args) {
     restrictProps(SORT_MISSING_FIRST | SORT_MISSING_LAST);
-    String ftypeS = getArg("valType", args);
-    if (ftypeS!=null) {
+    // valType has never been used for anything except to throw an error, so make it optional since the
+    // code (see getValueSource) gives you a FileFloatSource.
+    String ftypeS = args.remove("valType");
+    if (ftypeS != null) {
       ftype = schema.getFieldTypes().get(ftypeS);
-      if (ftype==null || !(ftype instanceof FloatField)) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Only float (FloatField) is currently supported as external field type.  got " + ftypeS);
+      if (ftype != null && !(ftype instanceof FloatField) && !(ftype instanceof TrieFloatField)) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Only float and pfloat (Trie|Float)Field are currently supported as external field type.  Got " + ftypeS);
       }
-    }   
+    }
     keyFieldName = args.remove("keyField");
     String defValS = args.remove("defVal");
-    defVal = defValS==null ? 0 : Float.parseFloat(defValS);
+    defVal = defValS == null ? 0 : Float.parseFloat(defValS);
     this.schema = schema;
   }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java Wed Dec 28 21:03:23 2011
@@ -57,7 +57,7 @@ public class JoinQParserPlugin extends Q
         String fromIndex = getParam("fromIndex");
         String toField = getParam("to");
         String v = localParams.get("v");
-        QParser fromQueryParser = subQuery(v, "lucene");
+        QParser fromQueryParser = subQuery(v, null);
         Query fromQuery = fromQueryParser.getQuery();
         JoinQuery jq = new JoinQuery(fromField, toField, fromIndex, fromQuery);
         return jq;

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java Wed Dec 28 21:03:23 2011
@@ -26,10 +26,10 @@ import java.text.DateFormat;
 import java.util.Date;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.PrintStreamInfoStream;

Modified: lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/elevate.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/elevate.xml?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/elevate.xml (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/elevate.xml Wed Dec 28 21:03:23 2011
@@ -34,9 +34,13 @@
  </query>
 
  <query text="AAAA">
-  <doc id="7.0" />
+  <doc id="7" />
  </query>
 
+ <query text="XXXX XXXX">
+  <doc id="5" />
+  <doc id="6" exclude="true" />
+ </query>
 
 
 </elevate>

Modified: lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/schema11.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/schema11.xml?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/schema11.xml (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/schema11.xml Wed Dec 28 21:03:23 2011
@@ -267,6 +267,24 @@
 valued. -->
     <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
 
+     <!-- These should pass right through and insure that we can declare external field types -->
+    <fieldType name="pfloat" class="solr.FloatField" omitNorms="true"/>
+    <fieldType name="eff_float" keyField="id" defVal="0"
+               stored="false" indexed="true"
+               class="solr.ExternalFileField" valType="float"/>
+
+    <fieldType name="eff_tfloat" keyField="id" defVal="0"
+               stored="false" indexed="true"
+               class="solr.ExternalFileField" valType="tfloat"/>
+
+    <fieldType name="eff_pfloat" keyField="id" defVal="0"
+               stored="false" indexed="true"
+               class="solr.ExternalFileField" valType="pfloat"/>
+
+    <!-- Be sure that the valType can be optional Since valType has done nothing up until now, this is preferred -->
+    <fieldType name="eff_none" keyField="id" defVal="0"
+               stored="false" indexed="true"
+               class="solr.ExternalFileField"/>
 
  </types>
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/solrconfig.xml?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/solrconfig.xml (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test-files/solr/conf/solrconfig.xml Wed Dec 28 21:03:23 2011
@@ -429,7 +429,6 @@
 			</arr>
  </requestHandler>
 
-
   <searchComponent name="tvComponent" class="org.apache.solr.handler.component.TermVectorComponent"/>
 
   <requestHandler name="tvrh" class="org.apache.solr.handler.component.SearchHandler">

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/TestJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/TestJoin.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/TestJoin.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/TestJoin.java Wed Dec 28 21:03:23 2011
@@ -18,6 +18,9 @@
 package org.apache.solr;
 
 import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.BooleanQuery;
+
 import org.apache.noggit.JSONUtil;
 import org.apache.noggit.ObjectBuilder;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -27,6 +30,7 @@ import org.apache.solr.request.SolrQuery
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.servlet.DirectSolrConnection;
+import org.apache.solr.search.QParser;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -92,6 +96,37 @@ public class TestJoin extends SolrTestCa
     assertJQ(req("q","{!join from=dept_s to=dept_id_s}title:MTS", "fl","id", "debugQuery","true")
         ,"/response=={'numFound':3,'start':0,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
     );
+    
+    // expected outcome for a sub query matching dave joined against departments
+    final String davesDepartments = 
+      "/response=={'numFound':2,'start':0,'docs':[{'id':'10'},{'id':'13'}]}";
+
+    // straight forward query
+    assertJQ(req("q","{!join from=dept_s to=dept_id_s}name:dave", 
+                 "fl","id"),
+             davesDepartments);
+
+    // variable deref for sub-query parsing
+    assertJQ(req("q","{!join from=dept_s to=dept_id_s v=$qq}", 
+                 "qq","{!dismax}dave",
+                 "qf","name",
+                 "fl","id", 
+                 "debugQuery","true"),
+             davesDepartments);
+
+    // variable deref for sub-query parsing w/localparams
+    assertJQ(req("q","{!join from=dept_s to=dept_id_s v=$qq}", 
+                 "qq","{!dismax qf=name}dave",
+                 "fl","id", 
+                 "debugQuery","true"),
+             davesDepartments);
+
+    // defType local param to control sub-query parsing
+    assertJQ(req("q","{!join from=dept_s to=dept_id_s defType=dismax}dave", 
+                 "qf","name",
+                 "fl","id", 
+                 "debugQuery","true"),
+             davesDepartments);
 
   }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java Wed Dec 28 21:03:23 2011
@@ -19,8 +19,8 @@ package org.apache.solr.core;
 
 import java.util.Map;
 
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.schema.SchemaField;
 import org.junit.BeforeClass;

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java Wed Dec 28 21:03:23 2011
@@ -17,13 +17,6 @@
 
 package org.apache.solr.handler.component;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.SolrTestCaseJ4;
@@ -36,11 +29,14 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.component.QueryElevationComponent.ElevationObj;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
-import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.Map;
 
 
 public class QueryElevationComponentTest extends SolrTestCaseJ4 {
@@ -48,11 +44,15 @@ public class QueryElevationComponentTest
 
   @Before
   @Override
-  public void setUp() throws Exception{
+  public void setUp() throws Exception {
     super.setUp();
   }
 
   private void init(String schema) throws Exception {
+    init("solrconfig-elevate.xml", schema);
+  }
+
+  private void init(String config, String schema) throws Exception {
     //write out elevate-data.xml to the Data dir first by copying it from conf, which we know exists, this way we can test both conf and data configurations
     createTempDir();
     File parent = new File(TEST_HOME(), "conf");
@@ -60,7 +60,8 @@ public class QueryElevationComponentTest
     File elevateDataFile = new File(dataDir, "elevate-data.xml");
     FileUtils.copyFile(elevateFile, elevateDataFile);
 
-    initCore("solrconfig-elevate.xml",schema);
+
+    initCore(config,schema);
     clearIndex();
     assertU(commit());
   }
@@ -75,6 +76,39 @@ public class QueryElevationComponentTest
       init("schema11.xml");
       clearIndex();
       assertU(commit());
+      assertU(adoc("id", "1", "text", "XXXX XXXX", "str_s", "a"));
+      assertU(adoc("id", "2", "text", "YYYY", "str_s", "b"));
+      assertU(adoc("id", "3", "text", "ZZZZ", "str_s", "c"));
+
+      assertU(adoc("id", "4", "text", "XXXX XXXX", "str_s", "x"));
+      assertU(adoc("id", "5", "text", "YYYY YYYY", "str_s", "y"));
+      assertU(adoc("id", "6", "text", "XXXX XXXX", "str_s", "z"));
+      assertU(adoc("id", "7", "text", "AAAA", "str_s", "a"));
+      assertU(adoc("id", "8", "text", "AAAA", "str_s", "a"));
+      assertU(adoc("id", "9", "text", "AAAA AAAA", "str_s", "a"));
+      assertU(commit());
+
+      assertQ("", req(CommonParams.Q, "AAAA", CommonParams.QT, "/elevate",
+          CommonParams.FL, "id, score, [elevated]")
+          , "//*[@numFound='3']"
+          , "//result/doc[1]/float[@name='id'][.='7.0']"
+          , "//result/doc[2]/float[@name='id'][.='8.0']"
+          , "//result/doc[3]/float[@name='id'][.='9.0']",
+          "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[2]/bool[@name='[elevated]'][.='false']",
+          "//result/doc[3]/bool[@name='[elevated]'][.='false']"
+      );
+    } finally {
+      delete();
+    }
+  }
+
+  @Test
+  public void testTrieFieldType() throws Exception {
+    try {
+      init("schema.xml");
+      clearIndex();
+      assertU(commit());
       assertU(adoc("id", "1", "text", "XXXX XXXX",           "str_s", "a" ));
       assertU(adoc("id", "2", "text", "YYYY",      "str_s", "b" ));
       assertU(adoc("id", "3", "text", "ZZZZ", "str_s", "c" ));
@@ -90,9 +124,9 @@ public class QueryElevationComponentTest
       assertQ("", req(CommonParams.Q, "AAAA", CommonParams.QT, "/elevate",
           CommonParams.FL, "id, score, [elevated]")
               ,"//*[@numFound='3']"
-              ,"//result/doc[1]/float[@name='id'][.='7.0']"
-              ,"//result/doc[2]/float[@name='id'][.='8.0']"
-              ,"//result/doc[3]/float[@name='id'][.='9.0']",
+              ,"//result/doc[1]/int[@name='id'][.='7']"
+              ,"//result/doc[2]/int[@name='id'][.='8']"
+              ,"//result/doc[3]/int[@name='id'][.='9']",
               "//result/doc[1]/bool[@name='[elevated]'][.='true']",
               "//result/doc[2]/bool[@name='[elevated]'][.='false']",
               "//result/doc[3]/bool[@name='[elevated]'][.='false']"
@@ -102,56 +136,56 @@ public class QueryElevationComponentTest
     }
   }
 
+
   @Test
-  public void testInterface() throws Exception
-  {
+  public void testInterface() throws Exception {
     try {
       init("schema12.xml");
       SolrCore core = h.getCore();
 
       NamedList<String> args = new NamedList<String>();
-      args.add( QueryElevationComponent.FIELD_TYPE, "string" );
-      args.add( QueryElevationComponent.CONFIG_FILE, "elevate.xml" );
+      args.add(QueryElevationComponent.FIELD_TYPE, "string");
+      args.add(QueryElevationComponent.CONFIG_FILE, "elevate.xml");
 
       QueryElevationComponent comp = new QueryElevationComponent();
-      comp.init( args );
-      comp.inform( core );
+      comp.init(args);
+      comp.inform(core);
 
       SolrQueryRequest req = req();
       IndexReader reader = req.getSearcher().getIndexReader();
-      Map<String, ElevationObj> map = comp.getElevationMap( reader, core );
+      Map<String, ElevationObj> map = comp.getElevationMap(reader, core);
       req.close();
 
       // Make sure the boosts loaded properly
-      assertEquals( 4, map.size() );
-      assertEquals( 1, map.get( "XXXX" ).priority.size() );
-      assertEquals( 2, map.get( "YYYY" ).priority.size() );
-      assertEquals( 3, map.get( "ZZZZ" ).priority.size() );
-      assertEquals( null, map.get( "xxxx" ) );
-      assertEquals( null, map.get( "yyyy" ) );
-      assertEquals( null, map.get( "zzzz" ) );
+      assertEquals(5, map.size());
+      assertEquals(1, map.get("XXXX").priority.size());
+      assertEquals(2, map.get("YYYY").priority.size());
+      assertEquals(3, map.get("ZZZZ").priority.size());
+      assertEquals(null, map.get("xxxx"));
+      assertEquals(null, map.get("yyyy"));
+      assertEquals(null, map.get("zzzz"));
 
       // Now test the same thing with a lowercase filter: 'lowerfilt'
       args = new NamedList<String>();
-      args.add( QueryElevationComponent.FIELD_TYPE, "lowerfilt" );
-      args.add( QueryElevationComponent.CONFIG_FILE, "elevate.xml" );
+      args.add(QueryElevationComponent.FIELD_TYPE, "lowerfilt");
+      args.add(QueryElevationComponent.CONFIG_FILE, "elevate.xml");
 
       comp = new QueryElevationComponent();
-      comp.init( args );
-      comp.inform( core );
-      map = comp.getElevationMap( reader, core );
-      assertEquals( 4, map.size() );
-      assertEquals( null, map.get( "XXXX" ) );
-      assertEquals( null, map.get( "YYYY" ) );
-      assertEquals( null, map.get( "ZZZZ" ) );
-      assertEquals( 1, map.get( "xxxx" ).priority.size() );
-      assertEquals( 2, map.get( "yyyy" ).priority.size() );
-      assertEquals( 3, map.get( "zzzz" ).priority.size() );
+      comp.init(args);
+      comp.inform(core);
+      map = comp.getElevationMap(reader, core);
+      assertEquals(5, map.size());
+      assertEquals(null, map.get("XXXX"));
+      assertEquals(null, map.get("YYYY"));
+      assertEquals(null, map.get("ZZZZ"));
+      assertEquals(1, map.get("xxxx").priority.size());
+      assertEquals(2, map.get("yyyy").priority.size());
+      assertEquals(3, map.get("zzzz").priority.size());
 
-      assertEquals( "xxxx", comp.getAnalyzedQuery( "XXXX" ) );
-      assertEquals( "xxxxyyyy", comp.getAnalyzedQuery( "XXXX YYYY" ) );
+      assertEquals("xxxx", comp.getAnalyzedQuery("XXXX"));
+      assertEquals("xxxxyyyy", comp.getAnalyzedQuery("XXXX YYYY"));
 
-      assertQ("Make sure QEC handles null queries", req("qt","/elevate", "q.alt","*:*", "defType","dismax"),
+      assertQ("Make sure QEC handles null queries", req("qt", "/elevate", "q.alt", "*:*", "defType", "dismax"),
           "//*[@numFound='0']");
     } finally {
       delete();
@@ -163,157 +197,202 @@ public class QueryElevationComponentTest
   public void testMarker() throws Exception {
     try {
       init("schema12.xml");
-      assertU(adoc("id", "1", "title", "XXXX XXXX",           "str_s1", "a" ));
-      assertU(adoc("id", "2", "title", "YYYY",      "str_s1", "b" ));
-      assertU(adoc("id", "3", "title", "ZZZZ", "str_s1", "c" ));
-
-      assertU(adoc("id", "4", "title", "XXXX XXXX",                 "str_s1", "x" ));
-      assertU(adoc("id", "5", "title", "YYYY YYYY",         "str_s1", "y" ));
-      assertU(adoc("id", "6", "title", "XXXX XXXX", "str_s1", "z" ));
-      assertU(adoc("id", "7", "title", "AAAA", "str_s1", "a" ));
+      assertU(adoc("id", "1", "title", "XXXX XXXX", "str_s1", "a"));
+      assertU(adoc("id", "2", "title", "YYYY", "str_s1", "b"));
+      assertU(adoc("id", "3", "title", "ZZZZ", "str_s1", "c"));
+
+      assertU(adoc("id", "4", "title", "XXXX XXXX", "str_s1", "x"));
+      assertU(adoc("id", "5", "title", "YYYY YYYY", "str_s1", "y"));
+      assertU(adoc("id", "6", "title", "XXXX XXXX", "str_s1", "z"));
+      assertU(adoc("id", "7", "title", "AAAA", "str_s1", "a"));
       assertU(commit());
 
       assertQ("", req(CommonParams.Q, "XXXX", CommonParams.QT, "/elevate",
           CommonParams.FL, "id, score, [elevated]")
-              ,"//*[@numFound='3']"
-              ,"//result/doc[1]/str[@name='id'][.='1']"
-              ,"//result/doc[2]/str[@name='id'][.='4']"
-              ,"//result/doc[3]/str[@name='id'][.='6']",
-              "//result/doc[1]/bool[@name='[elevated]'][.='true']",
-              "//result/doc[2]/bool[@name='[elevated]'][.='false']",
-              "//result/doc[3]/bool[@name='[elevated]'][.='false']"
-              );
+          , "//*[@numFound='3']"
+          , "//result/doc[1]/str[@name='id'][.='1']"
+          , "//result/doc[2]/str[@name='id'][.='4']"
+          , "//result/doc[3]/str[@name='id'][.='6']",
+          "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[2]/bool[@name='[elevated]'][.='false']",
+          "//result/doc[3]/bool[@name='[elevated]'][.='false']"
+      );
 
       assertQ("", req(CommonParams.Q, "AAAA", CommonParams.QT, "/elevate",
           CommonParams.FL, "id, score, [elevated]")
               ,"//*[@numFound='1']"
               ,"//result/doc[1]/str[@name='id'][.='7']",
-              "//result/doc[1]/bool[@name='[elevated]'][.='false']"
+              "//result/doc[1]/bool[@name='[elevated]'][.='true']"
               );
 
       assertQ("", req(CommonParams.Q, "AAAA", CommonParams.QT, "/elevate",
           CommonParams.FL, "id, score, [elev]")
-              ,"//*[@numFound='1']"
-              ,"//result/doc[1]/str[@name='id'][.='7']",
-              "not(//result/doc[1]/bool[@name='[elevated]'][.='false'])",
-              "not(//result/doc[1]/bool[@name='[elev]'][.='false'])" // even though we asked for elev, there is no Transformer registered w/ that, so we shouldn't get a result
-              );
+          , "//*[@numFound='1']"
+          , "//result/doc[1]/str[@name='id'][.='7']",
+          "not(//result/doc[1]/bool[@name='[elevated]'][.='false'])",
+          "not(//result/doc[1]/bool[@name='[elev]'][.='false'])" // even though we asked for elev, there is no Transformer registered w/ that, so we shouldn't get a result
+      );
+    } finally {
+      delete();
+    }
+  }
+
+  @Test
+  public void testMarkExcludes() throws Exception {
+    try {
+      init("schema12.xml");
+      assertU(adoc("id", "1", "title", "XXXX XXXX", "str_s1", "a"));
+      assertU(adoc("id", "2", "title", "YYYY", "str_s1", "b"));
+      assertU(adoc("id", "3", "title", "ZZZZ", "str_s1", "c"));
+
+      assertU(adoc("id", "4", "title", "XXXX XXXX", "str_s1", "x"));
+      assertU(adoc("id", "5", "title", "YYYY YYYY", "str_s1", "y"));
+      assertU(adoc("id", "6", "title", "XXXX XXXX", "str_s1", "z"));
+      assertU(adoc("id", "7", "title", "AAAA", "str_s1", "a"));
+      assertU(commit());
+
+      assertQ("", req(CommonParams.Q, "XXXX XXXX", CommonParams.QT, "/elevate",
+          QueryElevationParams.MARK_EXCLUDES, "true",
+          CommonParams.FL, "id, score, [excluded]")
+          , "//*[@numFound='4']"
+          , "//result/doc[1]/str[@name='id'][.='5']"
+          , "//result/doc[2]/str[@name='id'][.='6']"
+          , "//result/doc[3]/str[@name='id'][.='1']"
+          , "//result/doc[4]/str[@name='id'][.='4']",
+          "//result/doc[1]/bool[@name='[excluded]'][.='false']",
+          "//result/doc[2]/bool[@name='[excluded]'][.='true']",
+          "//result/doc[3]/bool[@name='[excluded]'][.='false']",
+          "//result/doc[4]/bool[@name='[excluded]'][.='false']"
+      );
+      //ask for excluded as a field, but don't actually request the MARK_EXCLUDES
+      //thus, number 6 should not be returned, b/c it is excluded
+      assertQ("", req(CommonParams.Q, "XXXX XXXX", CommonParams.QT, "/elevate",
+          QueryElevationParams.MARK_EXCLUDES, "false",
+          CommonParams.FL, "id, score, [excluded]")
+          , "//*[@numFound='3']"
+          , "//result/doc[1]/str[@name='id'][.='5']"
+          , "//result/doc[2]/str[@name='id'][.='1']"
+          , "//result/doc[3]/str[@name='id'][.='4']",
+          "//result/doc[1]/bool[@name='[excluded]'][.='false']",
+          "//result/doc[2]/bool[@name='[excluded]'][.='false']",
+          "//result/doc[3]/bool[@name='[excluded]'][.='false']"
+      );
+
     } finally {
       delete();
     }
   }
 
   @Test
-  public void testSorting() throws Exception
-  {
+  public void testSorting() throws Exception {
     try {
       init("schema12.xml");
-      assertU(adoc("id", "a", "title", "ipod",           "str_s1", "a" ));
-      assertU(adoc("id", "b", "title", "ipod ipod",      "str_s1", "b" ));
-      assertU(adoc("id", "c", "title", "ipod ipod ipod", "str_s1", "c" ));
-
-      assertU(adoc("id", "x", "title", "boosted",                 "str_s1", "x" ));
-      assertU(adoc("id", "y", "title", "boosted boosted",         "str_s1", "y" ));
-      assertU(adoc("id", "z", "title", "boosted boosted boosted", "str_s1", "z" ));
+      assertU(adoc("id", "a", "title", "ipod", "str_s1", "a"));
+      assertU(adoc("id", "b", "title", "ipod ipod", "str_s1", "b"));
+      assertU(adoc("id", "c", "title", "ipod ipod ipod", "str_s1", "c"));
+
+      assertU(adoc("id", "x", "title", "boosted", "str_s1", "x"));
+      assertU(adoc("id", "y", "title", "boosted boosted", "str_s1", "y"));
+      assertU(adoc("id", "z", "title", "boosted boosted boosted", "str_s1", "z"));
       assertU(commit());
 
       String query = "title:ipod";
 
-      Map<String,String> args = new HashMap<String, String>();
-      args.put( CommonParams.Q, query );
-      args.put( CommonParams.QT, "/elevate" );
-      args.put( CommonParams.FL, "id,score" );
-      args.put( "indent", "true" );
+      Map<String, String> args = new HashMap<String, String>();
+      args.put(CommonParams.Q, query);
+      args.put(CommonParams.QT, "/elevate");
+      args.put(CommonParams.FL, "id,score");
+      args.put("indent", "true");
       //args.put( CommonParams.FL, "id,title,score" );
-      SolrQueryRequest req = new LocalSolrQueryRequest( h.getCore(), new MapSolrParams( args) );
+      SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), new MapSolrParams(args));
       IndexReader reader = req.getSearcher().getIndexReader();
-      QueryElevationComponent booster = (QueryElevationComponent)req.getCore().getSearchComponent( "elevate" );
+      QueryElevationComponent booster = (QueryElevationComponent) req.getCore().getSearchComponent("elevate");
 
       assertQ("Make sure standard sort works as expected", req
-              ,"//*[@numFound='3']"
-              ,"//result/doc[1]/str[@name='id'][.='a']"
-              ,"//result/doc[2]/str[@name='id'][.='b']"
-              ,"//result/doc[3]/str[@name='id'][.='c']"
-              );
+          , "//*[@numFound='3']"
+          , "//result/doc[1]/str[@name='id'][.='a']"
+          , "//result/doc[2]/str[@name='id'][.='b']"
+          , "//result/doc[3]/str[@name='id'][.='c']"
+      );
 
       // Explicitly set what gets boosted
       booster.elevationCache.clear();
-      booster.setTopQueryResults( reader, query, new String[] { "x", "y", "z" }, null );
+      booster.setTopQueryResults(reader, query, new String[]{"x", "y", "z"}, null);
 
 
       assertQ("All six should make it", req
-              ,"//*[@numFound='6']"
-              ,"//result/doc[1]/str[@name='id'][.='x']"
-              ,"//result/doc[2]/str[@name='id'][.='y']"
-              ,"//result/doc[3]/str[@name='id'][.='z']"
-              ,"//result/doc[4]/str[@name='id'][.='a']"
-              ,"//result/doc[5]/str[@name='id'][.='b']"
-              ,"//result/doc[6]/str[@name='id'][.='c']"
-              );
+          , "//*[@numFound='6']"
+          , "//result/doc[1]/str[@name='id'][.='x']"
+          , "//result/doc[2]/str[@name='id'][.='y']"
+          , "//result/doc[3]/str[@name='id'][.='z']"
+          , "//result/doc[4]/str[@name='id'][.='a']"
+          , "//result/doc[5]/str[@name='id'][.='b']"
+          , "//result/doc[6]/str[@name='id'][.='c']"
+      );
 
       booster.elevationCache.clear();
 
       // now switch the order:
-      booster.setTopQueryResults( reader, query, new String[] { "a", "x" }, null );
+      booster.setTopQueryResults(reader, query, new String[]{"a", "x"}, null);
       assertQ("All four should make it", req
-              ,"//*[@numFound='4']"
-              ,"//result/doc[1]/str[@name='id'][.='a']"
-              ,"//result/doc[2]/str[@name='id'][.='x']"
-              ,"//result/doc[3]/str[@name='id'][.='b']"
-              ,"//result/doc[4]/str[@name='id'][.='c']"
-              );
+          , "//*[@numFound='4']"
+          , "//result/doc[1]/str[@name='id'][.='a']"
+          , "//result/doc[2]/str[@name='id'][.='x']"
+          , "//result/doc[3]/str[@name='id'][.='b']"
+          , "//result/doc[4]/str[@name='id'][.='c']"
+      );
 
       // Test reverse sort
-      args.put( CommonParams.SORT, "score asc" );
+      args.put(CommonParams.SORT, "score asc");
       assertQ("All four should make it", req
-          ,"//*[@numFound='4']"
-          ,"//result/doc[4]/str[@name='id'][.='a']"
-          ,"//result/doc[3]/str[@name='id'][.='x']"
-          ,"//result/doc[2]/str[@name='id'][.='b']"
-          ,"//result/doc[1]/str[@name='id'][.='c']"
-          );
+          , "//*[@numFound='4']"
+          , "//result/doc[4]/str[@name='id'][.='a']"
+          , "//result/doc[3]/str[@name='id'][.='x']"
+          , "//result/doc[2]/str[@name='id'][.='b']"
+          , "//result/doc[1]/str[@name='id'][.='c']"
+      );
 
       // Try normal sort by 'id'
       // default 'forceBoost' should be false
-      assertEquals( false, booster.forceElevation );
-      args.put( CommonParams.SORT, "str_s1 asc" );
-      assertQ( null, req
-          ,"//*[@numFound='4']"
-          ,"//result/doc[1]/str[@name='id'][.='a']"
-          ,"//result/doc[2]/str[@name='id'][.='b']"
-          ,"//result/doc[3]/str[@name='id'][.='c']"
-          ,"//result/doc[4]/str[@name='id'][.='x']"
-          );
+      assertEquals(false, booster.forceElevation);
+      args.put(CommonParams.SORT, "str_s1 asc");
+      assertQ(null, req
+          , "//*[@numFound='4']"
+          , "//result/doc[1]/str[@name='id'][.='a']"
+          , "//result/doc[2]/str[@name='id'][.='b']"
+          , "//result/doc[3]/str[@name='id'][.='c']"
+          , "//result/doc[4]/str[@name='id'][.='x']"
+      );
 
       booster.forceElevation = true;
-      assertQ( null, req
-          ,"//*[@numFound='4']"
-          ,"//result/doc[1]/str[@name='id'][.='a']"
-          ,"//result/doc[2]/str[@name='id'][.='x']"
-          ,"//result/doc[3]/str[@name='id'][.='b']"
-          ,"//result/doc[4]/str[@name='id'][.='c']"
-          );
+      assertQ(null, req
+          , "//*[@numFound='4']"
+          , "//result/doc[1]/str[@name='id'][.='a']"
+          , "//result/doc[2]/str[@name='id'][.='x']"
+          , "//result/doc[3]/str[@name='id'][.='b']"
+          , "//result/doc[4]/str[@name='id'][.='c']"
+      );
 
       //Test exclusive (not to be confused with exclusion)
       args.put(QueryElevationParams.EXCLUSIVE, "true");
-      booster.setTopQueryResults( reader, query, new String[] { "x", "a" },  new String[] {} );
-      assertQ( null, req
-          ,"//*[@numFound='2']"
-          ,"//result/doc[1]/str[@name='id'][.='x']"
-          ,"//result/doc[2]/str[@name='id'][.='a']"
-          );
+      booster.setTopQueryResults(reader, query, new String[]{"x", "a"}, new String[]{});
+      assertQ(null, req
+          , "//*[@numFound='2']"
+          , "//result/doc[1]/str[@name='id'][.='x']"
+          , "//result/doc[2]/str[@name='id'][.='a']"
+      );
 
       // Test exclusion
       booster.elevationCache.clear();
-      args.remove( CommonParams.SORT );
-      args.remove( QueryElevationParams.EXCLUSIVE);
-      booster.setTopQueryResults( reader, query, new String[] { "x" },  new String[] { "a" } );
-      assertQ( null, req
-          ,"//*[@numFound='3']"
-          ,"//result/doc[1]/str[@name='id'][.='x']"
-          ,"//result/doc[2]/str[@name='id'][.='b']"
-          ,"//result/doc[3]/str[@name='id'][.='c']"
-          );
+      args.remove(CommonParams.SORT);
+      args.remove(QueryElevationParams.EXCLUSIVE);
+      booster.setTopQueryResults(reader, query, new String[]{"x"}, new String[]{"a"});
+      assertQ(null, req
+          , "//*[@numFound='3']"
+          , "//result/doc[1]/str[@name='id'][.='x']"
+          , "//result/doc[2]/str[@name='id'][.='b']"
+          , "//result/doc[3]/str[@name='id'][.='c']"
+      );
 
 
       req.close();
@@ -321,57 +400,55 @@ public class QueryElevationComponentTest
       delete();
     }
   }
-  
+
   // write a test file to boost some docs
-  private void writeFile( File file, String query, String ... ids ) throws Exception
-  {
-    PrintWriter out = new PrintWriter( new FileOutputStream( file ) ); 
-    out.println( "<?xml version=\"1.0\" encoding=\"UTF-8\" ?>" );
-    out.println( "<elevate>" );
-    out.println( "<query text=\""+query+"\">" );
-    for( String id : ids ) {
-      out.println( " <doc id=\""+id+"\"/>" );
+  private void writeFile(File file, String query, String... ids) throws Exception {
+    PrintWriter out = new PrintWriter(new FileOutputStream(file));
+    out.println("<?xml version=\"1.0\" encoding=\"UTF-8\" ?>");
+    out.println("<elevate>");
+    out.println("<query text=\"" + query + "\">");
+    for (String id : ids) {
+      out.println(" <doc id=\"" + id + "\"/>");
     }
-    out.println( "</query>" );
-    out.println( "</elevate>" );
+    out.println("</query>");
+    out.println("</elevate>");
     out.flush();
     out.close();
-    
-    log.info( "OUT:"+file.getAbsolutePath() );
+
+    log.info("OUT:" + file.getAbsolutePath());
   }
 
   @Test
-  public void testElevationReloading() throws Exception
-  {
+  public void testElevationReloading() throws Exception {
     try {
       init("schema12.xml");
       String testfile = "data-elevation.xml";
-      File f = new File( h.getCore().getDataDir(), testfile );
-      writeFile( f, "aaa", "A" );
+      File f = new File(h.getCore().getDataDir(), testfile);
+      writeFile(f, "aaa", "A");
 
-      QueryElevationComponent comp = (QueryElevationComponent)h.getCore().getSearchComponent("elevate");
+      QueryElevationComponent comp = (QueryElevationComponent) h.getCore().getSearchComponent("elevate");
       NamedList<String> args = new NamedList<String>();
-      args.add( QueryElevationComponent.CONFIG_FILE, testfile );
-      comp.init( args );
-      comp.inform( h.getCore() );
+      args.add(QueryElevationComponent.CONFIG_FILE, testfile);
+      comp.init(args);
+      comp.inform(h.getCore());
 
       SolrQueryRequest req = req();
       IndexReader reader = req.getSearcher().getIndexReader();
       Map<String, ElevationObj> map = comp.getElevationMap(reader, h.getCore());
-      assertTrue( map.get( "aaa" ).priority.containsKey( new BytesRef("A") ) );
-      assertNull( map.get( "bbb" ) );
+      assertTrue(map.get("aaa").priority.containsKey(new BytesRef("A")));
+      assertNull(map.get("bbb"));
       req.close();
 
       // now change the file
-      writeFile( f, "bbb", "B" );
+      writeFile(f, "bbb", "B");
       assertU(adoc("id", "10000")); // will get same reader if no index change
       assertU(commit());
 
       req = req();
       reader = req.getSearcher().getIndexReader();
       map = comp.getElevationMap(reader, h.getCore());
-      assertNull( map.get( "aaa" ) );
-      assertTrue( map.get( "bbb" ).priority.containsKey( new BytesRef("B") ) );
+      assertNull(map.get("aaa"));
+      assertTrue(map.get("bbb").priority.containsKey(new BytesRef("B")));
       req.close();
     } finally {
       delete();

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java Wed Dec 28 21:03:23 2011
@@ -35,8 +35,10 @@ public class BadIndexSchemaTest extends 
     try {
       initCore( "solrconfig.xml", schema );
     } catch (SolrException e) {
-      // short circut out if we found what we expected
+      // short circuit out if we found what we expected
       if (-1 != e.getMessage().indexOf(errString)) return;
+      // Test the cause too in case the expected error is wrapped
+      if (-1 != e.getCause().getMessage().indexOf(errString)) return;
 
       // otherwise, rethrow it, possibly completley unrelated
       throw new SolrException
@@ -44,6 +46,7 @@ public class BadIndexSchemaTest extends 
          "Unexpected error, expected error matching: " + errString, e);
     } finally {
       SolrConfig.severeErrors.clear();
+      deleteCore();
     }
     fail("Did not encounter any exception from: " + schema);
   }
@@ -75,4 +78,10 @@ public class BadIndexSchemaTest extends 
   public void testSevereErrorsForUnexpectedAnalyzer() throws Exception {
     doTest("bad-schema-nontext-analyzer.xml", "StrField (bad_type)");
   }
+
+  @Test
+  public void testBadExternalFileField() throws Exception {
+    doTest("bad-schema-external-filefield.xml",
+        "Only float and pfloat");
+  }
 }

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/schema/TestCollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/schema/TestCollationField.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/schema/TestCollationField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/schema/TestCollationField.java Wed Dec 28 21:03:23 2011
@@ -25,7 +25,7 @@ import java.util.Locale;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.codecs.Codec;
 import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java Wed Dec 28 21:03:23 2011
@@ -17,8 +17,8 @@
 
 package org.apache.solr.search.function;
 
+import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.TFIDFSimilarity;

Modified: lucene/dev/branches/solrcloud/solr/example/solr/conf/elevate.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/example/solr/conf/elevate.xml?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/example/solr/conf/elevate.xml (original)
+++ lucene/dev/branches/solrcloud/solr/example/solr/conf/elevate.xml Wed Dec 28 21:03:23 2011
@@ -19,8 +19,10 @@
 <!-- If this file is found in the config directory, it will only be
      loaded once at startup.  If it is found in Solr's data
      directory, it will be re-loaded every commit.
--->
 
+   See http://wiki.apache.org/solr/QueryElevationComponent for more info
+
+-->
 <elevate>
  <query text="foo bar">
   <doc id="1" />

Modified: lucene/dev/branches/solrcloud/solr/lib/commons-codec-NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/lib/commons-codec-NOTICE.txt?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/lib/commons-codec-NOTICE.txt (original)
+++ lucene/dev/branches/solrcloud/solr/lib/commons-codec-NOTICE.txt Wed Dec 28 21:03:23 2011
@@ -1,5 +1,5 @@
 Apache Commons Codec
-Copyright 2002-2009 The Apache Software Foundation
+Copyright 2002-2011 The Apache Software Foundation
 
 This product includes software developed by
 The Apache Software Foundation (http://www.apache.org/).

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/params/QueryElevationParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/params/QueryElevationParams.java?rev=1225335&r1=1225334&r2=1225335&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/params/QueryElevationParams.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/params/QueryElevationParams.java Wed Dec 28 21:03:23 2011
@@ -34,4 +34,18 @@ public interface QueryElevationParams {
    * See http://wiki.apache.org/solr/DocTransformers
    */
   String EDITORIAL_MARKER_FIELD_NAME = "editorialMarkerFieldName";
+  /**
+   * The name of the field that excluded editorial results will be written out as when using the QueryElevationComponent, which
+   * automatically configures the EditorialMarkerFactory.  The default name is "excluded".  This is only used
+   * when {@link #MARK_EXCLUDES} is set to true at query time.
+   * <br/>
+   * See http://wiki.apache.org/solr/DocTransformers
+   */
+  String EXCLUDE_MARKER_FIELD_NAME = "excludeMarkerFieldName";
+
+  /**
+   * Instead of removing excluded items from the results, passing in this parameter allows you to get back the excluded items, but to mark them
+   * as excluded.
+   */
+  String MARK_EXCLUDES = "markExcludes";
 }