You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/13 08:33:34 UTC

svn commit: r1513364 [3/5] - in /lucene/dev/branches/lucene3069: ./ dev-tools/ dev-tools/idea/solr/core/src/java/ dev-tools/idea/solr/core/src/test/ dev-tools/maven/ dev-tools/maven/solr/core/src/java/ dev-tools/maven/solr/core/src/test/ dev-tools/scri...

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java Tue Aug 13 06:33:29 2013
@@ -16,50 +16,51 @@ package org.apache.solr.handler.loader;
  * limitations under the License.
  */
 
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.update.processor.UpdateRequestProcessor;
-import org.apache.solr.update.AddUpdateCommand;
-import org.apache.solr.update.CommitUpdateCommand;
-import org.apache.solr.update.RollbackUpdateCommand;
-import org.apache.solr.update.DeleteUpdateCommand;
-import org.apache.solr.util.xslt.TransformerProvider;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.common.util.ContentStream;
-import org.apache.solr.common.util.ContentStreamBase;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.XMLErrorLogger;
+import com.google.common.collect.Lists;
+import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
 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.StrUtils;
+import org.apache.solr.common.util.XMLErrorLogger;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.handler.RequestHandlerUtils;
 import org.apache.solr.handler.UpdateRequestHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+import org.apache.solr.update.RollbackUpdateCommand;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.util.EmptyEntityResolver;
-import org.apache.commons.io.IOUtils;
+import org.apache.solr.util.xslt.TransformerProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.InputSource;
 import org.xml.sax.XMLReader;
 
-import javax.xml.stream.XMLStreamReader;
-import javax.xml.stream.XMLStreamException;
+import javax.xml.parsers.SAXParserFactory;
 import javax.xml.stream.FactoryConfigurationError;
-import javax.xml.stream.XMLStreamConstants;
 import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamConstants;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
 import javax.xml.transform.Transformer;
 import javax.xml.transform.TransformerException;
 import javax.xml.transform.dom.DOMResult;
 import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.sax.SAXSource;
-import javax.xml.parsers.SAXParserFactory;
-
 import java.io.ByteArrayInputStream;
-import java.io.InputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -381,6 +382,7 @@ public class XMLLoader extends ContentSt
     float boost = 1.0f;
     boolean isNull = false;
     String update = null;
