You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/05/07 13:21:14 UTC

svn commit: r1479862 [30/38] - in /lucene/dev/branches/lucene4258: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/solr/ dev-tools/maven/solr/core/src/java/ dev-tools/maven/solr/solrj/src/java/ dev-t...

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java Tue May  7 11:20:55 2013
@@ -69,6 +69,30 @@ public class DefaultSolrHighlighter exte
     this.solrCore = solrCore;
   }
 
+  // Thread safe registry
+  protected final Map<String,SolrFormatter> formatters =
+    new HashMap<String, SolrFormatter>();
+
+  // Thread safe registry
+  protected final Map<String,SolrEncoder> encoders =
+    new HashMap<String, SolrEncoder>();
+
+  // Thread safe registry
+  protected final Map<String,SolrFragmenter> fragmenters =
+    new HashMap<String, SolrFragmenter>() ;
+
+  // Thread safe registry
+  protected final Map<String, SolrFragListBuilder> fragListBuilders =
+    new HashMap<String, SolrFragListBuilder>() ;
+
+  // Thread safe registry
+  protected final Map<String, SolrFragmentsBuilder> fragmentsBuilders =
+    new HashMap<String, SolrFragmentsBuilder>() ;
+
+  // Thread safe registry
+  protected final Map<String, SolrBoundaryScanner> boundaryScanners =
+    new HashMap<String, SolrBoundaryScanner>() ;
+
   @Override
   public void init(PluginInfo info) {
     formatters.clear();
@@ -429,21 +453,15 @@ public class DefaultSolrHighlighter exte
     )) return;
     // END: Hack
     
-    SolrParams params = req.getParams(); 
-    StorableField[] docFields = doc.getFields(fieldName);
-    List<String> listFields = new ArrayList<String>();
-    for (StorableField field : docFields) {
-      listFields.add(field.stringValue());
-    }
+    SolrParams params = req.getParams();
 
     // preserve order of values in a multiValued list
     boolean preserveMulti = params.getFieldBool(fieldName, HighlightParams.PRESERVE_MULTI, false);
 
-    String[] docTexts = (String[]) listFields.toArray(new String[listFields.size()]);
-   
-    // according to Document javadoc, doc.getValues() never returns null. check empty instead of null
-    if (docTexts.length == 0) return;
-    
+    List<StorableField> allFields = doc.getFields();
+    if (allFields != null && allFields.size() == 0) return; // No explicit contract that getFields returns != null,
+                                                            // although currently it can't.
+
     TokenStream tstream = null;
     int numFragments = getMaxSnippets(fieldName, params);
     boolean mergeContiguousFragments = isMergeContiguousFragments(fieldName, params);
@@ -456,15 +474,25 @@ public class DefaultSolrHighlighter exte
     if (tvStream != null) {
       tots = new TermOffsetsTokenStream(tvStream);
     }