+    Collection<SolrInputDocument> subDocs = null;
     Map<String, Map<String, Object>> updateMap = null;
     boolean complete = false;
     while (!complete) {
@@ -395,9 +397,14 @@ public class XMLLoader extends ContentSt
 
         case XMLStreamConstants.END_ELEMENT:
           if ("doc".equals(parser.getLocalName())) {
+            if (subDocs != null && !subDocs.isEmpty()) {
+              doc.addChildDocuments(subDocs);
+              subDocs = null;
+            }
             complete = true;
             break;
           } else if ("field".equals(parser.getLocalName())) {
+            // should I warn in some text has been found too
             Object v = isNull ? null : text.toString();
             if (update != null) {
               if (updateMap == null) updateMap = new HashMap<String, Map<String, Object>>();
@@ -425,34 +432,43 @@ public class XMLLoader extends ContentSt
             }
             doc.addField(name, v, boost);
             boost = 1.0f;
+            // field is over
+            name = null;
           }
           break;
 
         case XMLStreamConstants.START_ELEMENT:
           text.setLength(0);
           String localName = parser.getLocalName();
-          if (!"field".equals(localName)) {
-            log.warn("unexpected XML tag doc/" + localName);
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                    "unexpected XML tag doc/" + localName);
+          if ("doc".equals(localName)) {
+            if (subDocs == null)
+              subDocs = Lists.newArrayList();
+            subDocs.add(readDoc(parser));
           }
-          boost = 1.0f;
-          update = null;
-          isNull = false;
-          String attrVal = "";
-          for (int i = 0; i < parser.getAttributeCount(); i++) {
-            attrName = parser.getAttributeLocalName(i);
-            attrVal = parser.getAttributeValue(i);
-            if ("name".equals(attrName)) {
-              name = attrVal;
-            } else if ("boost".equals(attrName)) {
-              boost = Float.parseFloat(attrVal);
-            } else if ("null".equals(attrName)) {
-              isNull = StrUtils.parseBoolean(attrVal);
-            } else if ("update".equals(attrName)) {
-              update = attrVal;
-            } else {
-              log.warn("Unknown attribute doc/field/@" + attrName);
+          else {
+            if (!"field".equals(localName)) {
+              log.warn("unexpected XML tag doc/" + localName);
+              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                  "unexpected XML tag doc/" + localName);
+            }
+            boost = 1.0f;
+            update = null;
+            isNull = false;
+            String attrVal = "";
+            for (int i = 0; i < parser.getAttributeCount(); i++) {
+              attrName = parser.getAttributeLocalName(i);
+              attrVal = parser.getAttributeValue(i);
+              if ("name".equals(attrName)) {
+                name = attrVal;
+              } else if ("boost".equals(attrName)) {
+                boost = Float.parseFloat(attrVal);
+              } else if ("null".equals(attrName)) {
+                isNull = StrUtils.parseBoolean(attrVal);
+              } else if ("update".equals(attrName)) {
+                update = attrVal;
+              } else {
+                log.warn("Unknown attribute doc/field/@" + attrName);
+              }
             }
           }
           break;

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java Tue Aug 13 06:33:29 2013
@@ -113,7 +113,7 @@ public class DefaultSolrHighlighter exte
     formatters.put("", fmt);
     formatters.put(null, fmt);
 
-    // Load the formatters
+    // Load the encoders
     SolrEncoder enc = solrCore.initPlugins(info.getChildren("encoder"), encoders,SolrEncoder.class,null);
     if (enc == null) enc = new DefaultEncoder();
     encoders.put("", enc);

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/logging/ListenerConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/logging/ListenerConfig.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/logging/ListenerConfig.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/logging/ListenerConfig.java Tue Aug 13 06:33:29 2013
@@ -19,8 +19,18 @@ package org.apache.solr.logging;
  */
 
 public class ListenerConfig {
-  public int size = 50;
-  public String threshold = null;
+
+  public final int size;
+  public final String threshold;
+
+  public ListenerConfig(int size, String threshold) {
+    this.size = size;
+    this.threshold = threshold;
+  }
+
+  public ListenerConfig() {
+    this(50, null);
+  }
   
   // Down the line, settings for solr URL/core to store logging
 }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/logging/LogWatcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/logging/LogWatcher.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/logging/LogWatcher.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/logging/LogWatcher.java Tue Aug 13 06:33:29 2013
@@ -19,7 +19,6 @@ 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;
@@ -119,35 +118,32 @@ public abstract class LogWatcher<E> {
    * 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 config a LogWatcherConfig object, containing the configuration for this LogWatcher.
    * @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) {
+  public static LogWatcher newRegisteredLogWatcher(LogWatcherConfig config, SolrResourceLoader loader) {
 
-    if (!config.getBool(ConfigSolr.CfgProp.SOLR_LOGGING_ENABLED, true))
+    if (!config.isEnabled())
       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) {
+      if (config.getWatcherSize() > 0) {
         log.info("Registering Log Listener [{}]", logWatcher.getName());
-        logWatcher.registerListener(v);
+        logWatcher.registerListener(config.asListenerConfig());
       }
     }
 
     return logWatcher;
   }
 
-  private static LogWatcher createWatcher(ConfigSolr config, SolrResourceLoader loader) {
+  private static LogWatcher createWatcher(LogWatcherConfig config, SolrResourceLoader loader) {
 
-    String fname = config.get(ConfigSolr.CfgProp.SOLR_LOGGING_CLASS, null);
+    String fname = config.getLoggingClass();
     String slf4jImpl;
 
     try {

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Tue Aug 13 06:33:29 2013
@@ -768,7 +768,7 @@ public class SimpleFacets {
       // facet.offset when sorting by index order.
 
       if (startTermBytes != null) {
-        if (termsEnum.seekCeil(startTermBytes, true) == TermsEnum.SeekStatus.END) {
+        if (termsEnum.seekCeil(startTermBytes) == TermsEnum.SeekStatus.END) {
           termsEnum = null;
         } else {
           term = termsEnum.term();

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/request/UnInvertedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/request/UnInvertedField.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/request/UnInvertedField.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/request/UnInvertedField.java Tue Aug 13 06:33:29 2013
@@ -231,13 +231,13 @@ public class UnInvertedField extends Doc
       TermsEnum te = getOrdTermsEnum(searcher.getAtomicReader());
       if (te != null && prefix != null && prefix.length() > 0) {
         final BytesRef prefixBr = new BytesRef(prefix);
-        if (te.seekCeil(prefixBr, true) == TermsEnum.SeekStatus.END) {
+        if (te.seekCeil(prefixBr) == TermsEnum.SeekStatus.END) {
           startTerm = numTermsInField;
         } else {
           startTerm = (int) te.ord();
         }
         prefixBr.append(UnicodeUtil.BIG_TERM);
-        if (te.seekCeil(prefixBr, true) == TermsEnum.SeekStatus.END) {
+        if (te.seekCeil(prefixBr) == TermsEnum.SeekStatus.END) {
           endTerm = numTermsInField;
         } else {
           endTerm = (int) te.ord();

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java Tue Aug 13 06:33:29 2013
@@ -240,12 +240,14 @@ public abstract class AbstractSpatialFie
 
     //We get the valueSource for the score then the filter and combine them.
     ValueSource valueSource;
-    if ("distance".equals(score))
-      valueSource = strategy.makeDistanceValueSource(spatialArgs.getShape().getCenter());
-    else if ("recipDistance".equals(score))
+    if ("distance".equals(score)) {
+      double multiplier = 1.0;//TODO support units=kilometers
+      valueSource = strategy.makeDistanceValueSource(spatialArgs.getShape().getCenter(), multiplier);
+    } else if ("recipDistance".equals(score)) {
       valueSource = strategy.makeRecipDistanceValueSource(spatialArgs.getShape());
-    else
+    } else {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'score' local-param must be one of 'none', 'distance', or 'recipDistance'");
+    }
     FunctionQuery functionQuery = new FunctionQuery(valueSource);
 
     if (localParams != null && !localParams.getBool(FILTER_PARAM, true))

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/IndexSchema.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/IndexSchema.java Tue Aug 13 06:33:29 2013
@@ -397,10 +397,6 @@ public class IndexSchema {
       return analyzer != null ? analyzer : getDynamicFieldType(fieldName).getAnalyzer();
     }
 
-    @Override
-    protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
-      return components;
-    }
   }
 
   private class SolrQueryAnalyzer extends SolrIndexAnalyzer {

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Tue Aug 13 06:33:29 2013
@@ -389,6 +389,9 @@ public final class ManagedIndexSchema ex
     newSchema.uniqueKeyField = uniqueKeyField;
     newSchema.uniqueKeyFieldName = uniqueKeyFieldName;
     newSchema.uniqueKeyFieldType = uniqueKeyFieldType;
+    
+    // After the schema is persisted, resourceName is the same as managedSchemaResourceName
+    newSchema.resourceName = managedSchemaResourceName;
 
     if (includeFieldDataStructures) {
       // These need new collections, since addFields() can add members to them

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java Tue Aug 13 06:33:29 2013
@@ -309,7 +309,7 @@ class JoinQuery extends Query {
       if (prefix == null) {
         term = termsEnum.next();
       } else {
-        if (termsEnum.seekCeil(prefix, true) != TermsEnum.SeekStatus.END) {
+        if (termsEnum.seekCeil(prefix) != TermsEnum.SeekStatus.END) {
           term = termsEnum.term();
         }
       }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/QParserPlugin.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/QParserPlugin.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/QParserPlugin.java Tue Aug 13 06:33:29 2013
@@ -20,6 +20,8 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.search.join.BlockJoinChildQParserPlugin;
+import org.apache.solr.search.join.BlockJoinParentQParserPlugin;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 
 import java.net.URL;
@@ -47,7 +49,9 @@ public abstract class QParserPlugin impl
     JoinQParserPlugin.NAME, JoinQParserPlugin.class,
     SurroundQParserPlugin.NAME, SurroundQParserPlugin.class,
     SwitchQParserPlugin.NAME, SwitchQParserPlugin.class,
-    MaxScoreQParserPlugin.NAME, MaxScoreQParserPlugin.class
+    MaxScoreQParserPlugin.NAME, MaxScoreQParserPlugin.class,
+    BlockJoinParentQParserPlugin.NAME, BlockJoinParentQParserPlugin.class,
+    BlockJoinChildQParserPlugin.NAME, BlockJoinChildQParserPlugin.class
   };
 
   /** return a {@link QParser} */

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Tue Aug 13 06:33:29 2013
@@ -718,7 +718,7 @@ public class SolrIndexSearcher extends I
     if (terms == null) return -1;
     BytesRef termBytes = t.bytes();
     final TermsEnum termsEnum = terms.iterator(null);
-    if (!termsEnum.seekExact(termBytes, false)) {
+    if (!termsEnum.seekExact(termBytes)) {
       return -1;
     }
     DocsEnum docs = termsEnum.docs(atomicReader.getLiveDocs(), null, DocsEnum.FLAG_NONE);
@@ -742,7 +742,7 @@ public class SolrIndexSearcher extends I
       if (terms == null) continue;
       
       TermsEnum te = terms.iterator(null);
-      if (te.seekExact(idBytes, true)) {
+      if (te.seekExact(idBytes)) {
         DocsEnum docs = te.docs(reader.getLiveDocs(), null, DocsEnum.FLAG_NONE);
         int id = docs.nextDoc();
         if (id == DocIdSetIterator.NO_MORE_DOCS) continue;

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java Tue Aug 13 06:33:29 2013
@@ -297,7 +297,7 @@ public class FileFloatSource extends Val
           continue;  // go to next line in file.. leave values as default.
         }
 
-        if (!termsEnum.seekExact(internalKey, false)) {
+        if (!termsEnum.seekExact(internalKey)) {
           if (notFoundCount<10) {  // collect first 10 not found for logging
             notFound.add(key);
           }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java Tue Aug 13 06:33:29 2013
@@ -20,28 +20,37 @@ package org.apache.solr.search.function.
 import com.spatial4j.core.distance.DistanceUtils;
 import com.spatial4j.core.exception.InvalidShapeException;
 import com.spatial4j.core.io.ParseUtils;
+import com.spatial4j.core.shape.Point;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.ConstNumberSource;
 import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
 import org.apache.lucene.queries.function.valuesource.MultiValueSource;
 import org.apache.lucene.queries.function.valuesource.VectorValueSource;
+import org.apache.lucene.spatial.SpatialStrategy;
 import org.apache.solr.common.params.SpatialParams;
+import org.apache.solr.schema.AbstractSpatialFieldType;
+import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.FunctionQParser;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.search.ValueSourceParser;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 /**
- * Parses "geodist" creating {@link HaversineConstFunction} or {@link HaversineFunction}.
+ * Parses "geodist" creating {@link HaversineConstFunction} or {@link HaversineFunction}
+ * or calling {@link SpatialStrategy#makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}.
  */
 public class GeoDistValueSourceParser extends ValueSourceParser {
 
   @Override
   public ValueSource parse(FunctionQParser fp) throws SyntaxError {
     // TODO: dispatch through SpatialQueryable in the future?
+
+    //note: parseValueSourceList can't handle a field reference to an AbstractSpatialFieldType,
+    // so those fields are expressly handled via sfield=
     List<ValueSource> sources = fp.parseValueSourceList();
 
     // "m" is a multi-value source, "x" is a single-value source
@@ -104,7 +113,7 @@ public class GeoDistValueSourceParser ex
     }
 
     // We have all the parameters at this point, now check if one of the points is constant
-    double[] constants;
+    double[] constants;//latLon
     constants = getConstants(mv1);
     MultiValueSource other = mv2;
     if (constants == null) {
@@ -112,6 +121,24 @@ public class GeoDistValueSourceParser ex
       other = mv1;
     }
 
+    // At this point we dispatch to one of:
+    // * SpatialStrategy.makeDistanceValueSource
+    // * HaversineConstFunction
+    // * HaversineFunction
+
+    // sfield can only be in mv2, according to the logic above
+    if (mv2 instanceof SpatialStrategyMultiValueSource) {
+      if (constants == null)
+        throw new SyntaxError("When using AbstractSpatialFieldType (e.g. RPT not LatLonType)," +
+            " the point must be supplied as constants");
+      // note: uses Haversine by default but can be changed via distCalc=...
+      SpatialStrategy strategy = ((SpatialStrategyMultiValueSource) mv2).strategy;
+      Point queryPoint = strategy.getSpatialContext().makePoint(constants[1], constants[0]);
+      //TODO Spatial4j 0.4 will have a direct constant
+      double multiplier = DistanceUtils.degrees2Dist(1, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+      return strategy.makeDistanceValueSource(queryPoint, multiplier);
+    }
+
     if (constants != null && other instanceof VectorValueSource) {
       return new HaversineConstFunction(constants[0], constants[1], (VectorValueSource)other);
     }
@@ -155,11 +182,33 @@ public class GeoDistValueSourceParser ex
     String sfield = fp.getParam(SpatialParams.FIELD);
     if (sfield == null) return null;
     SchemaField sf = fp.getReq().getSchema().getField(sfield);
-    ValueSource vs = sf.getType().getValueSource(sf, fp);
-    if (!(vs instanceof MultiValueSource)) {
-      throw new SyntaxError("Spatial field must implement MultiValueSource:" + sf);
+    FieldType type = sf.getType();
+    if (type instanceof AbstractSpatialFieldType) {
+      AbstractSpatialFieldType asft = (AbstractSpatialFieldType) type;
+      return new SpatialStrategyMultiValueSource(asft.getStrategy(sfield));
+    }
+    ValueSource vs = type.getValueSource(sf, fp);
+    if (vs instanceof MultiValueSource) {
+      return (MultiValueSource)vs;
+    }
+    throw new SyntaxError("Spatial field must implement MultiValueSource or extend AbstractSpatialFieldType:" + sf);
+  }
+
+  /** An unfortunate hack to use a {@link SpatialStrategy} instead of
+   * a ValueSource. */
+  private static class SpatialStrategyMultiValueSource extends VectorValueSource {
+
+    final SpatialStrategy strategy;
+
+    public SpatialStrategyMultiValueSource(SpatialStrategy strategy) {
+      super(Collections.EMPTY_LIST);
+      this.strategy = strategy;
+    }
+
+    @Override
+    public List<ValueSource> getSources() {
+      throw new IllegalStateException();
     }
-    return (MultiValueSource)vs;
   }
 
 }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java Tue Aug 13 06:33:29 2013
@@ -28,9 +28,11 @@ import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 import java.net.URL;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -68,6 +70,11 @@ public class SolrRequestParsers 
   public static final String SIMPLE = "simple";
   public static final String STANDARD = "standard";
   
+  private static final Charset CHARSET_US_ASCII = Charset.forName("US-ASCII");
+  
+  public static final String INPUT_ENCODING_KEY = "ie";
+  private static final byte[] INPUT_ENCODING_BYTES = INPUT_ENCODING_KEY.getBytes(CHARSET_US_ASCII);
+
   private final HashMap<String, SolrRequestParser> parsers =
       new HashMap<String, SolrRequestParser>();
   private final boolean enableRemoteStreams;
@@ -242,7 +249,7 @@ public class SolrRequestParsers 
             }
           }
         };
-        parseFormDataContent(in, Long.MAX_VALUE, IOUtils.CHARSET_UTF_8, map);
+        parseFormDataContent(in, Long.MAX_VALUE, IOUtils.CHARSET_UTF_8, map, true);
       } catch (IOException ioe) {
         throw new SolrException(ErrorCode.BAD_REQUEST, ioe);
       }
@@ -256,23 +263,53 @@ public class SolrRequestParsers 
    * @param charset to be used to decode resulting bytes after %-decoding
    * @param map place all parameters in this map
    */
-  @SuppressWarnings("fallthrough")
-  static long parseFormDataContent(final InputStream postContent, final long maxLen, final Charset charset, final Map<String,String[]> map) throws IOException {
-    final CharsetDecoder charsetDecoder = charset.newDecoder()
-      .onMalformedInput(CodingErrorAction.REPORT)
-      .onUnmappableCharacter(CodingErrorAction.REPORT);
+  @SuppressWarnings({"fallthrough", "resource"})
+  static long parseFormDataContent(final InputStream postContent, final long maxLen, Charset charset, final Map<String,String[]> map, boolean supportCharsetParam) throws IOException {
+    CharsetDecoder charsetDecoder = supportCharsetParam ? null : getCharsetDecoder(charset);
+    final LinkedList<Object> buffer = supportCharsetParam ? new LinkedList<Object>() : null;
     long len = 0L, keyPos = 0L, valuePos = 0L;
-    final ByteArrayOutputStream2 keyStream = new ByteArrayOutputStream2(),
-      valueStream = new ByteArrayOutputStream2();
-    ByteArrayOutputStream2 currentStream = keyStream;
+    final ByteArrayOutputStream keyStream = new ByteArrayOutputStream(),
+      valueStream = new ByteArrayOutputStream();
+    ByteArrayOutputStream currentStream = keyStream;
     for(;;) {
       int b = postContent.read();
       switch (b) {
         case -1: // end of stream
         case '&': // separator
           if (keyStream.size() > 0) {
-            final String key = decodeChars(keyStream, keyPos, charsetDecoder), value = decodeChars(valueStream, valuePos, charsetDecoder);
-            MultiMapSolrParams.addParam(key, value, map);
+            final byte[] keyBytes = keyStream.toByteArray(), valueBytes = valueStream.toByteArray();
+            if (Arrays.equals(keyBytes, INPUT_ENCODING_BYTES)) {
+              // we found a charset declaration in the raw bytes
+              if (charsetDecoder != null) {
+                throw new SolrException(ErrorCode.BAD_REQUEST,
+                  supportCharsetParam ? (
+                    "Query string invalid: duplicate '"+
+                    INPUT_ENCODING_KEY + "' (input encoding) key."
+                  ) : (
+                    "Key '" + INPUT_ENCODING_KEY + "' (input encoding) cannot "+
+                    "be used in POSTed application/x-www-form-urlencoded form data. "+
+                    "To set the input encoding of POSTed form data, use the "+
+                    "'Content-Type' header and provide a charset!"
+                  )
+                );
+              }
+              // decode the charset from raw bytes
+              charset = Charset.forName(decodeChars(valueBytes, keyPos, getCharsetDecoder(CHARSET_US_ASCII)));
+              charsetDecoder = getCharsetDecoder(charset);
+              // finally decode all buffered tokens
+              decodeBuffer(buffer, map, charsetDecoder);
+            } else if (charsetDecoder == null) {
+              // we have no charset decoder until now, buffer the keys / values for later processing:
+              buffer.add(keyBytes);
+              buffer.add(Long.valueOf(keyPos));
+              buffer.add(valueBytes);
+              buffer.add(Long.valueOf(valuePos));
+            } else {
+              // we already have a charsetDecoder, so we can directly decode without buffering:
+              final String key = decodeChars(keyBytes, keyPos, charsetDecoder),
+                  value = decodeChars(valueBytes, valuePos, charsetDecoder);
+              MultiMapSolrParams.addParam(key, value, map);
+            }
           } else if (valueStream.size() > 0) {
             throw new SolrException(ErrorCode.BAD_REQUEST, "application/x-www-form-urlencoded invalid: missing key");
           }
@@ -309,12 +346,23 @@ public class SolrRequestParsers 
         throw new SolrException(ErrorCode.BAD_REQUEST, "application/x-www-form-urlencoded content exceeds upload limit of " + (maxLen/1024L) + " KB");
       }
     }
+    // if we have not seen a charset declaration, decode the buffer now using the default one (UTF-8 or given via Content-Type):
+    if (buffer != null && !buffer.isEmpty()) {
+      assert charsetDecoder == null;
+      decodeBuffer(buffer, map, getCharsetDecoder(charset));
+    }
     return len;
   }
   
-  private static String decodeChars(ByteArrayOutputStream2 stream, long position, CharsetDecoder charsetDecoder) {
+  private static CharsetDecoder getCharsetDecoder(Charset charset) {
+    return charset.newDecoder()
+      .onMalformedInput(CodingErrorAction.REPORT)
+      .onUnmappableCharacter(CodingErrorAction.REPORT);
+  }
+  
+  private static String decodeChars(byte[] bytes, long position, CharsetDecoder charsetDecoder) {
     try {
-      return charsetDecoder.decode(ByteBuffer.wrap(stream.buffer(), 0, stream.size())).toString();
+      return charsetDecoder.decode(ByteBuffer.wrap(bytes)).toString();
     } catch (CharacterCodingException cce) {
       throw new SolrException(ErrorCode.BAD_REQUEST,
         "URLDecoder: Invalid character encoding detected after position " + position +
@@ -323,10 +371,18 @@ public class SolrRequestParsers 
     }
   }
   
-  /** Makes the buffer of ByteArrayOutputStream available without copy. */
-  static final class ByteArrayOutputStream2 extends ByteArrayOutputStream {
-    byte[] buffer() {
-      return buf;
+  private static void decodeBuffer(final LinkedList<Object> input, final Map<String,String[]> map, CharsetDecoder charsetDecoder) {
+    for (final Iterator<Object> it = input.iterator(); it.hasNext(); ) {
+      final byte[] keyBytes = (byte[]) it.next();
+      it.remove();
+      final Long keyPos = (Long) it.next();
+      it.remove();
+      final byte[] valueBytes = (byte[]) it.next();
+      it.remove();
+      final Long valuePos = (Long) it.next();
+      it.remove();
+      MultiMapSolrParams.addParam(decodeChars(keyBytes, keyPos.longValue(), charsetDecoder),
+          decodeChars(valueBytes, valuePos.longValue(), charsetDecoder), map);
     }
   }
   
@@ -361,281 +417,272 @@ public class SolrRequestParsers 
   public void setAddRequestHeadersToContext(boolean addRequestHeadersToContext) {
     this.addHttpRequestToContext = addRequestHeadersToContext;
   }
-}
 
-//-----------------------------------------------------------------
-//-----------------------------------------------------------------
+  //-----------------------------------------------------------------
+  //-----------------------------------------------------------------
 
-// I guess we don't really even need the interface, but i'll keep it here just for kicks
-interface SolrRequestParser 
-{
-  public SolrParams parseParamsAndFillStreams(
-    final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception;
-}
+  // I guess we don't really even need the interface, but i'll keep it here just for kicks
+  interface SolrRequestParser 
+  {
+    public SolrParams parseParamsAndFillStreams(
+      final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception;
+  }
 
 
-//-----------------------------------------------------------------
-//-----------------------------------------------------------------
-
-/**
- * The simple parser just uses the params directly, does not support POST URL-encoded forms
- */
-class SimpleRequestParser implements SolrRequestParser
-{
-  @Override
-  public SolrParams parseParamsAndFillStreams( 
-      final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+  //-----------------------------------------------------------------
+  //-----------------------------------------------------------------
+
+  /**
+   * The simple parser just uses the params directly, does not support POST URL-encoded forms
+   */
+  static class SimpleRequestParser implements SolrRequestParser
   {
-    return SolrRequestParsers.parseQueryString(req.getQueryString());
+    @Override
+    public SolrParams parseParamsAndFillStreams( 
+        final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+    {
+      return parseQueryString(req.getQueryString());
+    }
   }
-}
 
-/**
- * Wrap an HttpServletRequest as a ContentStream
- */
-class HttpRequestContentStream extends ContentStreamBase
-{
-  private final HttpServletRequest req;
-  
-  public HttpRequestContentStream( HttpServletRequest req ) {
-    this.req = req;
-    
-    contentType = req.getContentType();
-    // name = ???
-    // sourceInfo = ???
+  /**
+   * Wrap an HttpServletRequest as a ContentStream
+   */
+  static class HttpRequestContentStream extends ContentStreamBase
+  {
+    private final HttpServletRequest req;
     
-    String v = req.getHeader( "Content-Length" );
-    if( v != null ) {
-      size = Long.valueOf( v );
+    public HttpRequestContentStream( HttpServletRequest req ) {
+      this.req = req;
+      
+      contentType = req.getContentType();
+      // name = ???
+      // sourceInfo = ???
+      
+      String v = req.getHeader( "Content-Length" );
+      if( v != null ) {
+        size = Long.valueOf( v );
+      }
     }
-  }
 
-  @Override
-  public InputStream getStream() throws IOException {
-    return req.getInputStream();
+    @Override
+    public InputStream getStream() throws IOException {
+      return req.getInputStream();
+    }
   }
-}
 
 
-/**
- * Wrap a FileItem as a ContentStream
- */
-class FileItemContentStream extends ContentStreamBase
-{
-  private final FileItem item;
-  
-  public FileItemContentStream( FileItem f )
+  /**
+   * Wrap a FileItem as a ContentStream
+   */
+  static class FileItemContentStream extends ContentStreamBase
   {
-    item = f;
-    contentType = item.getContentType();
-    name = item.getName();
-    sourceInfo = item.getFieldName();
-    size = item.getSize();
-  }
-    
-  @Override
-  public InputStream getStream() throws IOException {
-    return item.getInputStream();
-  }
-}
-
-/**
- * The raw parser just uses the params directly
- */
-class RawRequestParser implements SolrRequestParser
-{
-  @Override
-  public SolrParams parseParamsAndFillStreams( 
-      final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+    private final FileItem item;
+    
+    public FileItemContentStream( FileItem f )
+    {
+      item = f;
+      contentType = item.getContentType();
+      name = item.getName();
+      sourceInfo = item.getFieldName();
+      size = item.getSize();
+    }
+      
+    @Override
+    public InputStream getStream() throws IOException {
+      return item.getInputStream();
+    }
+  }
+
+  /**
+   * The raw parser just uses the params directly
+   */
+  static class RawRequestParser implements SolrRequestParser
   {
-    streams.add( new HttpRequestContentStream( req ) );
-    return SolrRequestParsers.parseQueryString( req.getQueryString() );
+    @Override
+    public SolrParams parseParamsAndFillStreams( 
+        final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+    {
+      streams.add( new HttpRequestContentStream( req ) );
+      return parseQueryString( req.getQueryString() );
+    }
   }
-}
 
 
 
-/**
- * Extract Multipart streams
- */
-class MultipartRequestParser implements SolrRequestParser
-{
-  private final int uploadLimitKB;
-  
-  public MultipartRequestParser( int limit )
-  {
-    uploadLimitKB = limit;
-  }
-  
-  @Override
-  public SolrParams parseParamsAndFillStreams( 
-      final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+  /**
+   * Extract Multipart streams
+   */
+  static class MultipartRequestParser implements SolrRequestParser
   {
-    if( !ServletFileUpload.isMultipartContent(req) ) {
-      throw new SolrException( ErrorCode.BAD_REQUEST, "Not multipart content! "+req.getContentType() );
-    }
+    private final int uploadLimitKB;
     
-    MultiMapSolrParams params = SolrRequestParsers.parseQueryString( req.getQueryString() );
+    public MultipartRequestParser( int limit )
+    {
+      uploadLimitKB = limit;
+    }
     
-    // Create a factory for disk-based file items
-    DiskFileItemFactory factory = new DiskFileItemFactory();
+    @Override
+    public SolrParams parseParamsAndFillStreams( 
+        final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+    {
+      if( !ServletFileUpload.isMultipartContent(req) ) {
+        throw new SolrException( ErrorCode.BAD_REQUEST, "Not multipart content! "+req.getContentType() );
+      }
+      
+      MultiMapSolrParams params = parseQueryString( req.getQueryString() );
+      
+      // Create a factory for disk-based file items
+      DiskFileItemFactory factory = new DiskFileItemFactory();
 
-    // Set factory constraints
-    // TODO - configure factory.setSizeThreshold(yourMaxMemorySize);
-    // TODO - configure factory.setRepository(yourTempDirectory);
-
-    // Create a new file upload handler
-    ServletFileUpload upload = new ServletFileUpload(factory);
-    upload.setSizeMax( ((long) uploadLimitKB) * 1024L );
-
-    // Parse the request
-    List items = upload.parseRequest(req);
-    Iterator iter = items.iterator();
-    while (iter.hasNext()) {
-        FileItem item = (FileItem) iter.next();
-
-        // If its a form field, put it in our parameter map
-        if (item.isFormField()) {
-          MultiMapSolrParams.addParam( 
-            item.getFieldName(), 
-            item.getString(), params.getMap() );
-        }
-        // Add the stream
-        else { 
-          streams.add( new FileItemContentStream( item ) );
-        }
+      // Set factory constraints
+      // TODO - configure factory.setSizeThreshold(yourMaxMemorySize);
+      // TODO - configure factory.setRepository(yourTempDirectory);
+
+      // Create a new file upload handler
+      ServletFileUpload upload = new ServletFileUpload(factory);
+      upload.setSizeMax( ((long) uploadLimitKB) * 1024L );
+
+      // Parse the request
+      List items = upload.parseRequest(req);
+      Iterator iter = items.iterator();
+      while (iter.hasNext()) {
+          FileItem item = (FileItem) iter.next();
+
+          // If its a form field, put it in our parameter map
+          if (item.isFormField()) {
+            MultiMapSolrParams.addParam( 
+              item.getFieldName(), 
+              item.getString(), params.getMap() );
+          }
+          // Add the stream
+          else { 
+            streams.add( new FileItemContentStream( item ) );
+          }
+      }
+      return params;
     }
-    return params;
   }
-}
 
 
-/**
- * Extract application/x-www-form-urlencoded form data for POST requests
- */
-class FormDataRequestParser implements SolrRequestParser
-{
-  private final int uploadLimitKB;
-  
-  public FormDataRequestParser( int limit )
-  {
-    uploadLimitKB = limit;
-  }
-  
-  @Override
-  public SolrParams parseParamsAndFillStreams( 
-      final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+  /**
+   * Extract application/x-www-form-urlencoded form data for POST requests
+   */
+  static class FormDataRequestParser implements SolrRequestParser
   {
-    if (!isFormData(req)) {
-      throw new SolrException( ErrorCode.BAD_REQUEST, "Not application/x-www-form-urlencoded content: "+req.getContentType() );
-    }
-    
-    final Map<String,String[]> map = new HashMap<String, String[]>();
+    private final int uploadLimitKB;
     
-    // also add possible URL parameters and include into the map (parsed using UTF-8):
-    final String qs = req.getQueryString();
-    if (qs != null) {
-      SolrRequestParsers.parseQueryString(qs, map);
+    public FormDataRequestParser( int limit )
+    {
+      uploadLimitKB = limit;
     }
     
-    // may be -1, so we check again later. But if its already greater we can stop processing!
-    final long totalLength = req.getContentLength();
-    final long maxLength = ((long) uploadLimitKB) * 1024L;
-    if (totalLength > maxLength) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, "application/x-www-form-urlencoded content length (" +
-        totalLength + " bytes) exceeds upload limit of " + uploadLimitKB + " KB");
+    @Override
+    public SolrParams parseParamsAndFillStreams( 
+        final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+    {
+      if (!isFormData(req)) {
+        throw new SolrException( ErrorCode.BAD_REQUEST, "Not application/x-www-form-urlencoded content: "+req.getContentType() );
+      }
+      
+      final Map<String,String[]> map = new HashMap<String, String[]>();
+      
+      // also add possible URL parameters and include into the map (parsed using UTF-8):
+      final String qs = req.getQueryString();
+      if (qs != null) {
+        parseQueryString(qs, map);
+      }
+      
+      // may be -1, so we check again later. But if its already greater we can stop processing!
+      final long totalLength = req.getContentLength();
+      final long maxLength = ((long) uploadLimitKB) * 1024L;
+      if (totalLength > maxLength) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "application/x-www-form-urlencoded content length (" +
+          totalLength + " bytes) exceeds upload limit of " + uploadLimitKB + " KB");
+      }
+      
+      // get query String from request body, using the charset given in content-type:
+      final String cs = ContentStreamBase.getCharsetFromContentType(req.getContentType());
+      final Charset charset = (cs == null) ? IOUtils.CHARSET_UTF_8 : Charset.forName(cs);
+      InputStream in = null;
+      try {
+        in = req.getInputStream();
+        final long bytesRead = parseFormDataContent(FastInputStream.wrap(in), maxLength, charset, map, false);
+        if (bytesRead == 0L && totalLength > 0L) {
+          throw getParameterIncompatibilityException();
+        }
+      } catch (IOException ioe) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, ioe);
+      } catch (IllegalStateException ise) {
+        throw (SolrException) getParameterIncompatibilityException().initCause(ise);
+      } finally {
+        IOUtils.closeWhileHandlingException(in);
+      }
+      
+      return new MultiMapSolrParams(map);
     }
     
-    // get query String from request body, using the charset given in content-type:
-    final String cs = ContentStreamBase.getCharsetFromContentType(req.getContentType());
-    final Charset charset = (cs == null) ? IOUtils.CHARSET_UTF_8 : Charset.forName(cs);
-    InputStream in = null;
-    try {
-      in = req.getInputStream();
-      final long bytesRead = SolrRequestParsers.parseFormDataContent(FastInputStream.wrap(in), maxLength, charset, map);
-      if (bytesRead == 0L && totalLength > 0L) {
-        throw getParameterIncompatibilityException();
-      }
-    } catch (IOException ioe) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, ioe);
-    } catch (IllegalStateException ise) {
-      throw (SolrException) getParameterIncompatibilityException().initCause(ise);
-    } finally {
-      IOUtils.closeWhileHandlingException(in);
+    private SolrException getParameterIncompatibilityException() {
+      return new SolrException(ErrorCode.SERVER_ERROR,
+        "Solr requires that request parameters sent using application/x-www-form-urlencoded " +
+        "content-type can be read through the request input stream. Unfortunately, the " +
+        "stream was empty / not available. This may be caused by another servlet filter calling " +
+        "ServletRequest.getParameter*() before SolrDispatchFilter, please remove it."
+      );
     }
     
-    return new MultiMapSolrParams(map);
-  }
-  
-  private SolrException getParameterIncompatibilityException() {
-    return new SolrException(ErrorCode.SERVER_ERROR,
-      "Solr requires that request parameters sent using application/x-www-form-urlencoded " +
-      "content-type can be read through the request input stream. Unfortunately, the " +
-      "stream was empty / not available. This may be caused by another servlet filter calling " +
-      "ServletRequest.getParameter*() before SolrDispatchFilter, please remove it."
-    );
-  }
-  
-  public boolean isFormData(HttpServletRequest req) {
-    String contentType = req.getContentType();
-    if (contentType != null) {
-      int idx = contentType.indexOf( ';' );
-      if( idx > 0 ) { // remove the charset definition "; charset=utf-8"
-        contentType = contentType.substring( 0, idx );
-      }
-      contentType = contentType.trim();
-      if( "application/x-www-form-urlencoded".equalsIgnoreCase(contentType)) {
-        return true;
+    public boolean isFormData(HttpServletRequest req) {
+      String contentType = req.getContentType();
+      if (contentType != null) {
+        int idx = contentType.indexOf( ';' );
+        if( idx > 0 ) { // remove the charset definition "; charset=utf-8"
+          contentType = contentType.substring( 0, idx );
+        }
+        contentType = contentType.trim();
+        if( "application/x-www-form-urlencoded".equalsIgnoreCase(contentType)) {
+          return true;
+        }
       }
+      return false;
     }
-    return false;
   }
-}
 
 
-/**
- * The default Logic
- */
-class StandardRequestParser implements SolrRequestParser
-{
-  MultipartRequestParser multipart;
-  RawRequestParser raw;
-  FormDataRequestParser formdata;
-  
-  StandardRequestParser(MultipartRequestParser multi, RawRequestParser raw, FormDataRequestParser formdata) 
-  {
-    this.multipart = multi;
-    this.raw = raw;
-    this.formdata = formdata;
-  }
-  
-  @Override
-  public SolrParams parseParamsAndFillStreams( 
-      final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+  /**
+   * The default Logic
+   */
+  static class StandardRequestParser implements SolrRequestParser
   {
-    String method = req.getMethod().toUpperCase(Locale.ROOT);
-    if ("GET".equals(method) || "HEAD".equals(method) 
-        || ("PUT".equals(method) && req.getRequestURI().contains("/schema"))) {
-      return SolrRequestParsers.parseQueryString(req.getQueryString());
-    }
-    if ("POST".equals( method ) ) {
-      if (formdata.isFormData(req)) {
-        return formdata.parseParamsAndFillStreams(req, streams);
-      }
-      if (ServletFileUpload.isMultipartContent(req)) {
-        return multipart.parseParamsAndFillStreams(req, streams);
+    MultipartRequestParser multipart;
+    RawRequestParser raw;
+    FormDataRequestParser formdata;
+    
+    StandardRequestParser(MultipartRequestParser multi, RawRequestParser raw, FormDataRequestParser formdata) 
+    {
+      this.multipart = multi;
+      this.raw = raw;
+      this.formdata = formdata;
+    }
+    
+    @Override
+    public SolrParams parseParamsAndFillStreams( 
+        final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
+    {
+      String method = req.getMethod().toUpperCase(Locale.ROOT);
+      if ("GET".equals(method) || "HEAD".equals(method) 
+          || ("PUT".equals(method) && req.getRequestURI().contains("/schema"))) {
+        return parseQueryString(req.getQueryString());
+      }
+      if ("POST".equals( method ) ) {
+        if (formdata.isFormData(req)) {
+          return formdata.parseParamsAndFillStreams(req, streams);
+        }
+        if (ServletFileUpload.isMultipartContent(req)) {
+          return multipart.parseParamsAndFillStreams(req, streams);
+        }
+        return raw.parseParamsAndFillStreams(req, streams);
       }
-      return raw.parseParamsAndFillStreams(req, streams);
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Unsupported method: " + method + " for request " + req);
     }
-    throw new SolrException(ErrorCode.BAD_REQUEST, "Unsupported method: " + method + " for request " + req);
   }
-}
-
-
-
-
-
-
-
-
-
+}
\ No newline at end of file

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java Tue Aug 13 06:33:29 2013
@@ -202,7 +202,7 @@ public class SpellCheckCollator {
       //then be sure all of the new words have the same optional/required/prohibited status in the query.
       while(indexOfSpace>-1 && indexOfSpace<corr.length()-1) {
         addParenthesis = true;
-        char previousChar = tok.startOffset()>0 ? collation.charAt(tok.startOffset()-1) : ' ';
+        char previousChar = tok.startOffset()>0 ? origQuery.charAt(tok.startOffset()-1) : ' ';
         if(previousChar=='-' || previousChar=='+') {
           corrSb.insert(indexOfSpace + bump, previousChar);
           if(requiredOrProhibited==null) {

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java Tue Aug 13 06:33:29 2013
@@ -33,23 +33,33 @@ import org.apache.solr.spelling.suggest.
 public class FuzzyLookupFactory extends LookupFactory {
 
   /**
+   * If <code>true</code>, maxEdits, minFuzzyLength, transpositions and nonFuzzyPrefix 
+   * will be measured in Unicode code points (actual letters) instead of bytes.
+   */
+  public static final String UNICODE_AWARE = "unicodeAware";
+
+  /**
    * Maximum number of edits allowed, used by {@link LevenshteinAutomata#toAutomaton(int)}
+   * in bytes or Unicode code points (if {@link #UNICODE_AWARE} option is set to true).
    */
   public static final String MAX_EDITS = "maxEdits";
   
   /**
    * If transpositions are allowed, Fuzzy suggestions will be computed based on a primitive 
    * edit operation. If it is false, it will be based on the classic Levenshtein algorithm.
+   * Transpositions of bytes or Unicode code points (if {@link #UNICODE_AWARE} option is set to true).
    */
   public static final String TRANSPOSITIONS = "transpositions";
   
   /**
    * Length of common (non-fuzzy) prefix for the suggestions
+   * in bytes or Unicode code points (if {@link #UNICODE_AWARE} option is set to true).
    */
   public static final String NON_FUZZY_PREFIX = "nonFuzzyPrefix";
   
   /**
    * Minimum length of lookup key before any edits are allowed for the suggestions
+   * in bytes or Unicode code points (if {@link #UNICODE_AWARE} option is set to true).
    */
   public static final String MIN_FUZZY_LENGTH = "minFuzzyLength";
   
@@ -113,9 +123,13 @@ public class FuzzyLookupFactory extends 
     ? Integer.parseInt(params.get(MIN_FUZZY_LENGTH).toString())
     :FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH;
     
+    boolean unicodeAware = (params.get(UNICODE_AWARE) != null)
+    ? Boolean.valueOf(params.get(UNICODE_AWARE).toString())
+    : FuzzySuggester.DEFAULT_UNICODE_AWARE;
+    
     return new FuzzySuggester(indexAnalyzer, queryAnalyzer, options, 
         maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions, maxEdits, 
-        transpositions, nonFuzzyPrefix, minFuzzyLength);
+        transpositions, nonFuzzyPrefix, minFuzzyLength, unicodeAware);
   }
 
   @Override

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java Tue Aug 13 06:33:29 2013
@@ -18,6 +18,7 @@
 package org.apache.solr.update;
 
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexDocument;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.common.SolrException;
@@ -27,10 +28,15 @@ import org.apache.solr.request.SolrQuery
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
 /**
  *
  */
-public class AddUpdateCommand extends UpdateCommand {
+public class AddUpdateCommand extends UpdateCommand implements Iterable<IndexDocument> {
    // optional id in "internal" indexed form... if it is needed and not supplied,
    // it will be obtained from the doc.
    private BytesRef indexedId;
@@ -143,8 +149,64 @@ public class AddUpdateCommand extends Up
     }
     return id;
   }
-  
-   @Override
+
+  public boolean isBlock() {
+    return solrDoc.hasChildDocuments();
+  }
+
+  @Override
+  public Iterator<IndexDocument> iterator() {
+    return new Iterator<IndexDocument>() {
+      Iterator<SolrInputDocument> iter;
+
+      {
+        List<SolrInputDocument> all = flatten(solrDoc);
+
+        SchemaField uniq = req.getSchema().getUniqueKeyField();
+        String idField = getHashableId();
+
+        for (SolrInputDocument sdoc : all) {
+          sdoc.setField("_root_", idField);      // should this be a string or the same type as the ID?
+          // TODO: if possible concurrent modification exception (if SolrInputDocument not cloned and is being forwarded to replicas)
+          // then we could add this field to the generated lucene document instead.
+        }
+
+        iter = all.iterator();
+     }
+
+      @Override
+      public boolean hasNext() {
+        return iter.hasNext();
+      }
+
+      @Override
+      public IndexDocument next() {
+        return DocumentBuilder.toDocument(iter.next(), req.getSchema());
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  private List<SolrInputDocument> flatten(SolrInputDocument root) {
+    List<SolrInputDocument> unwrappedDocs = new ArrayList<SolrInputDocument>();
+    recUnwrapp(unwrappedDocs, root);
+    Collections.reverse(unwrappedDocs);
+    return unwrappedDocs;
+  }
+
+  private void recUnwrapp(List<SolrInputDocument> unwrappedDocs, SolrInputDocument currentDoc) {
+    unwrappedDocs.add(currentDoc);
+    for (SolrInputDocument child : currentDoc.getChildDocuments()) {
+      recUnwrapp(unwrappedDocs, child);
+    }
+  }
+
+
+  @Override
   public String toString() {
      StringBuilder sb = new StringBuilder(super.toString());
      sb.append(",id=").append(getPrintableId());
@@ -153,4 +215,6 @@ public class AddUpdateCommand extends Up
      sb.append('}');
      return sb.toString();
    }
- }
+
+
+}

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java Tue Aug 13 06:33:29 2013
@@ -20,17 +20,8 @@
 
 package org.apache.solr.update;
 
-import java.io.IOException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
@@ -55,12 +46,22 @@ import org.apache.solr.schema.IndexSchem
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.FunctionRangeQuery;
 import org.apache.solr.search.QParser;
-import org.apache.solr.search.SyntaxError;
 import org.apache.solr.search.QueryUtils;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.SyntaxError;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 import org.apache.solr.util.RefCounted;
 
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
+
 /**
  * <code>DirectUpdateHandler2</code> implements an UpdateHandler where documents are added
  * directly to the main Lucene index as opposed to adding to a separate smaller index.
@@ -198,19 +199,23 @@ public class DirectUpdateHandler2 extend
             // normal update
             
             Term updateTerm;
-            Term idTerm = new Term(idField.getName(), cmd.getIndexedId());
+            Term idTerm = new Term(cmd.isBlock() ? "_root_" : idField.getName(), cmd.getIndexedId());
             boolean del = false;
             if (cmd.updateTerm == null) {
               updateTerm = idTerm;
             } else {
+              // this is only used by the dedup update processor
               del = true;
               updateTerm = cmd.updateTerm;
             }
-            
-            Document luceneDocument = cmd.getLuceneDocument();
-            // SolrCore.verbose("updateDocument",updateTerm,luceneDocument,writer);
-            writer.updateDocument(updateTerm, luceneDocument,
-                schema.getAnalyzer());
+
+            if (cmd.isBlock()) {
+              writer.updateDocuments(updateTerm, cmd, schema.getAnalyzer());
+            } else {
+              Document luceneDocument = cmd.getLuceneDocument();
+              // SolrCore.verbose("updateDocument",updateTerm,luceneDocument,writer);
+              writer.updateDocument(updateTerm, luceneDocument, schema.getAnalyzer());
+            }
             // SolrCore.verbose("updateDocument",updateTerm,"DONE");
             
             if (del) { // ensure id remains unique
@@ -233,7 +238,12 @@ public class DirectUpdateHandler2 extend
           
         } else {
           // allow duplicates
-          writer.addDocument(cmd.getLuceneDocument(), schema.getAnalyzer());
+          if (cmd.isBlock()) {
+            writer.addDocuments(cmd, schema.getAnalyzer());
+          } else {
+            writer.addDocument(cmd.getLuceneDocument(), schema.getAnalyzer());
+          }
+
           if (ulog != null) ulog.add(cmd);
         }
         
@@ -432,11 +442,11 @@ public class DirectUpdateHandler2 extend
 
     log.info("start " + cmd);
     
-    IndexReader[] readers = cmd.readers;
-    if (readers != null && readers.length > 0) {
+    List<DirectoryReader> readers = cmd.readers;
+    if (readers != null && readers.size() > 0) {
       RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
       try {
-        iw.get().addIndexes(readers);
+        iw.get().addIndexes(readers.toArray(new IndexReader[readers.size()]));
       } finally {
         iw.decref();
       }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/MergeIndexesCommand.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/MergeIndexesCommand.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/MergeIndexesCommand.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/MergeIndexesCommand.java Tue Aug 13 06:33:29 2013
@@ -17,9 +17,14 @@
 
 package org.apache.solr.update;
 
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.solr.request.SolrQueryRequest;
 
+import java.util.List;
+
 /**
  * A merge indexes command encapsulated in an object.
  *
@@ -27,9 +32,9 @@ import org.apache.solr.request.SolrQuery
  *
  */
 public class MergeIndexesCommand extends UpdateCommand {
-  public DirectoryReader[] readers;
+  public List<DirectoryReader> readers;
 
-  public MergeIndexesCommand(DirectoryReader[] readers, SolrQueryRequest req) {
+  public MergeIndexesCommand(List<DirectoryReader> readers, SolrQueryRequest req) {
     super(req);
     this.readers = readers;
   }
@@ -42,12 +47,13 @@ public class MergeIndexesCommand extends
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder(super.toString());
-    if (readers != null && readers.length > 0) {
-      sb.append(readers[0].directory());
-      for (int i = 1; i < readers.length; i++) {
-        sb.append(",").append(readers[i].directory());
+    Joiner joiner = Joiner.on(",");
+    Iterable<String> directories = Iterables.transform(readers, new Function<DirectoryReader, String>() {
+      public String apply(DirectoryReader reader) {
+        return reader.directory().toString();
       }
-    }
+    });
+    joiner.skipNulls().join(sb, directories);
     sb.append('}');
     return sb.toString();
   }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Tue Aug 13 06:33:29 2013
@@ -69,7 +69,6 @@ public class SolrIndexConfig {
   public final String lockType;
   public final PluginInfo mergePolicyInfo;
   public final PluginInfo mergeSchedulerInfo;
-  public final int termIndexInterval;
   
   public final PluginInfo mergedSegmentWarmerInfo;
   
@@ -95,7 +94,6 @@ public class SolrIndexConfig {
     ramBufferSizeMB = 100;
     writeLockTimeout = -1;
     lockType = LOCK_TYPE_NATIVE;
-    termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
     mergePolicyInfo = null;
     mergeSchedulerInfo = null;
     defaultMergePolicyClassName = TieredMergePolicy.class.getName();
@@ -119,15 +117,19 @@ public class SolrIndexConfig {
       def = new SolrIndexConfig(solrConfig);
     }
 
+    // sanity check: this will throw an error for us if there is more then one
+    // config section
+    Object unused = solrConfig.getNode(prefix, false);
+
     luceneVersion = solrConfig.luceneMatchVersion;
 
     // Assert that end-of-life parameters or syntax is not in our config.
     // Warn for luceneMatchVersion's before LUCENE_36, fail fast above
     assertWarnOrFail("The <mergeScheduler>myclass</mergeScheduler> syntax is no longer supported in solrconfig.xml. Please use syntax <mergeScheduler class=\"myclass\"/> instead.",
-        !((solrConfig.get(prefix+"/mergeScheduler/text()",null) != null) && (solrConfig.get(prefix+"/mergeScheduler/@class",null) == null)),
+        !((solrConfig.getNode(prefix+"/mergeScheduler",false) != null) && (solrConfig.get(prefix+"/mergeScheduler/@class",null) == null)),
         true);
     assertWarnOrFail("The <mergePolicy>myclass</mergePolicy> syntax is no longer supported in solrconfig.xml. Please use syntax <mergePolicy class=\"myclass\"/> instead.",
-        !((solrConfig.get(prefix+"/mergePolicy/text()",null) != null) && (solrConfig.get(prefix+"/mergePolicy/@class",null) == null)),
+        !((solrConfig.getNode(prefix+"/mergePolicy",false) != null) && (solrConfig.get(prefix+"/mergePolicy/@class",null) == null)),
         true);
     assertWarnOrFail("The <luceneAutoCommit>true|false</luceneAutoCommit> parameter is no longer valid in solrconfig.xml.",
         solrConfig.get(prefix+"/luceneAutoCommit", null) == null,
@@ -148,7 +150,10 @@ public class SolrIndexConfig {
     mergeSchedulerInfo = getPluginInfo(prefix + "/mergeScheduler", solrConfig, def.mergeSchedulerInfo);
     mergePolicyInfo = getPluginInfo(prefix + "/mergePolicy", solrConfig, def.mergePolicyInfo);
     
-    termIndexInterval = solrConfig.getInt(prefix + "/termIndexInterval", def.termIndexInterval);
+    String val = solrConfig.get(prefix + "/termIndexInterval", null);
+    if (val != null) {
+      throw new IllegalArgumentException("Illegal parameter 'termIndexInterval'");
+    }
 
     boolean infoStreamEnabled = solrConfig.getBool(prefix + "/infoStream", false);
     if(infoStreamEnabled) {
@@ -198,9 +203,6 @@ public class SolrIndexConfig {
     if (ramBufferSizeMB != -1)
       iwc.setRAMBufferSizeMB(ramBufferSizeMB);
 
-    if (termIndexInterval != -1)
-      iwc.setTermIndexInterval(termIndexInterval);
-
     if (writeLockTimeout != -1)
       iwc.setWriteLockTimeout(writeLockTimeout);
 

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Tue Aug 13 06:33:29 2013
@@ -17,16 +17,6 @@ package org.apache.solr.update.processor
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
@@ -74,6 +64,16 @@ import org.apache.solr.update.VersionInf
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
 // NOT mt-safe... create a new processor for each add thread
@@ -917,7 +917,9 @@ public class DistributedUpdateProcessor 
       outParams.set(DISTRIB_UPDATE_PARAM, DistribPhase.TOLEADER.toString());
 
       SolrParams params = req.getParams();
-      Collection<Slice> slices = coll.getRouter().getSearchSlices(params.get(ShardParams.SHARD_KEYS), params, coll);
+      String route = params.get(ShardParams._ROUTE_);
+      if(route == null) route = params.get(ShardParams.SHARD_KEYS);// deprecated . kept for backcompat
+      Collection<Slice> slices = coll.getRouter().getSearchSlices(route, params, coll);
 
       List<Node> leaders =  new ArrayList<Node>(slices.size());
       for (Slice slice : slices) {

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema-xinclude.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema-xinclude.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema-xinclude.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema-xinclude.xml Tue Aug 13 06:33:29 2013
@@ -1,4 +1,7 @@
 <?xml version="1.0" encoding="UTF-8" ?>
+<!DOCTYPE schema [
+        <!ENTITY schema_entity_include SYSTEM "schema-snippet-types.incl">
+]>
 <!--
  Licensed to the Apache Software Foundation (ASF) under one or more
  contributor license agreements.  See the NOTICE file distributed with
@@ -19,6 +22,7 @@
  <types>
   <fieldType name="string" class="solr.StrField"/>
   <xi:include href="schema-snippet-type.xml" />
+  &schema_entity_include;
  </types>
  <fields>
    <xi:include href="schema-snippet-field.xml" />

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema.xml Tue Aug 13 06:33:29 2013
@@ -441,6 +441,8 @@
 
  <fields>
    <field name="id" type="int" indexed="true" stored="true" multiValued="false" required="false"/>
+   <field name="_root_" type="int" indexed="true" stored="true" multiValued="false" required="false"/>
+
    <field name="signatureField" type="string" indexed="true" stored="false"/>
    <field name="uuid" type="uuid" stored="true" />
    <field name="name" type="nametext" indexed="true" stored="true"/>
@@ -581,6 +583,7 @@
    <field name="store" type="location" indexed="true" stored="true" omitNorms="false"/>
 
    <field name="lower" type="lowertok" indexed="false" stored="true" multiValued="true" />
+   <field name="_route_" type="string" indexed="true" stored="true" multiValued="false" />
 
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
         will be used if the name matches any of the patterns.

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema12.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema12.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema12.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema12.xml Tue Aug 13 06:33:29 2013
@@ -420,6 +420,9 @@
    <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
     <!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
     <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <!-- sub-centimeter accuracy for RPT; distance calcs -->
+    <fieldType name="location_rpt" class="solr.SpatialRecursivePrefixTreeFieldType"
+      geo="true" distErrPct="0.025" maxDistErr="0.00000009" units="degrees" />
 
   <fieldType name="currency" class="solr.CurrencyField" currencyConfig="currency.xml" multiValued="false" />
  </types>
@@ -517,6 +520,7 @@
    <field name="pointD" type="xyd" indexed="true" stored="true" multiValued="false"/>
    <field name="point_hash" type="geohash" indexed="true" stored="true" multiValued="false"/>
    <field name="store" type="location" indexed="true" stored="true"/>
+   <field name="store_rpt" type="location_rpt" indexed="true" stored="false" />
    
    <!-- Test currency field -->
    <field name="amount_c" type="currency" indexed="true" stored="true" multiValued="false"/>
@@ -609,5 +613,6 @@
    <copyField source="subject" dest="text"/>
 
    <copyField source="foo_copysource_*" dest="bar_copydest_*" />
+   <copyField source="store" dest="store_rpt" />
  
 </schema>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema15.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema15.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema15.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema15.xml Tue Aug 13 06:33:29 2013
@@ -523,13 +523,15 @@
    <field name="uniq3" type="string" indexed="true" stored="true"/>
    <field name="nouniq" type="string" indexed="true" stored="true" multiValued="true"/>
 
-   <!-- for versioning -->
-   <field name="_version_" type="long" indexed="true" stored="true"/>
-
-    
+   
    <field name="copyfield_source" type="string" indexed="true" stored="true" multiValued="true"/>
 
+   <!-- for versioning -->
+   <field name="_version_" type="long" indexed="true" stored="true"/>
+   <!-- points to the root document of a block of nested documents -->
+   <field name="_root_" type="string" indexed="true" stored="true"/>
 
+ 
 
    <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false"/>
 
@@ -560,6 +562,7 @@
    <dynamicField name="*_l"  type="long"   indexed="true"  stored="true"/>
    <dynamicField name="*_t"  type="text"    indexed="true"  stored="true"/>
    <dynamicField name="*_tt"  type="text"    indexed="true"  stored="true"/>
+   <dynamicField name="*_ws"  type="nametext" indexed="true" stored="true"/>
    <dynamicField name="*_b"  type="boolean" indexed="true"  stored="true"/>
    <dynamicField name="*_f"  type="float"  indexed="true"  stored="true"/>
    <dynamicField name="*_d"  type="double" indexed="true"  stored="true"/>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml Tue Aug 13 06:33:29 2013
@@ -24,5 +24,6 @@
   <dataDir>${solr.data.dir:}</dataDir>
   <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
+  <requestHandler name="standard" class="solr.StandardRequestHandler" />
+  <requestHandler name="/update" class="solr.UpdateRequestHandler" />
 </config>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml Tue Aug 13 06:33:29 2013
@@ -25,12 +25,27 @@
   <requestHandler name="standard" class="solr.StandardRequestHandler"/>
 
   <indexConfig>
-   <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
-   <deletionPolicy class="solr.SolrDeletionPolicy">
-    <str name="keepOptimizedOnly">true</str>
-    <str name="maxCommitsToKeep">3</str>
-    <str name="maxCommitAge">100MILLISECONDS</str>
-   </deletionPolicy>
+    <!-- we can't include solrconfig.snippet.randomindexconfig.xml because we need
+         to configure an explicit deletion policy, but we still wnat to randomize as much 
+         as possible. 
+    -->
+    <mergePolicy class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <maxIndexingThreads>${solr.tests.maxIndexingThreads}</maxIndexingThreads>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+    
+    <lockType>single</lockType>
+
+
+    <deletionPolicy class="solr.SolrDeletionPolicy">
+      <str name="keepOptimizedOnly">true</str>
+      <str name="maxCommitsToKeep">3</str>
+      <str name="maxCommitAge">100MILLISECONDS</str>
+    </deletionPolicy>
   </indexConfig>
 
 </config>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml Tue Aug 13 06:33:29 2013
@@ -21,7 +21,22 @@
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <indexConfig>
+
+    <!-- we can't include solrconfig.snippet.randomindexconfig.xml because we need
+         to configure an explicit deletion policy, but we still wnat to randomize as much 
+         as possible. 
+    -->
+    <mergePolicy class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
     <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <maxIndexingThreads>${solr.tests.maxIndexingThreads}</maxIndexingThreads>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+    
+    <lockType>single</lockType>
+
     <deletionPolicy class="org.apache.solr.core.FakeDeletionPolicy">
       <str name="var1">value1</str>
 		  <str name="var2">value2</str>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml Tue Aug 13 06:33:29 2013
@@ -25,5 +25,6 @@
     <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
     <maxIndexingThreads>123</maxIndexingThreads>
     <infoStream>true</infoStream>
+    <mergePolicy class="org.apache.solr.util.RandomMergePolicy" />
   </indexConfig>
 </config>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml Tue Aug 13 06:33:29 2013
@@ -22,10 +22,7 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <indexConfig>
-    <lockType>single</lockType>
-    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
-  </indexConfig>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml?rev=1513364&r1=1513363&r2=1513364&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml Tue Aug 13 06:33:29 2013
@@ -21,10 +21,7 @@
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
 
-  <indexConfig>
-    <lockType>single</lockType>
-    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
-  </indexConfig>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>