+    int mvToExamine = Integer.parseInt(req.getParams().get(HighlightParams.MAX_MULTIVALUED_TO_EXAMINE,
+        Integer.toString(Integer.MAX_VALUE)));
+    int mvToMatch = Integer.parseInt(req.getParams().get(HighlightParams.MAX_MULTIVALUED_TO_MATCH,
+        Integer.toString(Integer.MAX_VALUE)));
+
+    for (StorableField thisField : allFields) {
+      if (mvToExamine <= 0 || mvToMatch <= 0) break;
+
+      if (! thisField.name().equals(fieldName)) continue; // Is there a better way to do this?
 
-    for (int j = 0; j < docTexts.length; j++) {
+      --mvToExamine;
+      String thisText = thisField.stringValue();
       if( tots != null ) {
         // if we're using TermOffsets optimization, then get the next
         // field value's TokenStream (i.e. get field j's TokenStream) from tots:
-        tstream = tots.getMultiValuedTokenStream( docTexts[j].length() );
+        tstream = tots.getMultiValuedTokenStream( thisText.length() );
       } else {
         // fall back to analyzer
-        tstream = createAnalyzerTStream(schema, fieldName, docTexts[j]);
+        tstream = createAnalyzerTStream(schema, fieldName, thisText);
       }
       
       int maxCharsToAnalyze = params.getFieldInt(fieldName,
@@ -491,21 +519,23 @@ public class DefaultSolrHighlighter exte
       }
       
       if (maxCharsToAnalyze < 0) {
-        highlighter.setMaxDocCharsToAnalyze(docTexts[j].length());
+        highlighter.setMaxDocCharsToAnalyze(thisText.length());
       } else {
         highlighter.setMaxDocCharsToAnalyze(maxCharsToAnalyze);
       }
 
       try {
-        TextFragment[] bestTextFragments = highlighter.getBestTextFragments(tstream, docTexts[j], mergeContiguousFragments, numFragments);
+        TextFragment[] bestTextFragments = highlighter.getBestTextFragments(tstream, thisText, mergeContiguousFragments, numFragments);
         for (int k = 0; k < bestTextFragments.length; k++) {
           if (preserveMulti) {
             if (bestTextFragments[k] != null) {
               frags.add(bestTextFragments[k]);
+              --mvToMatch;
             }
           } else {
             if ((bestTextFragments[k] != null) && (bestTextFragments[k].getScore() > 0)) {
               frags.add(bestTextFragments[k]);
+              --mvToMatch;
             }
           }
         }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java Tue May  7 11:20:55 2013
@@ -26,10 +26,12 @@ import java.util.Set;
 
 import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.postingshighlight.DefaultPassageFormatter;
 import org.apache.lucene.search.postingshighlight.Passage;
 import org.apache.lucene.search.postingshighlight.PassageFormatter;
 import org.apache.lucene.search.postingshighlight.PassageScorer;
 import org.apache.lucene.search.postingshighlight.PostingsHighlighter;
+import org.apache.lucene.search.postingshighlight.WholeBreakIterator;
 import org.apache.solr.common.params.HighlightParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
@@ -49,99 +51,67 @@ import org.apache.solr.util.plugin.Plugi
  * <p>
  * Example configuration:
  * <pre class="prettyprint">
+ *   &lt;requestHandler name="standard" class="solr.StandardRequestHandler"&gt;
+ *     &lt;lst name="defaults"&gt;
+ *       &lt;int name="hl.snippets"&gt;1&lt;/int&gt;
+ *       &lt;str name="hl.tag.pre"&gt;&amp;lt;em&amp;gt;&lt;/str&gt;
+ *       &lt;str name="hl.tag.post"&gt;&amp;lt;/em&amp;gt;&lt;/str&gt;
+ *       &lt;str name="hl.tag.ellipsis"&gt;... &lt;/str&gt;
+ *       &lt;bool name="hl.defaultSummary"&gt;true&lt;/bool&gt;
+ *       &lt;str name="hl.encoder"&gt;simple&lt;/str&gt;
+ *       &lt;float name="hl.score.k1"&gt;1.2&lt;/float&gt;
+ *       &lt;float name="hl.score.b"&gt;0.75&lt;/float&gt;
+ *       &lt;float name="hl.score.pivot"&gt;87&lt;/float&gt;
+ *       &lt;str name="hl.bs.language"&gt;&lt;/str&gt;
+ *       &lt;str name="hl.bs.country"&gt;&lt;/str&gt;
+ *       &lt;str name="hl.bs.variant"&gt;&lt;/str&gt;
+ *       &lt;str name="hl.bs.type"&gt;SENTENCE&lt;/str&gt;
+ *       &lt;int name="hl.maxAnalyzedChars"&gt;10000&lt;/int&gt;
+ *     &lt;/lst&gt;
+ *   &lt;/requestHandler&gt;
+ * </pre>
+ * ...
+ * <pre class="prettyprint">
  *   &lt;searchComponent class="solr.HighlightComponent" name="highlight"&gt;
- *     &lt;highlighting class="org.apache.solr.highlight.PostingsSolrHighlighter"
- *                      preTag="&amp;lt;em&amp;gt;"
- *                      postTag="&amp;lt;/em&amp;gt;"
- *                      ellipsis="... "
- *                      k1="1.2"
- *                      b="0.75"
- *                      pivot="87"
- *                      maxLength=10000
- *                      summarizeEmpty=true/&gt;
+ *     &lt;highlighting class="org.apache.solr.highlight.PostingsSolrHighlighter"/&gt;
  *   &lt;/searchComponent&gt;
  * </pre>
  * <p>
  * Notes:
  *  <ul>
  *    <li>fields to highlight must be configured with storeOffsetsWithPositions="true"
- *    <li>hl.fl specifies the field list.
- *    <li>hl.snippets specifies how many underlying sentence fragments form the resulting snippet.
+ *    <li>hl.q (string) can specify the query
+ *    <li>hl.fl (string) specifies the field list.
+ *    <li>hl.snippets (int) specifies how many underlying passages form the resulting snippet.
+ *    <li>hl.tag.pre (string) specifies text which appears before a highlighted term.
+ *    <li>hl.tag.post (string) specifies text which appears after a highlighted term.
+ *    <li>hl.tag.ellipsis (string) specifies text which joins non-adjacent passages.
+ *    <li>hl.defaultSummary (bool) specifies if a field should have a default summary.
+ *    <li>hl.encoder (string) can be 'html' (html escapes content) or 'simple' (no escaping).
+ *    <li>hl.score.k1 (float) specifies bm25 scoring parameter 'k1'
+ *    <li>hl.score.b (float) specifies bm25 scoring parameter 'b'
+ *    <li>hl.score.pivot (float) specifies bm25 scoring parameter 'avgdl'
+ *    <li>hl.bs.type (string) specifies how to divide text into passages: [SENTENCE, LINE, WORD, CHAR, WHOLE]
+ *    <li>hl.bs.language (string) specifies language code for BreakIterator. default is empty string (root locale)
+ *    <li>hl.bs.country (string) specifies country code for BreakIterator. default is empty string (root locale)
+ *    <li>hl.bs.variant (string) specifies country code for BreakIterator. default is empty string (root locale)
+ *    <li>hl.maxAnalyzedChars specifies how many characters at most will be processed in a document.
+ *        NOTE: currently hl.maxAnalyzedChars cannot yet be specified per-field
  *  </ul>
  *  
  * @lucene.experimental 
  */
 public class PostingsSolrHighlighter extends SolrHighlighter implements PluginInfoInitialized {
-  protected PostingsHighlighter highlighter;
 
   @Override
   public void initalize(SolrConfig config) {}
   
   @Override
-  public void init(PluginInfo info) {
-    Map<String,String> attributes = info.attributes;
-    BreakIterator breakIterator = BreakIterator.getSentenceInstance(Locale.ROOT);
-    
-    // scorer parameters: k1/b/pivot
-    String k1 = attributes.get("k1");
-    if (k1 == null) {
-      k1 = "1.2";
-    }
-    
-    String b = attributes.get("b");
-    if (b == null) {
-      b = "0.75";
-    }
-    
-    String pivot = attributes.get("pivot");
-    if (pivot == null) {
-      pivot = "87";
-    }
-    PassageScorer scorer = new PassageScorer(Float.parseFloat(k1), Float.parseFloat(b), Float.parseFloat(pivot));
-    
-    // formatter parameters: preTag/postTag/ellipsis
-    String preTag = attributes.get("preTag");
-    if (preTag == null) {
-      preTag = "<em>";
-    }
-    String postTag = attributes.get("postTag");
-    if (postTag == null) {
-      postTag = "</em>";
-    }
-    String ellipsis = attributes.get("ellipsis");
-    if (ellipsis == null) {
-      ellipsis = "... ";
-    }
-    PassageFormatter formatter = new PassageFormatter(preTag, postTag, ellipsis);
-
-    String summarizeEmpty = attributes.get("summarizeEmpty");
-    final boolean summarizeEmptyBoolean;
-    if (summarizeEmpty == null) {
-      summarizeEmptyBoolean = true;
-    } else {
-      summarizeEmptyBoolean = Boolean.parseBoolean(summarizeEmpty);
-    }
-
-    // maximum content size to process
-    int maxLength = PostingsHighlighter.DEFAULT_MAX_LENGTH;
-    if (attributes.containsKey("maxLength")) {
-      maxLength = Integer.parseInt(attributes.get("maxLength"));
-    }
-    highlighter = new PostingsHighlighter(maxLength, breakIterator, scorer, formatter) {
-        @Override
-        protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) {
-          if (summarizeEmptyBoolean) {
-            return super.getEmptyHighlight(fieldName, bi, maxPassages);
-          } else {
-            return new Passage[0];
-          }
-        }
-      };
-  }
+  public void init(PluginInfo info) {}
 
   @Override
   public NamedList<Object> doHighlighting(DocList docs, Query query, SolrQueryRequest req, String[] defaultFields) throws IOException {
-    SolrParams params = req.getParams(); 
+    final SolrParams params = req.getParams(); 
     
     // if highlighting isnt enabled, then why call doHighlighting?
     if (isHighlightingEnabled(params)) {
@@ -152,10 +122,54 @@ public class PostingsSolrHighlighter ext
       String[] keys = getUniqueKeys(searcher, docIDs);
       
       // query-time parameters
+      int maxLength = params.getInt(HighlightParams.MAX_CHARS, PostingsHighlighter.DEFAULT_MAX_LENGTH);
       String[] fieldNames = getHighlightFields(query, req, defaultFields);
-      int numSnippets = params.getInt(HighlightParams.SNIPPETS, 1);
       
-      Map<String,String[]> snippets = highlighter.highlightFields(fieldNames, query, searcher, docIDs, numSnippets);
+      int maxPassages[] = new int[fieldNames.length];
+      for (int i = 0; i < fieldNames.length; i++) {
+        maxPassages[i] = params.getFieldInt(fieldNames[i], HighlightParams.SNIPPETS, 1);
+      }
+      
+      PostingsHighlighter highlighter = new PostingsHighlighter(maxLength) {
+        @Override
+        protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) {
+          boolean defaultSummary = params.getFieldBool(fieldName, HighlightParams.DEFAULT_SUMMARY, true);
+          if (defaultSummary) {
+            return super.getEmptyHighlight(fieldName, bi, maxPassages);
+          } else {
+            return new Passage[0];
+          }
+        }
+
+        @Override
+        protected PassageFormatter getFormatter(String fieldName) {
+          String preTag = params.getFieldParam(fieldName, HighlightParams.TAG_PRE, "<em>");
+          String postTag = params.getFieldParam(fieldName, HighlightParams.TAG_POST, "</em>");
+          String ellipsis = params.getFieldParam(fieldName, HighlightParams.TAG_ELLIPSIS, "... ");
+          String encoder = params.getFieldParam(fieldName, HighlightParams.ENCODER, "simple");
+          return new DefaultPassageFormatter(preTag, postTag, ellipsis, "html".equals(encoder));
+        }
+
+        @Override
+        protected PassageScorer getScorer(String fieldName) {
+          float k1 = params.getFieldFloat(fieldName, HighlightParams.SCORE_K1, 1.2f);
+          float b = params.getFieldFloat(fieldName, HighlightParams.SCORE_B, 0.75f);
+          float pivot = params.getFieldFloat(fieldName, HighlightParams.SCORE_PIVOT, 87f);
+          return new PassageScorer(k1, b, pivot);
+        }
+
+        @Override
+        protected BreakIterator getBreakIterator(String field) {
+          String language = params.getFieldParam(field, HighlightParams.BS_LANGUAGE);
+          String country = params.getFieldParam(field, HighlightParams.BS_COUNTRY);
+          String variant = params.getFieldParam(field, HighlightParams.BS_VARIANT);
+          Locale locale = parseLocale(language, country, variant);
+          String type = params.getFieldParam(field, HighlightParams.BS_TYPE);
+          return parseBreakIterator(type, locale);
+        }
+      };
+      
+      Map<String,String[]> snippets = highlighter.highlightFields(fieldNames, query, searcher, docIDs, maxPassages);
       return encodeSnippets(keys, fieldNames, snippets);
     } else {
       return null;
@@ -222,4 +236,36 @@ public class PostingsSolrHighlighter ext
       return new String[docIDs.length];
     }
   }
+  
+  /** parse a break iterator type for the specified locale */
+  protected BreakIterator parseBreakIterator(String type, Locale locale) {
+    if (type == null || "SENTENCE".equals(type)) {
+      return BreakIterator.getSentenceInstance(locale);
+    } else if ("LINE".equals(type)) {
+      return BreakIterator.getLineInstance(locale);
+    } else if ("WORD".equals(type)) {
+      return BreakIterator.getWordInstance(locale);
+    } else if ("CHARACTER".equals(type)) {
+      return BreakIterator.getCharacterInstance(locale);
+    } else if ("WHOLE".equals(type)) {
+      return new WholeBreakIterator();
+    } else {
+      throw new IllegalArgumentException("Unknown " + HighlightParams.BS_TYPE + ": " + type);
+    }
+  }
+  
+  /** parse a locale from a language+country+variant spec */
+  protected Locale parseLocale(String language, String country, String variant) {
+    if (language == null && country == null && variant == null) {
+      return Locale.ROOT;
+    } else if (language != null && country == null && variant != null) {
+      throw new IllegalArgumentException("To specify variant, country is required");
+    } else if (language != null && country != null && variant != null) {
+      return new Locale(language, country, variant);
+    } else if (language != null && country != null) {
+      return new Locale(language, country);
+    } else { 
+      return new Locale(language);
+    }
+  }
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java Tue May  7 11:20:55 2013
@@ -19,9 +19,7 @@ package org.apache.solr.highlight;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,34 +36,9 @@ public abstract class SolrHighlighter
 {
   public static Logger log = LoggerFactory.getLogger(SolrHighlighter.class);
 
-  // Thread safe registry
-  protected final Map<String,SolrFormatter> formatters =
-    new HashMap<String, SolrFormatter>();
-
-  // Thread safe registry
-  protected final Map<String,SolrEncoder> encoders =
-    new HashMap<String, SolrEncoder>();
-
-  // Thread safe registry
-  protected final Map<String,SolrFragmenter> fragmenters =
-    new HashMap<String, SolrFragmenter>() ;
-
-  // Thread safe registry
-  protected final Map<String, SolrFragListBuilder> fragListBuilders =
-    new HashMap<String, SolrFragListBuilder>() ;
-
-  // Thread safe registry
-  protected final Map<String, SolrFragmentsBuilder> fragmentsBuilders =
-    new HashMap<String, SolrFragmentsBuilder>() ;
-
-  // Thread safe registry
-  protected final Map<String, SolrBoundaryScanner> boundaryScanners =
-    new HashMap<String, SolrBoundaryScanner>() ;
-
   @Deprecated
   public abstract void initalize( SolrConfig config );
 
-
   /**
    * Check whether Highlighting is enabled for this request.
    * @param params The params controlling Highlighting

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/logging/LogWatcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/logging/LogWatcher.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/logging/LogWatcher.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/logging/LogWatcher.java Tue May  7 11:20:55 2013
@@ -17,21 +17,29 @@
 
 package org.apache.solr.logging;
 
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.core.ConfigSolr;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.logging.jul.JulWatcher;
+import org.apache.solr.logging.log4j.Log4jWatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.impl.StaticLoggerBinder;
+
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.core.CoreContainer;
-
 /**
  * A Class to monitor Logging events and hold N events in memory
  * 
  * This is abstract so we can support both JUL and Log4j (and other logging platforms)
  */
 public abstract class LogWatcher<E> {
+
+  private static final Logger log = LoggerFactory.getLogger(LogWatcher.class);
   
   protected CircularList<E> history;
   protected long last = -1;
@@ -98,10 +106,82 @@ public abstract class LogWatcher<E> {
   public abstract long getTimestamp(E event);
   public abstract SolrDocument toSolrDocument(E event);
   
-  public abstract void registerListener(ListenerConfig cfg, CoreContainer container);
+  public abstract void registerListener(ListenerConfig cfg);
 
   public void reset() {
     history.clear();
     last = -1;
   }
+
+  /**
+   * Create and register a LogWatcher.
+   *
+   * JUL and Log4j watchers are supported out-of-the-box.  You can register your own
+   * LogWatcher implementation via the plugins architecture
+   *
+   * @param config the CoreContainer's config, with logging configuration details
+   * @param loader a SolrResourceLoader, to be used to load plugin LogWatcher implementations.
+   *               Can be null if
+   *
+   * @return a LogWatcher configured for the container's logging framework
+   */
+  public static LogWatcher newRegisteredLogWatcher(ConfigSolr config, SolrResourceLoader loader) {
+
+    if (!config.getBool(ConfigSolr.CfgProp.SOLR_LOGGING_ENABLED, true))
+      return null;
+
+    LogWatcher logWatcher = createWatcher(config, loader);
+
+    if (logWatcher != null) {
+      ListenerConfig v = new ListenerConfig();
+      v.size = config.getInt(ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_SIZE, 50);
+      v.threshold = config.get(ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, null);
+      if (v.size > 0) {
+        log.info("Registering Log Listener");
+        logWatcher.registerListener(v);
+      }
+    }
+
+    return logWatcher;
+  }
+
+  private static LogWatcher createWatcher(ConfigSolr config, SolrResourceLoader loader) {
+
+    String fname = config.get(ConfigSolr.CfgProp.SOLR_LOGGING_CLASS, null);
+    String slf4jImpl;
+
+    try {
+      slf4jImpl = StaticLoggerBinder.getSingleton().getLoggerFactoryClassStr();
+      if (fname == null) {
+        if (slf4jImpl.indexOf("Log4j") > 0) {
+          fname = "Log4j";
+        } else if (slf4jImpl.indexOf("JDK") > 0) {
+          fname = "JUL";
+        }
+      }
+    }
+    catch (Throwable e) {
+      log.warn("Unable to read SLF4J version.  LogWatcher will be disabled: " + e);
+      return null;
+    }
+
+    if (fname == null) {
+      log.info("No LogWatcher configured");
+      return null;
+    }
+
+    if ("JUL".equalsIgnoreCase(fname))
+      return new JulWatcher(slf4jImpl);
+    if ("Log4j".equals(fname))
+      return new Log4jWatcher(slf4jImpl);
+
+    try {
+      return loader != null ? loader.newInstance(fname, LogWatcher.class) : null;
+    }
+    catch (Throwable e) {
+      log.warn("Unable to load LogWatcher {}: {}", fname, e);
+    }
+
+    return null;
+  }
 }
\ No newline at end of file

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/logging/jul/JulWatcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/logging/jul/JulWatcher.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/logging/jul/JulWatcher.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/logging/jul/JulWatcher.java Tue May  7 11:20:55 2013
@@ -16,6 +16,13 @@
  */
 package org.apache.solr.logging.jul;
 
+import com.google.common.base.Throwables;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.logging.CircularList;
+import org.apache.solr.logging.ListenerConfig;
+import org.apache.solr.logging.LogWatcher;
+import org.apache.solr.logging.LoggerInfo;
+
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Date;
@@ -28,15 +35,6 @@ import java.util.logging.LogManager;
 import java.util.logging.LogRecord;
 import java.util.logging.Logger;
 
-import org.apache.solr.common.SolrDocument;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.logging.CircularList;
-import org.apache.solr.logging.ListenerConfig;
-import org.apache.solr.logging.LoggerInfo;
-import org.apache.solr.logging.LogWatcher;
-
-import com.google.common.base.Throwables;
-
 public class JulWatcher extends LogWatcher<LogRecord> {
 
   final String name;
@@ -131,7 +129,7 @@ public class JulWatcher extends LogWatch
   }
 
   @Override
-  public void registerListener(ListenerConfig cfg, CoreContainer container) {
+  public void registerListener(ListenerConfig cfg) {
     if(history!=null) {
       throw new IllegalStateException("History already registered");
     }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java Tue May  7 11:20:55 2013
@@ -573,11 +573,17 @@ public abstract class SolrQueryParserBas
         throws SyntaxError {
     Query query = getFieldQuery(field, queryText, true);
 
-    if (query instanceof PhraseQuery) {
-      ((PhraseQuery) query).setSlop(slop);
-    }
-    if (query instanceof MultiPhraseQuery) {
-      ((MultiPhraseQuery) query).setSlop(slop);
+    // only set slop of the phrase query was a result of this parser
+    // and not a sub-parser.
+    if (subQParser == null) {
+
+      if (query instanceof PhraseQuery) {
+        ((PhraseQuery) query).setSlop(slop);
+      }
+      if (query instanceof MultiPhraseQuery) {
+        ((MultiPhraseQuery) query).setSlop(slop);
+      }
+
     }
 
     return query;
@@ -779,7 +785,7 @@ public abstract class SolrQueryParserBas
       float boostVal = Float.parseFloat(boost.image);
       // avoid boosting null queries, such as those caused by stop words
       if (q != null) {
-        q.setBoost(boostVal);
+        q.setBoost(q.getBoost() * boostVal);
       }
     }
     return q;
@@ -932,7 +938,8 @@ public abstract class SolrQueryParserBas
   }
 
 
-  // called from parser
+  private QParser subQParser = null;
+
   protected Query getFieldQuery(String field, String queryText, boolean quoted) throws SyntaxError {
     checkNullField(field);
     // intercept magic field name of "_" to use as a hook for our
@@ -940,8 +947,8 @@ public abstract class SolrQueryParserBas
     if (field.charAt(0) == '_' && parser != null) {
       MagicFieldName magic = MagicFieldName.get(field);
       if (null != magic) {
-        QParser nested = parser.subQuery(queryText, magic.subParser);
-        return nested.getQuery();
+        subQParser = parser.subQuery(queryText, magic.subParser);
+        return subQParser.getQuery();
       }
     }
     SchemaField sf = schema.getFieldOrNull(field);

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/NumericFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/NumericFacets.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/NumericFacets.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/NumericFacets.java Tue May  7 11:20:55 2013
@@ -39,6 +39,7 @@ import org.apache.lucene.search.FieldCac
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.StringHelper;
 import org.apache.solr.common.params.FacetParams;
@@ -171,7 +172,7 @@ final class NumericFacets {
             longs = new FieldCache.Longs() {
               @Override
               public long get(int docID) {
-                return Float.floatToIntBits(floats.get(docID));
+                return NumericUtils.floatToSortableInt(floats.get(docID));
               }
             };
             break;
@@ -180,7 +181,7 @@ final class NumericFacets {
             longs = new FieldCache.Longs() {
               @Override
               public long get(int docID) {
-                return Double.doubleToLongBits(doubles.get(docID));
+                return NumericUtils.doubleToSortableLong(doubles.get(docID));
               }
             };
             break;

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Tue May  7 11:20:55 2013
@@ -105,17 +105,18 @@ public class SimpleFacets {
   /** The main set of documents all facet counts should be relative to */
   protected DocSet docsOrig;
   /** Configuration params behavior should be driven by */
-  protected SolrParams params;
-  protected SolrParams required;
+  protected final SolrParams orig;
   /** Searcher to use for all calculations */
-  protected SolrIndexSearcher searcher;
-  protected SolrQueryRequest req;
-  protected ResponseBuilder rb;
+  protected final SolrIndexSearcher searcher;
+  protected final SolrQueryRequest req;
+  protected final ResponseBuilder rb;
 
   protected SimpleOrderedMap<Object> facetResponse;
 
   // per-facet values
   protected SolrParams localParams; // localParams on this particular facet command
+  protected SolrParams params;      // local+original
+  protected SolrParams required;    // required version of params
   protected String facetValue;      // the field to or query to facet on (minus local params)
   protected DocSet docs;            // the base docset for this particular facet
   protected String key;             // what name should the results be stored under
@@ -134,7 +135,7 @@ public class SimpleFacets {
     this.req = req;
     this.searcher = req.getSearcher();
     this.docs = this.docsOrig = docs;
-    this.params = params;
+    this.params = orig = params;
     this.required = new RequiredSolrParams(params);
     this.rb = rb;
   }
@@ -147,7 +148,11 @@ public class SimpleFacets {
     key = param;
     threads = -1;
 
-    if (localParams == null) return;
+    if (localParams == null) {
+      return;
+    }
+    params = SolrParams.wrapDefaults(localParams, orig);
+    required = new RequiredSolrParams(params);
 
     // remove local params unless it's a query
     if (type != FacetParams.FACET_QUERY) { // TODO Cut over to an Enum here
@@ -315,7 +320,7 @@ public class SimpleFacets {
     }
     
     TermAllGroupsCollector collector = new TermAllGroupsCollector(groupField);
-    Filter mainQueryFilter = docsOrig.getTopFilter(); // This returns a filter that only matches documents matching with q param and fq params
+    Filter mainQueryFilter = docs.getTopFilter(); // This returns a filter that only matches documents matching with q param and fq params
     searcher.search(facetQuery, mainQueryFilter, collector);
     return collector.getGroupCount();
   }
@@ -460,12 +465,16 @@ public class SimpleFacets {
     TermGroupFacetCollector collector = TermGroupFacetCollector.createTermGroupFacetCollector(groupField, field, multiToken, prefixBR, 128);
     searcher.search(new MatchAllDocsQuery(), base.getTopFilter(), collector);
     boolean orderByCount = sort.equals(FacetParams.FACET_SORT_COUNT) || sort.equals(FacetParams.FACET_SORT_COUNT_LEGACY);
-    TermGroupFacetCollector.GroupedFacetResult result = collector.mergeSegmentResults(offset + limit, mincount, orderByCount);
+    TermGroupFacetCollector.GroupedFacetResult result 
+      = collector.mergeSegmentResults(limit < 0 ? Integer.MAX_VALUE : 
+                                      (offset + limit), 
+                                      mincount, orderByCount);
 
     CharsRef charsRef = new CharsRef();
     FieldType facetFieldType = searcher.getSchema().getFieldType(field);
     NamedList<Integer> facetCounts = new NamedList<Integer>();
-    List<TermGroupFacetCollector.FacetEntry> scopedEntries = result.getFacetEntries(offset, limit);
+    List<TermGroupFacetCollector.FacetEntry> scopedEntries 
+      = result.getFacetEntries(offset, limit < 0 ? Integer.MAX_VALUE : limit);
     for (TermGroupFacetCollector.FacetEntry facetEntry : scopedEntries) {
       facetFieldType.indexedToReadable(facetEntry.getValue(), charsRef);
       facetCounts.add(charsRef.toString(), facetEntry.getCount());

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java Tue May  7 11:20:55 2013
@@ -71,7 +71,7 @@ public interface SolrQueryRequest {
   /** The solr core (coordinator, etc) associated with this request */
   public SolrCore getCore();
 
-  /** The index schema associated with this request */
+  /** The schema snapshot from core.getLatestSchema() at request creation. */
   public IndexSchema getSchema();
 
   /**

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java Tue May  7 11:20:55 2013
@@ -42,6 +42,7 @@ import java.util.HashMap;
  */
 public abstract class SolrQueryRequestBase implements SolrQueryRequest {
   protected final SolrCore core;
+  protected final IndexSchema schema;
   protected final SolrParams origParams;
   protected SolrParams params;
   protected Map<Object,Object> context;
@@ -49,6 +50,7 @@ public abstract class SolrQueryRequestBa
 
   public SolrQueryRequestBase(SolrCore core, SolrParams params) {
     this.core = core;
+    this.schema = null == core ? null : core.getLatestSchema();
     this.params = this.origParams = params;
   }
 
@@ -85,7 +87,7 @@ public abstract class SolrQueryRequestBa
   protected RefCounted<SolrIndexSearcher> searcherHolder;
   @Override
   public SolrIndexSearcher getSearcher() {
-    if(core == null) return null;//a request for a core admin will no have a core
+    if(core == null) return null;//a request for a core admin will not have a core
     // should this reach out and get a searcher from the core singleton, or
     // should the core populate one in a factory method to create requests?
     // or there could be a setSearcher() method that Solr calls
@@ -107,7 +109,7 @@ public abstract class SolrQueryRequestBa
   @Override
   public IndexSchema getSchema() {
     //a request for a core admin will no have a core
-    return core == null? null: core.getSchema();
+    return schema;
   }
 
   /**

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/response/CSVResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/response/CSVResponseWriter.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/response/CSVResponseWriter.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/response/CSVResponseWriter.java Tue May  7 11:20:55 2013
@@ -235,19 +235,27 @@ class CSVWriter extends TextResponseWrit
       // encapsulator will already be disabled if it wasn't specified
     }
 
-    Collection<String> fields = returnFields.getLuceneFieldNames();
+    Collection<String> fields = returnFields.getRequestedFieldNames();
     Object responseObj = rsp.getValues().get("response");
     boolean returnOnlyStored = false;
-    if (fields==null) {
+    if (fields==null||returnFields.hasPatternMatching()) {
       if (responseObj instanceof SolrDocumentList) {
         // get the list of fields from the SolrDocumentList
-        fields = new LinkedHashSet<String>();
+        if(fields==null) {
+          fields = new LinkedHashSet<String>();
+        }
         for (SolrDocument sdoc: (SolrDocumentList)responseObj) {
           fields.addAll(sdoc.getFieldNames());
         }
       } else {
         // get the list of fields from the index
-        fields = req.getSearcher().getFieldNames();
+        Collection<String> all = req.getSearcher().getFieldNames();
+        if(fields==null) {
+          fields = all;
+        }
+        else {
+          fields.addAll(all);
+        }
       }
       if (returnFields.wantsScore()) {
         fields.add("score");

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/rest/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/rest/package.html?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/rest/package.html (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/rest/package.html Tue May  7 11:20:55 2013
@@ -1,27 +1,29 @@
 <!doctype html public "-//w3c//dtd html 4.0 transitional//en">
 <!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+  
+      http://www.apache.org/licenses/LICENSE-2.0
+  
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+  -->
 
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
 <html>
 <head>
   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
 </head>
 <body>
 <p>
-  Provides RESTful API access to the Solr Schema using Restlet.
+  Solr RESTful APIs via Restlet.
 </p>
 </body>
 </html>

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java Tue May  7 11:20:55 2013
@@ -41,6 +41,8 @@ public abstract class AbstractSubTypeFie
   protected String suffix;
   protected int dynFieldProps;
   protected String[] suffixes;
+  protected String subFieldType = null;
+  protected String subSuffix = null;
   protected IndexSchema schema;   // needed for retrieving SchemaFields
 
   public FieldType getSubType() {
@@ -49,14 +51,15 @@ public abstract class AbstractSubTypeFie
 
   @Override
   protected void init(IndexSchema schema, Map<String, String> args) {
+    super.init(schema, args);
     this.schema = schema;
     //it's not a first class citizen for the IndexSchema
     SolrParams p = new MapSolrParams(args);
-    String subFT = p.get(SUB_FIELD_TYPE);
-    String subSuffix = p.get(SUB_FIELD_SUFFIX);
-    if (subFT != null) {
+    subFieldType = p.get(SUB_FIELD_TYPE);
+    subSuffix = p.get(SUB_FIELD_SUFFIX);
+    if (subFieldType != null) {
       args.remove(SUB_FIELD_TYPE);
-      subType = schema.getFieldTypeByName(subFT.trim());
+      subType = schema.getFieldTypeByName(subFieldType.trim());
       suffix = POLY_FIELD_SEPARATOR + subType.typeName;
     } else if (subSuffix != null) {
       args.remove(SUB_FIELD_SUFFIX);
@@ -66,7 +69,6 @@ public abstract class AbstractSubTypeFie
               + " must specify the " +
               SUB_FIELD_TYPE + " attribute or the " + SUB_FIELD_SUFFIX + " attribute.");
     }
-
   }
 
   /**
@@ -93,8 +95,17 @@ public abstract class AbstractSubTypeFie
     return proto;
   }
 
+  /**
+   * Registers the polyfield dynamic prototype for this field type: : "*___(field type name)" 
+   * 
+   * {@inheritDoc}
+   *  
+   * @param schema {@inheritDoc}
+   *
+   */
   @Override
   public void inform(IndexSchema schema) {
+    this.schema = schema;
     //Can't do this until here b/c the Dynamic Fields are not initialized until here.
     if (subType != null) {
       SchemaField proto = registerPolyFieldDynamicPrototype(schema, subType);
@@ -117,7 +128,7 @@ public abstract class AbstractSubTypeFie
     }
   }
 
-  protected SchemaField subField(SchemaField base, int i) {
+  protected SchemaField subField(SchemaField base, int i, IndexSchema schema) {
     return schema.getField(base.getName() + suffixes[i]);
   }
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/CurrencyField.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/CurrencyField.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/CurrencyField.java Tue May  7 11:20:55 2013
@@ -209,12 +209,15 @@ public class CurrencyField extends Field
   }
 
   /**
-   * When index schema is informed, add dynamic fields.
-   *
-   * @param indexSchema The index schema.
+   * When index schema is informed, add dynamic fields "*____currency" and "*____amount_raw". 
+   * 
+   * {@inheritDoc}
+   * 
+   * @param schema {@inheritDoc}
    */
   @Override
-  public void inform(IndexSchema indexSchema) {
+  public void inform(IndexSchema schema) {
+    this.schema = schema;
     createDynamicCurrencyField(FIELD_SUFFIX_CURRENCY,   fieldTypeCurrency);
     createDynamicCurrencyField(FIELD_SUFFIX_AMOUNT_RAW, fieldTypeAmountRaw);
   }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java Tue May  7 11:20:55 2013
@@ -55,7 +55,7 @@ import java.util.Map;
  *
  * @see ExternalFileFieldReloader
  */
-public class ExternalFileField extends FieldType {
+public class ExternalFileField extends FieldType implements SchemaAware {
   private FieldType ftype;
   private String keyFieldName;
   private IndexSchema schema;
@@ -127,4 +127,8 @@ public class ExternalFileField extends F
         schema.getField(keyFieldName);
   }
 
+  @Override
+  public void inform(IndexSchema schema) {
+    this.schema = schema;
+  }
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/ExternalFileFieldReloader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/ExternalFileFieldReloader.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/ExternalFileFieldReloader.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/ExternalFileFieldReloader.java Tue May  7 11:20:55 2013
@@ -50,7 +50,6 @@ import java.util.List;
  */
 public class ExternalFileFieldReloader extends AbstractSolrEventListener {
 
-  private IndexSchema schema;
   private String datadir;
   private List<FileFloatSource> fieldSources = new ArrayList<FileFloatSource>();
 
@@ -58,29 +57,36 @@ public class ExternalFileFieldReloader e
 
   public ExternalFileFieldReloader(SolrCore core) {
     super(core);
-    schema = core.getSchema();
     datadir = core.getDataDir();
   }
 
   @Override
   public void init(NamedList args) {
-    for (SchemaField field : schema.getFields().values()) {
-      FieldType type = field.getType();
-      if (type instanceof ExternalFileField) {
-        ExternalFileField eff = (ExternalFileField) type;
-        fieldSources.add(eff.getFileFloatSource(field, datadir));
-        log.info("Adding ExternalFileFieldReloader listener for field {}", field.getName());
-      }
-    }
+    cacheFieldSources(getCore().getLatestSchema());
   }
 
   @Override
   public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) {
     // We need to reload the caches for the new searcher
+    if (null == currentSearcher || newSearcher.getSchema() != currentSearcher.getSchema()) {
+      cacheFieldSources(newSearcher.getSchema());
+    }
     IndexReader reader = newSearcher.getIndexReader();
     for (FileFloatSource fieldSource : fieldSources) {
       fieldSource.refreshCache(reader);
     }
   }
-}
 
+  /** Caches FileFloatSource's from all ExternalFileField instances in the schema */
+  public void cacheFieldSources(IndexSchema schema) {
+    fieldSources.clear();
+    for (SchemaField field : schema.getFields().values()) {
+      FieldType type = field.getType();
+      if (type instanceof ExternalFileField) {
+        ExternalFileField eff = (ExternalFileField)type;
+        fieldSources.add(eff.getFileFloatSource(field, datadir));
+        log.info("Adding ExternalFileFieldReloader listener for field {}", field.getName());
+      }
+    }
+  }
+}

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldProperties.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldProperties.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldProperties.java Tue May  7 11:20:55 2013
@@ -64,7 +64,7 @@ public abstract class FieldProperties {
   static final Map<String,Integer> propertyMap = new HashMap<String,Integer>();
   static {
     for (String prop : propertyNames) {
-      propertyMap.put(prop, propertyNameToInt(prop));
+      propertyMap.put(prop, propertyNameToInt(prop, true));
     }
   }
 
@@ -74,13 +74,17 @@ public abstract class FieldProperties {
     return propertyNames[ Integer.numberOfTrailingZeros(property) ];
   }
 
-  static int propertyNameToInt(String name) {
+  static int propertyNameToInt(String name, boolean failOnError) {
     for (int i=0; i<propertyNames.length; i++) {
       if (propertyNames[i].equals(name)) {
         return 1 << i;
       }
     }
-    return 0;
+    if (failOnError && !"default".equals(name)) {
+      throw new IllegalArgumentException("Invalid field property: " + name);
+    } else {
+      return 0;
+    }
   }
 
 
@@ -105,16 +109,16 @@ public abstract class FieldProperties {
     return (bitfield & props) == 0;
   }
 
-  static int parseProperties(Map<String,String> properties, boolean which) {
+  static int parseProperties(Map<String,?> properties, boolean which, boolean failOnError) {
     int props = 0;
-    for (Map.Entry<String, String> entry : properties.entrySet()) {
-      String val = entry.getValue();
+    for (Map.Entry<String,?> entry : properties.entrySet()) {
+      Object val = entry.getValue();
       if(val == null) continue;
-      if (Boolean.parseBoolean(val) == which) {
-        props |= propertyNameToInt(entry.getKey());
+      boolean boolVal = val instanceof Boolean ? (Boolean)val : Boolean.parseBoolean(val.toString());
+      if (boolVal == which) {
+        props |= propertyNameToInt(entry.getKey(), failOnError);
       }
     }
     return props;
   }
-
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldType.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldType.java Tue May  7 11:20:55 2013
@@ -23,11 +23,12 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.lucene.analysis.util.AbstractAnalysisFactory.LUCENE_MATCH_VERSION_PARAM; 
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -58,7 +59,6 @@ import org.apache.solr.analysis.SolrAnal
 import org.apache.solr.analysis.TokenizerChain;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.response.TextResponseWriter;
@@ -95,7 +95,6 @@ public abstract class FieldType extends 
   private boolean isExplicitQueryAnalyzer;
   private boolean isExplicitAnalyzer;
 
-
   /** Returns true if fields of this type should be tokenized */
   public boolean isTokenized() {
     return (properties & TOKENIZED) != 0;
@@ -152,9 +151,10 @@ public abstract class FieldType extends 
 
     this.args = Collections.unmodifiableMap(args);
     Map<String,String> initArgs = new HashMap<String,String>(args);
+    initArgs.remove(CLASS_NAME); // consume the class arg 
 
-    trueProperties = FieldProperties.parseProperties(initArgs,true);
-    falseProperties = FieldProperties.parseProperties(initArgs,false);
+    trueProperties = FieldProperties.parseProperties(initArgs,true,false);
+    falseProperties = FieldProperties.parseProperties(initArgs,false,false);
 
     properties &= ~falseProperties;
     properties |= trueProperties;
@@ -413,7 +413,21 @@ public abstract class FieldType extends 
     return isExplicitAnalyzer;
   }
 
-    /**
+  /**
+   * @return the string used to specify the concrete class name in a serialized representation: the class arg.  
+   *         If the concrete class name was not specified via a class arg, returns {@code getClass().getName()}.
+   */
+  public String getClassArg() {
+    if (null != args) {
+      String className = args.get(CLASS_NAME);
+      if (null != className) {
+        return className;
+      }
+    }
+    return getClass().getName();
+  }
+
+  /**
    * Default analyzer for types that only produce 1 verbatim token...
    * A maximum size of chars to be read must be specified
    */
@@ -635,6 +649,7 @@ public abstract class FieldType extends 
    * Sub-classes should override this method to provide their own range query implementation. They should strive to
    * handle nulls in part1 and/or part2 as well as unequal minInclusive and maxInclusive parameters gracefully.
    *
+   * @param parser       the {@link org.apache.solr.search.QParser} calling the method
    * @param field        the schema field
    * @param part1        the lower boundary of the range, nulls are allowed.
    * @param part2        the upper boundary of the range, nulls are allowed
@@ -722,20 +737,27 @@ public abstract class FieldType extends 
     }
   }
 
-  private static final String TYPE_NAME = "name";
-  private static final String CLASS_NAME = "class";
-  private static final String ANALYZER = "analyzer";
-  private static final String INDEX_ANALYZER = "indexAnalyzer";
-  private static final String QUERY_ANALYZER = "queryAnalyzer";
-  private static final String MULTI_TERM_ANALYZER = "multiTermAnalyzer";
-  private static final String SIMILARITY = "similarity";
+  public static final String TYPE = "type";
+  public static final String TYPE_NAME = "name";
+  public static final String CLASS_NAME = "class";
+  public static final String ANALYZER = "analyzer";
+  public static final String INDEX = "index";
+  public static final String INDEX_ANALYZER = "indexAnalyzer";
+  public static final String QUERY = "query";
+  public static final String QUERY_ANALYZER = "queryAnalyzer";
+  public static final String MULTI_TERM = "multiterm";
+  public static final String MULTI_TERM_ANALYZER = "multiTermAnalyzer";
+  public static final String SIMILARITY = "similarity";
+  public static final String CHAR_FILTER = "charFilter";
+  public static final String CHAR_FILTERS = "charFilters";
+  public static final String TOKENIZER = "tokenizer";
+  public static final String FILTER = "filter";
+  public static final String FILTERS = "filters";
+
   private static final String POSTINGS_FORMAT = "postingsFormat";
   private static final String DOC_VALUES_FORMAT = "docValuesFormat";
   private static final String AUTO_GENERATE_PHRASE_QUERIES = "autoGeneratePhraseQueries";
   private static final String ARGS = "args";
-  private static final String CHAR_FILTERS = "charFilters";
-  private static final String TOKENIZER = "tokenizer";
-  private static final String FILTERS = "filters";
   private static final String POSITION_INCREMENT_GAP = "positionIncrementGap";
 
   /**
@@ -745,12 +767,14 @@ public abstract class FieldType extends 
   public SimpleOrderedMap<Object> getNamedPropertyValues(boolean showDefaults) {
     SimpleOrderedMap<Object> namedPropertyValues = new SimpleOrderedMap<Object>();
     namedPropertyValues.add(TYPE_NAME, getTypeName());
-    namedPropertyValues.add(CLASS_NAME, normalizeSPIname(getClass().getName()));
+    namedPropertyValues.add(CLASS_NAME, getClassArg());
     if (showDefaults) {
       Map<String,String> fieldTypeArgs = getNonFieldPropertyArgs();
       if (null != fieldTypeArgs) {
         for (String key : fieldTypeArgs.keySet()) {
-          namedPropertyValues.add(key, fieldTypeArgs.get(key));
+        if ( ! CLASS_NAME.equals(key) && ! TYPE_NAME.equals(key)) {
+            namedPropertyValues.add(key, fieldTypeArgs.get(key));
+          }
         }
       }
       if (this instanceof TextField) {
@@ -783,7 +807,7 @@ public abstract class FieldType extends 
       for (String key : args.keySet()) {
         if (fieldProperties.contains(key)) {
           namedPropertyValues.add(key, StrUtils.parseBool(args.get(key)));
-        } else {
+        } else if ( ! CLASS_NAME.equals(key) && ! TYPE_NAME.equals(key)) {
           namedPropertyValues.add(key, args.get(key));
         }
       }
@@ -802,8 +826,8 @@ public abstract class FieldType extends 
         namedPropertyValues.add(MULTI_TERM_ANALYZER, getAnalyzerProperties(((TextField) this).getMultiTermAnalyzer()));
       }
     }
-    if (null != getSimilarity()) {
-      namedPropertyValues.add(SIMILARITY, getSimilarityProperties());
+    if (null != getSimilarityFactory()) {
+      namedPropertyValues.add(SIMILARITY, getSimilarityFactory().getNamedPropertyValues());
     }
     if (null != getPostingsFormat()) {
       namedPropertyValues.add(POSTINGS_FORMAT, getPostingsFormat());
@@ -830,8 +854,7 @@ public abstract class FieldType extends 
    */
   protected static SimpleOrderedMap<Object> getAnalyzerProperties(Analyzer analyzer) {
     SimpleOrderedMap<Object> analyzerProps = new SimpleOrderedMap<Object>();
-    analyzerProps.add(CLASS_NAME, normalizeSPIname(analyzer.getClass().getName()));
-    
+
     if (analyzer instanceof TokenizerChain) {
       Map<String,String> factoryArgs;
       TokenizerChain tokenizerChain = (TokenizerChain)analyzer;
@@ -840,11 +863,19 @@ public abstract class FieldType extends 
         List<SimpleOrderedMap<Object>> charFilterProps = new ArrayList<SimpleOrderedMap<Object>>();
         for (CharFilterFactory charFilterFactory : charFilterFactories) {
           SimpleOrderedMap<Object> props = new SimpleOrderedMap<Object>();
-          props.add(CLASS_NAME, normalizeSPIname(charFilterFactory.getClass().getName()));
+          props.add(CLASS_NAME, charFilterFactory.getClassArg());
           factoryArgs = charFilterFactory.getOriginalArgs();
           if (null != factoryArgs) {
             for (String key : factoryArgs.keySet()) {
-              props.add(key, factoryArgs.get(key));
+              if ( ! CLASS_NAME.equals(key)) {
+                if (LUCENE_MATCH_VERSION_PARAM.equals(key)) {
+                  if (charFilterFactory.isExplicitLuceneMatchVersion()) {
+                    props.add(key, factoryArgs.get(key));
+                  }
+                } else {
+                   props.add(key, factoryArgs.get(key));
+                }
+              }
             }
           }
           charFilterProps.add(props);
@@ -854,11 +885,19 @@ public abstract class FieldType extends 
 
       SimpleOrderedMap<Object> tokenizerProps = new SimpleOrderedMap<Object>();
       TokenizerFactory tokenizerFactory = tokenizerChain.getTokenizerFactory();
-      tokenizerProps.add(CLASS_NAME, normalizeSPIname(tokenizerFactory.getClass().getName()));
+      tokenizerProps.add(CLASS_NAME, tokenizerFactory.getClassArg());
       factoryArgs = tokenizerFactory.getOriginalArgs();
       if (null != factoryArgs) {
         for (String key : factoryArgs.keySet()) {
-          tokenizerProps.add(key, factoryArgs.get(key));
+          if ( ! CLASS_NAME.equals(key)) {
+            if (LUCENE_MATCH_VERSION_PARAM.equals(key)) {
+              if (tokenizerFactory.isExplicitLuceneMatchVersion()) {
+                tokenizerProps.add(key, factoryArgs.get(key));
+              }
+            } else {
+              tokenizerProps.add(key, factoryArgs.get(key));
+            }
+          }
         }
       }
       analyzerProps.add(TOKENIZER, tokenizerProps);
@@ -868,42 +907,28 @@ public abstract class FieldType extends 
         List<SimpleOrderedMap<Object>> filterProps = new ArrayList<SimpleOrderedMap<Object>>();
         for (TokenFilterFactory filterFactory : filterFactories) {
           SimpleOrderedMap<Object> props = new SimpleOrderedMap<Object>();
-          props.add(CLASS_NAME, normalizeSPIname(filterFactory.getClass().getName()));
+          props.add(CLASS_NAME, filterFactory.getClassArg());
           factoryArgs = filterFactory.getOriginalArgs();
           if (null != factoryArgs) {
             for (String key : factoryArgs.keySet()) {
-              props.add(key, factoryArgs.get(key));
+              if ( ! CLASS_NAME.equals(key)) {
+                if (LUCENE_MATCH_VERSION_PARAM.equals(key)) {
+                  if (filterFactory.isExplicitLuceneMatchVersion()) {
+                    props.add(key, factoryArgs.get(key));
+                  }
+                } else {
+                  props.add(key, factoryArgs.get(key));
+                }
+              }
             }
           }
           filterProps.add(props);
         }
         analyzerProps.add(FILTERS, filterProps);
       }
+    } else { // analyzer is not instanceof TokenizerChain
+      analyzerProps.add(CLASS_NAME, analyzer.getClass().getName());
     }
     return analyzerProps;
   }
-
-  private static String normalizeSPIname(String fullyQualifiedName) {
-    if (fullyQualifiedName.startsWith("org.apache.lucene.") || fullyQualifiedName.startsWith("org.apache.solr.")) {
-      return "solr" + fullyQualifiedName.substring(fullyQualifiedName.lastIndexOf('.')); 
-    }
-    return fullyQualifiedName;
-  }
-
-  /** Returns a description of this field's similarity, if any */
-  protected SimpleOrderedMap<Object> getSimilarityProperties() {
-    SimpleOrderedMap<Object> props = new SimpleOrderedMap<Object>();
-    if (similarity != null) {
-      props.add(CLASS_NAME, normalizeSPIname(similarity.getClass().getName()));
-      SolrParams factoryParams = similarityFactory.getParams();
-      if (null != factoryParams) {
-        Iterator<String> iter = factoryParams.getParameterNamesIterator();
-        while (iter.hasNext()) {
-          String key = iter.next();
-          props.add(key, factoryParams.get(key));
-        }
-      }
-    }
-    return props;
-  }
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java Tue May  7 11:20:55 2013
@@ -137,11 +137,10 @@ public final class FieldTypePluginLoader
   @Override
   protected void init(FieldType plugin, Node node) throws Exception {
 
-    Map<String,String> params = DOMUtil.toMapExcept( node.getAttributes(), 
-                                                     "name","class" );
-    plugin.setArgs(schema, params );
+    Map<String,String> params = DOMUtil.toMapExcept( node.getAttributes(), "name");
+    plugin.setArgs(schema, params);
   }
-  
+
   @Override
   protected FieldType register(String name, 
                                FieldType plugin) throws Exception {
@@ -183,12 +182,7 @@ public final class FieldTypePluginLoader
   }
 
   private static class MultiTermChainBuilder {
-    static final KeywordTokenizerFactory keyFactory;
-
-    static {
-      keyFactory = new KeywordTokenizerFactory();
-      keyFactory.init(new HashMap<String,String>());
-    }
+    static final KeywordTokenizerFactory keyFactory = new KeywordTokenizerFactory(new HashMap<String,String>());
 
     ArrayList<CharFilterFactory> charFilters = null;
     ArrayList<TokenFilterFactory> filters = new ArrayList<TokenFilterFactory>(2);
@@ -305,14 +299,18 @@ public final class FieldTypePluginLoader
       ("[schema.xml] analyzer/charFilter", CharFilterFactory.class, false, false) {
 
       @Override
+      protected CharFilterFactory create(SolrResourceLoader loader, String name, String className, Node node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+        String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
+        params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, CharFilterFactory.class.getSimpleName()).toString());
+        CharFilterFactory factory = loader.newInstance(className, CharFilterFactory.class, getDefaultPackages(), new Class[] { Map.class }, new Object[] { params });
+        factory.setExplicitLuceneMatchVersion(null != configuredVersion);
+        return factory;
+      }
+
+      @Override
       protected void init(CharFilterFactory plugin, Node node) throws Exception {
         if( plugin != null ) {
-          final Map<String,String> params = DOMUtil.toMapExcept(node.getAttributes(),"class");
-
-          String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
-          plugin.setLuceneMatchVersion(parseConfiguredVersion(configuredVersion, plugin.getClass().getSimpleName()));
-
-          plugin.init( params );
           charFilters.add( plugin );
         }
       }
@@ -335,18 +333,23 @@ public final class FieldTypePluginLoader
     AbstractPluginLoader<TokenizerFactory> tokenizerLoader =
       new AbstractPluginLoader<TokenizerFactory>
       ("[schema.xml] analyzer/tokenizer", TokenizerFactory.class, false, false) {
+      
+      @Override
+      protected TokenizerFactory create(SolrResourceLoader loader, String name, String className, Node node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+        String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
+        params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, TokenizerFactory.class.getSimpleName()).toString());
+        TokenizerFactory factory = loader.newInstance(className, TokenizerFactory.class, getDefaultPackages(), new Class[] { Map.class }, new Object[] { params });
+        factory.setExplicitLuceneMatchVersion(null != configuredVersion);
+        return factory;
+      }
+      
       @Override
       protected void init(TokenizerFactory plugin, Node node) throws Exception {
         if( !tokenizers.isEmpty() ) {
           throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
               "The schema defines multiple tokenizers for: "+node );
         }
-        final Map<String,String> params = DOMUtil.toMapExcept(node.getAttributes(),"class");
-
-        String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
-        plugin.setLuceneMatchVersion(parseConfiguredVersion(configuredVersion, plugin.getClass().getSimpleName()));
-
-        plugin.init( params );
         tokenizers.add( plugin );
       }
 
@@ -372,14 +375,19 @@ public final class FieldTypePluginLoader
       new AbstractPluginLoader<TokenFilterFactory>("[schema.xml] analyzer/filter", TokenFilterFactory.class, false, false)
     {
       @Override
+      protected TokenFilterFactory create(SolrResourceLoader loader, String name, String className, Node node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+        String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
+        params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, TokenFilterFactory.class.getSimpleName()).toString());
+        TokenFilterFactory factory = loader.newInstance
+            (className, TokenFilterFactory.class, getDefaultPackages(), new Class[] { Map.class }, new Object[] { params });
+        factory.setExplicitLuceneMatchVersion(null != configuredVersion);
+        return factory;
+      }
+      
+      @Override
       protected void init(TokenFilterFactory plugin, Node node) throws Exception {
         if( plugin != null ) {
-          final Map<String,String> params = DOMUtil.toMapExcept(node.getAttributes(),"class");
-
-          String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
-          plugin.setLuceneMatchVersion(parseConfiguredVersion(configuredVersion, plugin.getClass().getSimpleName()));
-
-          plugin.init( params );
           filters.add( plugin );
         }
       }