You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2014/12/14 23:50:23 UTC

svn commit: r1645537 [4/7] - in /lucene/dev/branches/lucene2878: ./ lucene/ lucene/benchmark/ lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/ lucene/core/ lucene/core/src...

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java Sun Dec 14 22:50:20 2014
@@ -85,6 +85,7 @@ import org.apache.solr.util.plugin.SolrC
  */
 public class ExpandComponent extends SearchComponent implements PluginInfoInitialized, SolrCoreAware {
   public static final String COMPONENT_NAME = "expand";
+  private static final int finishingStage = ResponseBuilder.STAGE_GET_FIELDS;
   private PluginInfo info = PluginInfo.EMPTY_INFO;
 
   @Override
@@ -115,13 +116,6 @@ public class ExpandComponent extends Sea
     SolrQueryRequest req = rb.req;
     SolrParams params = req.getParams();
 
-    boolean isShard = params.getBool(ShardParams.IS_SHARD, false);
-    String ids = params.get(ShardParams.IDS);
-
-    if (ids == null && isShard) {
-      return;
-    }
-
     String field = params.get(ExpandParams.EXPAND_FIELD);
     if (field == null) {
       List<Query> filters = rb.getFilters();
@@ -246,8 +240,22 @@ public class ExpandComponent extends Sea
   }
 
   @Override
+  public int distributedProcess(ResponseBuilder rb) throws IOException {
+    if (rb.doExpand && rb.stage < finishingStage) {
+      return finishingStage;
+    }
+    return ResponseBuilder.STAGE_DONE;
+  }
+    
+  @Override
   public void modifyRequest(ResponseBuilder rb, SearchComponent who, ShardRequest sreq) {
-
+    SolrParams params = rb.req.getParams();
+    if (!params.getBool(COMPONENT_NAME, false)) return;
+    if (!rb.onePassDistributedQuery && (sreq.purpose & ShardRequest.PURPOSE_GET_FIELDS) == 0) {
+      sreq.params.set(COMPONENT_NAME, "false");
+    } else {
+      sreq.params.set(COMPONENT_NAME, "true");
+    }
   }
 
   @SuppressWarnings("unchecked")
@@ -285,7 +293,7 @@ public class ExpandComponent extends Sea
       return;
     }
 
-    if (rb.stage != ResponseBuilder.STAGE_GET_FIELDS) {
+    if (rb.stage != finishingStage) {
       return;
     }
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java Sun Dec 14 22:50:20 2014
@@ -292,7 +292,7 @@ public class QueryComponent extends Sear
     // -1 as flag if not set.
     long timeAllowed = params.getLong(CommonParams.TIME_ALLOWED, -1L);
     if (null != rb.getCursorMark() && 0 < timeAllowed) {
-      // fundementally incompatible
+      // fundamentally incompatible
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can not search using both " +
                               CursorMarkParams.CURSOR_MARK_PARAM + " and " + CommonParams.TIME_ALLOWED);
     }

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Sun Dec 14 22:50:20 2014
@@ -37,10 +37,10 @@ import java.util.concurrent.SynchronousQ
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
@@ -74,6 +74,7 @@ import org.apache.solr.common.util.StrUt
 import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.request.IntervalFacets.FacetInterval;
 import org.apache.solr.schema.BoolField;
+import org.apache.solr.schema.DateRangeField;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
@@ -1080,6 +1081,8 @@ public class SimpleFacets {
               (SolrException.ErrorCode.BAD_REQUEST,
                   "Unable to range facet on tried field of unexpected type:" + f);
       }
+    } else if (ft instanceof DateRangeField) {
+      calc = new DateRangeFieldEndpointCalculator(sf, null);
     } else {
       throw new SolrException
           (SolrException.ErrorCode.BAD_REQUEST,
@@ -1420,6 +1423,7 @@ public class SimpleFacets {
   }
   private static class DateRangeEndpointCalculator 
     extends RangeEndpointCalculator<Date> {
+    private static final String TYPE_ERR_MSG = "SchemaField must use field type extending TrieDateField or DateRangeField";
     private final Date now;
     public DateRangeEndpointCalculator(final SchemaField f, 
                                        final Date now) { 
@@ -1427,7 +1431,7 @@ public class SimpleFacets {
       this.now = now;
       if (! (field.getType() instanceof TrieDateField) ) {
         throw new IllegalArgumentException
-          ("SchemaField must use field type extending TrieDateField");
+          (TYPE_ERR_MSG);
       }
     }
     @Override
@@ -1440,6 +1444,36 @@ public class SimpleFacets {
     }
     @Override
     protected Object parseGap(final String rawval) {
+      return rawval;
+    }
+    @Override
+    public Date parseAndAddGap(Date value, String gap) throws java.text.ParseException {
+      final DateMathParser dmp = new DateMathParser();
+      dmp.setNow(value);
+      return dmp.parseMath(gap);
+    }
+  }
+  private static class DateRangeFieldEndpointCalculator
+      extends RangeEndpointCalculator<Date> {
+    private final Date now;
+    public DateRangeFieldEndpointCalculator(final SchemaField f,
+                                       final Date now) {
+      super(f);
+      this.now = now;
+      if (! (field.getType() instanceof DateRangeField) ) {
+        throw new IllegalArgumentException(DateRangeEndpointCalculator.TYPE_ERR_MSG);
+      }
+    }
+    @Override
+    public String formatValue(Date val) {
+      return TrieDateField.formatExternal(val);
+    }
+    @Override
+    protected Date parseVal(String rawval) {
+      return ((DateRangeField)field.getType()).parseMath(now, rawval);
+    }
+    @Override
+    protected Object parseGap(final String rawval) {
       return rawval;
     }
     @Override

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java Sun Dec 14 22:50:20 2014
@@ -24,6 +24,7 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.core.SolrCore;
 
+import java.io.Closeable;
 import java.util.Map;
 import java.util.HashMap;
 
@@ -40,7 +41,7 @@ import java.util.HashMap;
  *
  *
  */
-public abstract class SolrQueryRequestBase implements SolrQueryRequest {
+public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeable {
   protected final SolrCore core;
   protected final SolrParams origParams;
   protected volatile IndexSchema schema;

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java Sun Dec 14 22:50:20 2014
@@ -17,6 +17,19 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
 import com.google.common.base.Throwables;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
@@ -51,19 +64,6 @@ import org.apache.solr.util.SpatialUtils
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-
 /**
  * Abstract base class for Solr FieldTypes based on a Lucene 4 {@link SpatialStrategy}.
  *
@@ -159,7 +159,7 @@ public abstract class AbstractSpatialFie
   @Override
   public List<StorableField> createFields(SchemaField field, Object val, float boost) {
     String shapeStr = null;
-    Shape shape = null;
+    Shape shape;
     if (val instanceof Shape) {
       shape = ((Shape) val);
     } else {
@@ -178,14 +178,17 @@ public abstract class AbstractSpatialFie
     }
 
     if (field.stored()) {
-      if (shapeStr == null)
-        shapeStr = shapeToString(shape);
-      result.add(new StoredField(field.getName(), shapeStr));
+      result.add(new StoredField(field.getName(), getStoredValue(shape, shapeStr)));
     }
 
     return result;
   }
 
+  /** Called by {@link #createFields(SchemaField, Object, float)} to get the stored value. */
+  protected String getStoredValue(Shape shape, String shapeStr) {
+    return (shapeStr == null) ? shapeToString(shape) : shapeStr;
+  }
+
   protected Shape parseShape(String str) {
     if (str.length() == 0)
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "empty string shape");

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/schema/DateRangeField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/schema/DateRangeField.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/schema/DateRangeField.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/schema/DateRangeField.java Sun Dec 14 22:50:20 2014
@@ -17,24 +17,30 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import java.text.ParseException;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
 import com.spatial4j.core.shape.Shape;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.NumberRangePrefixTreeStrategy;
 import org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree;
+import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.NRShape;
+import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.query.SpatialOperation;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.search.QParser;
-
-import java.text.ParseException;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
+import org.apache.solr.search.SyntaxError;
 
 /**
+ * A field for indexed dates and date ranges. It's mostly compatible with TrieDateField.
+ *
  * @see NumberRangePrefixTreeStrategy
  * @see DateRangePrefixTree
  */
@@ -42,7 +48,9 @@ public class DateRangeField extends Abst
 
   private static final String OP_PARAM = "op";//local-param to resolve SpatialOperation
 
-  private final DateRangePrefixTree tree = DateRangePrefixTree.INSTANCE;
+  private static final DateRangePrefixTree tree = DateRangePrefixTree.INSTANCE;
+
+  private static final TrieDateField trieDateField = new TrieDateField();//used for utility methods
 
   @Override
   protected void init(IndexSchema schema, Map<String, String> args) {
@@ -61,24 +69,74 @@ public class DateRangeField extends Abst
 
   @Override
   public List<StorableField> createFields(SchemaField field, Object val, float boost) {
-    if (val instanceof Date || val instanceof Calendar)//From URP
-      val = tree.toShape(val);
+    if (val instanceof Date || val instanceof Calendar)//From URP?
+      val = tree.toUnitShape(val);
     return super.createFields(field, val, boost);
   }
 
   @Override
-  protected Shape parseShape(String str) {
-    try {
-      return tree.parseShape(str);
-    } catch (ParseException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Couldn't parse date because: "+ e.getMessage(), e);
+  protected String getStoredValue(Shape shape, String shapeStr) {
+    if (shape instanceof UnitNRShape) {
+      UnitNRShape unitShape = (UnitNRShape) shape;
+      if (unitShape.getLevel() == tree.getMaxLevels()) {
+        //fully precise date. We can be fully compatible with TrieDateField.
+        Date date = tree.toCalendar(unitShape).getTime();
+        return TrieDateField.formatExternal(date);
+      }
     }
+    return (shapeStr == null ? shape.toString() : shapeStr);//we don't normalize ranges here; should we?
+  }
+
+  @Override
+  protected NRShape parseShape(String str) {
+    if (str.contains(" TO ")) {
+      //TODO parsing range syntax doesn't support DateMath on either side or exclusive/inclusive
+      try {
+        return tree.parseShape(str);
+      } catch (ParseException e) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "Couldn't parse date because: "+ e.getMessage(), e);
+      }
+    } else {
+      return tree.toShape(parseCalendar(str));
+    }
+  }
+
+  private Calendar parseCalendar(String str) {
+    if (str.startsWith("NOW") || str.lastIndexOf('Z') >= 0) {
+      //use Solr standard date format parsing rules.
+      //TODO parse a Calendar instead of a Date, rounded according to DateMath syntax.
+      Date date = trieDateField.parseMath(null, str);
+      Calendar cal = tree.newCal();
+      cal.setTime(date);
+      return cal;
+    } else {
+      try {
+        return tree.parseCalendar(str);
+      } catch (ParseException e) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "Couldn't parse date because: "+ e.getMessage(), e);
+      }
+
+    }
+  }
+
+  /** For easy compatibility with {@link TrieDateField#parseMath(Date, String)}. */
+  public Date parseMath(Date now, String rawval) {
+    return trieDateField.parseMath(now, rawval);
   }
 
   @Override
   protected String shapeToString(Shape shape) {
-    return shape.toString();//generally round-trips for DateRangePrefixTree
+    if (shape instanceof UnitNRShape) {
+      UnitNRShape unitShape = (UnitNRShape) shape;
+      if (unitShape.getLevel() == tree.getMaxLevels()) {
+        //fully precise date. We can be fully compatible with TrieDateField.
+        Date date = tree.toCalendar(unitShape).getTime();
+        return TrieDateField.formatExternal(date);
+      }
+    }
+    return shape.toString();//range shape
   }
 
   @Override
@@ -96,15 +154,38 @@ public class DateRangeField extends Abst
   }
 
   @Override
-  public Query getRangeQuery(QParser parser, SchemaField field, String part1, String part2, boolean minInclusive, boolean maxInclusive) {
-    if (!minInclusive || !maxInclusive)
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "exclusive range boundary not supported");
-    if (part1 == null)
-      part1 = "*";
-    if (part2 == null)
-      part2 = "*";
-    Shape shape = tree.toRangeShape(parseShape(part1), parseShape(part2));
+  public Query getRangeQuery(QParser parser, SchemaField field, String startStr, String endStr, boolean minInclusive, boolean maxInclusive) {
+    if (parser == null) {//null when invoked by SimpleFacets.  But getQueryFromSpatialArgs expects to get localParams.
+      final SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
+      parser = new QParser("", null, requestInfo.getReq().getParams(), requestInfo.getReq()) {
+        @Override
+        public Query parse() throws SyntaxError {
+          throw new IllegalStateException();
+        }
+      };
+    }
+
+    Calendar startCal;
+    if (startStr == null) {
+      startCal = tree.newCal();
+    } else {
+      startCal = parseCalendar(startStr);
+      if (!minInclusive) {
+        startCal.add(Calendar.MILLISECOND, 1);
+      }
+    }
+    Calendar endCal;
+    if (endStr == null) {
+      endCal = tree.newCal();
+    } else {
+      endCal = parseCalendar(endStr);
+      if (!maxInclusive) {
+        endCal.add(Calendar.MILLISECOND, -1);
+      }
+    }
+    Shape shape = tree.toRangeShape(tree.toShape(startCal), tree.toShape(endCal));
     SpatialArgs spatialArgs = new SpatialArgs(SpatialOperation.Intersects, shape);
     return getQueryFromSpatialArgs(parser, field, spatialArgs);
   }
+
 }

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java Sun Dec 14 22:50:20 2014
@@ -681,10 +681,7 @@ public class SolrRequestParsers
         if (ServletFileUpload.isMultipartContent(req)) {
           return multipart.parseParamsAndFillStreams(req, streams);
         }
-        if (req.getContentType() != null) {
-          return raw.parseParamsAndFillStreams(req, streams);
-        }
-        throw new SolrException(ErrorCode.UNSUPPORTED_MEDIA_TYPE, "Must specify a Content-Type header with POST requests");
+        return raw.parseParamsAndFillStreams(req, streams);
       }
       throw new SolrException(ErrorCode.BAD_REQUEST, "Unsupported method: " + method + " for request " + req);
     }

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java Sun Dec 14 22:50:20 2014
@@ -793,11 +793,7 @@ public class DirectUpdateHandler2 extend
       }
 
       if (writer != null) {
-        try {
-          if (indexWriterCloseWaitsForMerges) writer.waitForMerges();
-        } finally {
-          writer.close();
-        }
+        writer.close();
       }
 
     } finally {

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Sun Dec 14 22:50:20 2014
@@ -81,8 +81,6 @@ public class SolrIndexConfig implements
   public final static String LOCK_TYPE_SINGLE = "single";
   public final static String LOCK_TYPE_NONE   = "none";
 
-  public final boolean checkIntegrityAtMerge;
-
   /**
    * Internal constructor for setting defaults based on Lucene Version
    */
@@ -101,7 +99,6 @@ public class SolrIndexConfig implements
     mergeSchedulerInfo = null;
     defaultMergePolicyClassName = TieredMergePolicy.class.getName();
     mergedSegmentWarmerInfo = null;
-    checkIntegrityAtMerge = false;
   }
   
   /**
@@ -174,7 +171,9 @@ public class SolrIndexConfig implements
       throw new IllegalArgumentException("Supplying a mergedSegmentWarmer will do nothing since nrtMode is false");
     }
 
-    checkIntegrityAtMerge = solrConfig.getBool(prefix + "/checkIntegrityAtMerge", def.checkIntegrityAtMerge);
+    assertWarnOrFail("Begining with Solr 5.0, <checkIntegrityAtMerge> option is no longer supported and should be removed from solrconfig.xml (these integrity checks are now automatic)",
+                     (null == solrConfig.getNode(prefix+"/checkIntegrityAtMerge",false)),
+                     true);
   }
   @Override
   public Map<String, Object> toMap() {

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java Sun Dec 14 22:50:20 2014
@@ -17,6 +17,8 @@
 
 package org.apache.solr.update.processor;
 
+import java.util.Collection;
+
 import org.apache.solr.common.SolrInputField;
 
 import org.slf4j.Logger;
@@ -61,8 +63,10 @@ public abstract class FieldValueMutating
   
   @Override
   protected final SolrInputField mutate(final SolrInputField src) {
+    Collection<Object> values = src.getValues();
+    if(values == null) return src;//don't mutate
     SolrInputField result = new SolrInputField(src.getName());
-    for (final Object srcVal : src.getValues()) {
+    for (final Object srcVal : values) {
       final Object destVal = mutateValue(srcVal);
       if (DELETE_VALUE_SINGLETON == destVal) { 
         /* NOOP */

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java Sun Dec 14 22:50:20 2014
@@ -32,6 +32,8 @@ import java.net.MalformedURLException;
 import java.net.ProtocolException;
 import java.net.URL;
 import java.net.URLEncoder;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.text.SimpleDateFormat;
@@ -581,14 +583,14 @@ public class SimplePostTool {
           URL postUrl = new URL(appendParam(solrUrl.toString(), 
               "literal.id="+URLEncoder.encode(u.toString(),"UTF-8") +
               "&literal.url="+URLEncoder.encode(u.toString(),"UTF-8")));
-          boolean success = postData(new ByteArrayInputStream(result.content), null, out, result.contentType, postUrl);
+          boolean success = postData(new ByteArrayInputStream(result.content.array(), result.content.arrayOffset(),result.content.limit() ), null, out, result.contentType, postUrl);
           if (success) {
             info("POSTed web resource "+u+" (depth: "+level+")");
             Thread.sleep(delay * 1000);
             numPages++;
             // Pull links from HTML pages only
             if(recursive > level && result.contentType.equals("text/html")) {
-              Set<URL> children = pageFetcher.getLinksFromWebPage(u, new ByteArrayInputStream(result.content), result.contentType, postUrl);
+              Set<URL> children = pageFetcher.getLinksFromWebPage(u, new ByteArrayInputStream(result.content.array(), result.content.arrayOffset(), result.content.limit()), result.contentType, postUrl);
               subStack.addAll(children);
             }
           } else {
@@ -609,23 +611,35 @@ public class SimplePostTool {
     }
     return numPages;    
   }
+  public static class BAOS extends ByteArrayOutputStream {
+    public ByteBuffer getByteBuffer() {
+      return ByteBuffer.wrap(super.buf,0,super.count);
+    }
+  }
+  public static ByteBuffer inputStreamToByteArray(InputStream is) throws IOException {
+    return inputStreamToByteArray(is,Integer.MAX_VALUE);
+
+  }
 
   /**
    * Reads an input stream into a byte array
+   *
    * @param is the input stream
    * @return the byte array
    * @throws IOException If there is a low-level I/O error.
    */
-  protected byte[] inputStreamToByteArray(InputStream is) throws IOException {
-    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+  public static ByteBuffer inputStreamToByteArray(InputStream is, long maxSize) throws IOException {
+    BAOS bos =  new BAOS();
+    long sz = 0;
     int next = is.read();
     while (next > -1) {
-        bos.write(next);
-        next = is.read();
+      if(++sz > maxSize) throw new BufferOverflowException();
+      bos.write(next);
+      next = is.read();
     }
     bos.flush();
     is.close();
-    return bos.toByteArray();
+    return bos.getByteBuffer();
   }
 
   /**
@@ -1198,6 +1212,6 @@ public class SimplePostTool {
     int httpStatus = 200;
     String contentType = "text/html";
     URL redirectUrl = null;
-    byte[] content;
+    ByteBuffer content;
   }
 }

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/analysisconfs/analysis-err-schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/analysisconfs/analysis-err-schema.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/analysisconfs/analysis-err-schema.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/analysisconfs/analysis-err-schema.xml Sun Dec 14 22:50:20 2014
@@ -22,14 +22,14 @@
 
 <schema name="test" version="1.5">
   <types>
-    <fieldtype name="long" class="solr.TrieLongField" stored="true" indexed="true" />
-    <fieldtype name="int" class="solr.TrieIntField" precisionStep="0"/>
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="long" class="solr.TrieLongField" stored="true" indexed="true" />
+    <fieldType name="int" class="solr.TrieIntField" precisionStep="0"/>
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
         <filter class="org.apache.solr.analysis.ThrowingMockTokenFilterFactory" exceptionClassName="java.lang.RuntimeException"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
  </types>
 
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-dup-field.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-dup-field.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-dup-field.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-dup-field.xml Sun Dec 14 22:50:20 2014
@@ -19,7 +19,7 @@
 <schema name="bad-schema-dup-field" version="1.0">
   <types>
     <fieldType name="string" class="solr.StrField"/>
-    <fieldtype name="text" class="solr.TextField" />
+    <fieldType name="text" class="solr.TextField" />
  </types>
 
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-dup-fieldType.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-dup-fieldType.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-dup-fieldType.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-dup-fieldType.xml Sun Dec 14 22:50:20 2014
@@ -19,7 +19,7 @@
 <schema name="bad-schema-dup-fieldType" version="1.0">
   <types>
     
-    <fieldtype name="text" class="solr.TextField" />
+    <fieldType name="text" class="solr.TextField" />
     <fieldType name="string" class="solr.StrField"/>
     
     <!-- BEGIN BAD STUFF -->

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-misplaced-asterisk-copyfield-dest-should-fail-test.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-misplaced-asterisk-copyfield-dest-should-fail-test.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-misplaced-asterisk-copyfield-dest-should-fail-test.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-misplaced-asterisk-copyfield-dest-should-fail-test.xml Sun Dec 14 22:50:20 2014
@@ -18,7 +18,7 @@
 
 <schema name="test" version="1.5">
   <types>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   </types>
   <fields>
     <field name="text" type="string" indexed="true" stored="true"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-misplaced-asterisk-copyfield-source-should-fail-test.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-misplaced-asterisk-copyfield-source-should-fail-test.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-misplaced-asterisk-copyfield-source-should-fail-test.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-misplaced-asterisk-copyfield-source-should-fail-test.xml Sun Dec 14 22:50:20 2014
@@ -18,7 +18,7 @@
 
 <schema name="test" version="1.5">
   <types>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   </types>
   <fields>
     <field name="text" type="string" indexed="true" stored="true"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-multiple-asterisk-copyfield-dest-should-fail-test.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-multiple-asterisk-copyfield-dest-should-fail-test.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-multiple-asterisk-copyfield-dest-should-fail-test.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-multiple-asterisk-copyfield-dest-should-fail-test.xml Sun Dec 14 22:50:20 2014
@@ -18,7 +18,7 @@
 
 <schema name="test" version="1.5">
   <types>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   </types>
   <fields>
     <field name="text" type="string" indexed="true" stored="true"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-multiple-asterisk-copyfield-source-should-fail-test.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-multiple-asterisk-copyfield-source-should-fail-test.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-multiple-asterisk-copyfield-source-should-fail-test.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-multiple-asterisk-copyfield-source-should-fail-test.xml Sun Dec 14 22:50:20 2014
@@ -18,7 +18,7 @@
 
 <schema name="test" version="1.5">
   <types>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   </types>
   <fields>
     <field name="text" type="string" indexed="true" stored="true"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-non-glob-copyfield-source-matching-nothing-should-fail-test.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-non-glob-copyfield-source-matching-nothing-should-fail-test.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-non-glob-copyfield-source-matching-nothing-should-fail-test.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/bad-schema-non-glob-copyfield-source-matching-nothing-should-fail-test.xml Sun Dec 14 22:50:20 2014
@@ -18,7 +18,7 @@
 
 <schema name="test" version="1.5">
   <types>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   </types>
 
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-add-schema-fields-update-processor.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-add-schema-fields-update-processor.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-add-schema-fields-update-processor.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-add-schema-fields-update-processor.xml Sun Dec 14 22:50:20 2014
@@ -23,8 +23,8 @@
   <fieldType name="tlong" class="solr.TrieLongField" precisionStep="8" multiValued="true" positionIncrementGap="0"/>
   <fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" multiValued="true" positionIncrementGap="0"/>
   <fieldType name="tdate" class="solr.TrieDateField" precisionStep="6" multiValued="true" positionIncrementGap="0"/>
-  <fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true" multiValued="true"/>
-  <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+  <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true" multiValued="true"/>
+  <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   <fieldType name="long" class="solr.TrieLongField" precisionStep="0" positionIncrementGap="0"/>
   <fieldType name="text" class="solr.TextField" multiValued="true" positionIncrementGap="100">
     <analyzer>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml Sun Dec 14 22:50:20 2014
@@ -31,7 +31,7 @@
     <!-- field type definitions... note that the "name" attribute is
          just a label to be used by field definitions.  The "class"
          attribute and any other attributes determine the real type and
-         behavior of the fieldtype.
+         behavior of the fieldType.
       -->
 
     <!--
@@ -58,14 +58,14 @@
     <fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" positionIncrementGap="0"/>
 
 
-    <fieldtype name="binary" class="solr.BinaryField"/>
-    <fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="binary" class="solr.BinaryField"/>
+    <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
 
     <!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
          seconds part (.999) is optional.
       -->
-    <fieldtype name="date" class="solr.TrieDateField" sortMissingLast="true"/>
+    <fieldType name="date" class="solr.TrieDateField" sortMissingLast="true"/>
 
 
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-class-name-shortening-on-serialization.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-class-name-shortening-on-serialization.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-class-name-shortening-on-serialization.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-class-name-shortening-on-serialization.xml Sun Dec 14 22:50:20 2014
@@ -17,23 +17,23 @@
 -->
 <schema name="test-class-name-shortening-on-serialization" version="1.5">
   <types>
-    <fieldtype name="fullClassNames" class="org.apache.solr.schema.TextField">
+    <fieldType name="fullClassNames" class="org.apache.solr.schema.TextField">
       <analyzer>
         <charFilter class="org.apache.solr.analysis.MockCharFilterFactory" remainder="0"/>
         <tokenizer class="org.apache.solr.analysis.MockTokenizerFactory"/>
         <filter class="org.apache.solr.analysis.MockTokenFilterFactory" stopset="empty"/>
       </analyzer>
       <similarity class="org.apache.lucene.misc.SweetSpotSimilarity"/>
-    </fieldtype>
-    <fieldtype name="shortenedClassNames" class="solr.TextField">
+    </fieldType>
+    <fieldType name="shortenedClassNames" class="solr.TextField">
       <analyzer>
         <charFilter class="solr.MockCharFilterFactory" remainder="0"/>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.MockTokenFilterFactory" stopset="empty"/>
       </analyzer>
       <similarity class="solr.SweetSpotSimilarityFactory"/>
-    </fieldtype>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    </fieldType>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   </types>
   <fields>
     <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="true"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-collate-dv.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-collate-dv.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-collate-dv.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-collate-dv.xml Sun Dec 14 22:50:20 2014
@@ -23,19 +23,19 @@
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
     <!-- basic text field -->
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
         <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
     
-    <fieldtype name="sort_ar_t"       class="solr.CollationField" language="ar"/>
-    <fieldtype name="sort_de_t"       class="solr.CollationField" language="de" strength="primary"/>
-    <fieldtype name="sort_tr_canon_t" class="solr.CollationField" language="tr" strength="primary"   decomposition="canonical"/>
-    <fieldtype name="sort_zh_full_t"  class="solr.CollationField" language="zh" strength="identical" decomposition="full"/>
-    <fieldtype name="sort_da_t"       class="solr.CollationField" language="da" strength="primary"/>
-    <fieldtype name="sort_custom_t"   class="solr.CollationField" custom="customrules.dat" strength="primary"/>
+    <fieldType name="sort_ar_t"       class="solr.CollationField" language="ar"/>
+    <fieldType name="sort_de_t"       class="solr.CollationField" language="de" strength="primary"/>
+    <fieldType name="sort_tr_canon_t" class="solr.CollationField" language="tr" strength="primary"   decomposition="canonical"/>
+    <fieldType name="sort_zh_full_t"  class="solr.CollationField" language="zh" strength="identical" decomposition="full"/>
+    <fieldType name="sort_da_t"       class="solr.CollationField" language="da" strength="primary"/>
+    <fieldType name="sort_custom_t"   class="solr.CollationField" custom="customrules.dat" strength="primary"/>
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-collate.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-collate.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-collate.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-collate.xml Sun Dec 14 22:50:20 2014
@@ -23,19 +23,19 @@
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
     <!-- basic text field -->
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
         <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
     
-    <fieldtype name="sort_ar_t"       class="solr.CollationField" language="ar"/>
-    <fieldtype name="sort_de_t"       class="solr.CollationField" language="de" strength="primary"/>
-    <fieldtype name="sort_tr_canon_t" class="solr.CollationField" language="tr" strength="primary"   decomposition="canonical"/>
-    <fieldtype name="sort_zh_full_t"  class="solr.CollationField" language="zh" strength="identical" decomposition="full"/>
-    <fieldtype name="sort_da_t"       class="solr.CollationField" language="da" strength="primary"/>
-    <fieldtype name="sort_custom_t"   class="solr.CollationField" custom="customrules.dat" strength="primary"/>
+    <fieldType name="sort_ar_t"       class="solr.CollationField" language="ar"/>
+    <fieldType name="sort_de_t"       class="solr.CollationField" language="de" strength="primary"/>
+    <fieldType name="sort_tr_canon_t" class="solr.CollationField" language="tr" strength="primary"   decomposition="canonical"/>
+    <fieldType name="sort_zh_full_t"  class="solr.CollationField" language="zh" strength="identical" decomposition="full"/>
+    <fieldType name="sort_da_t"       class="solr.CollationField" language="da" strength="primary"/>
+    <fieldType name="sort_custom_t"   class="solr.CollationField" custom="customrules.dat" strength="primary"/>
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-complex-phrase.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-complex-phrase.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-complex-phrase.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-complex-phrase.xml Sun Dec 14 22:50:20 2014
@@ -20,9 +20,9 @@
 
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
-    <fieldtype name="nametext" class="solr.TextField">
+    <fieldType name="nametext" class="solr.TextField">
       <analyzer class="org.apache.lucene.analysis.standard.StandardAnalyzer"/>
-    </fieldtype>
+    </fieldType>
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml Sun Dec 14 22:50:20 2014
@@ -31,7 +31,7 @@
     <!-- field type definitions... note that the "name" attribute is
          just a label to be used by field definitions.  The "class"
          attribute and any other attributes determine the real type and
-         behavior of the fieldtype.
+         behavior of the fieldType.
       -->
 
     <!--
@@ -58,40 +58,40 @@
     <fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" positionIncrementGap="0"/>
 
     <!-- Field type demonstrating an Analyzer failure -->
-    <fieldtype name="failtype1" class="solr.TextField">
+    <fieldType name="failtype1" class="solr.TextField">
       <analyzer type="index">
           <tokenizer class="solr.MockTokenizerFactory"/>
           <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
           <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
     <!-- Demonstrating ignoreCaseChange -->
-    <fieldtype name="wdf_nocase" class="solr.TextField">
+    <fieldType name="wdf_nocase" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
           <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="0"/>
           <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
 
     <!-- HighlitText optimizes storage for (long) columns which will be highlit -->
-    <fieldtype name="highlittext" class="solr.TextField" compressThreshold="345" />
+    <fieldType name="highlittext" class="solr.TextField" compressThreshold="345" />
 
-    <fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
 
     <!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
          seconds part (.999) is optional.
       -->
-    <fieldtype name="date" class="solr.TrieDateField" sortMissingLast="true"/>
+    <fieldType name="date" class="solr.TrieDateField" sortMissingLast="true"/>
 
     <!-- solr.TextField allows the specification of custom
          text analyzers specified as a tokenizer and a list
          of token filters.
       -->
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
         <filter class="solr.StandardFilterFactory"/>
@@ -99,68 +99,68 @@
         <filter class="solr.StopFilterFactory"/>
         <filter class="solr.PorterStemFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
 
-    <fieldtype name="nametext" class="solr.TextField">
+    <fieldType name="nametext" class="solr.TextField">
       <analyzer class="org.apache.lucene.analysis.core.WhitespaceAnalyzer"/>
-    </fieldtype>
+    </fieldType>
 
-    <fieldtype name="teststop" class="solr.TextField">
+    <fieldType name="teststop" class="solr.TextField">
        <analyzer>
         <tokenizer class="solr.LowerCaseTokenizerFactory"/>
         <filter class="solr.StandardFilterFactory"/>
         <filter class="solr.StopFilterFactory" words="stopwords.txt"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
-    <!-- fieldtypes in this section isolate tokenizers and tokenfilters for testing -->
-    <fieldtype name="lowertok" class="solr.TextField">
+    <!-- fieldTypes in this section isolate tokenizers and tokenfilters for testing -->
+    <fieldType name="lowertok" class="solr.TextField">
       <analyzer><tokenizer class="solr.LowerCaseTokenizerFactory"/></analyzer>
-    </fieldtype>
-    <fieldtype name="keywordtok" class="solr.TextField">
+    </fieldType>
+    <fieldType name="keywordtok" class="solr.TextField">
       <analyzer><tokenizer class="solr.MockTokenizerFactory" pattern="keyword"/></analyzer>
-    </fieldtype>
-    <fieldtype name="standardtok" class="solr.TextField">
+    </fieldType>
+    <fieldType name="standardtok" class="solr.TextField">
       <analyzer><tokenizer class="solr.StandardTokenizerFactory"/></analyzer>
-    </fieldtype>
-    <fieldtype name="lettertok" class="solr.TextField">
+    </fieldType>
+    <fieldType name="lettertok" class="solr.TextField">
       <analyzer><tokenizer class="solr.LetterTokenizerFactory"/></analyzer>
-    </fieldtype>
-    <fieldtype name="whitetok" class="solr.TextField">
+    </fieldType>
+    <fieldType name="whitetok" class="solr.TextField">
       <analyzer><tokenizer class="solr.MockTokenizerFactory"/></analyzer>
-    </fieldtype>
-    <fieldtype name="HTMLstandardtok" class="solr.TextField">
+    </fieldType>
+    <fieldType name="HTMLstandardtok" class="solr.TextField">
       <analyzer>
       <charFilter class="solr.HTMLStripCharFilterFactory"/>
       <tokenizer class="solr.StandardTokenizerFactory"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="HTMLwhitetok" class="solr.TextField">
+    </fieldType>
+    <fieldType name="HTMLwhitetok" class="solr.TextField">
       <analyzer>
       <charFilter class="solr.HTMLStripCharFilterFactory"/>
       <tokenizer class="solr.MockTokenizerFactory"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="standardtokfilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="standardtokfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
         <filter class="solr.StandardFilterFactory"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="standardfilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="standardfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.StandardFilterFactory"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="lowerfilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="lowerfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="patternreplacefilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="patternreplacefilt" class="solr.TextField">
       <analyzer type="index">
         <tokenizer class="solr.MockTokenizerFactory" pattern="keyword"/>
         <filter class="solr.PatternReplaceFilterFactory"
@@ -170,52 +170,52 @@
       <analyzer type="query">
         <tokenizer class="solr.MockTokenizerFactory" pattern="keyword"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="porterfilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="porterfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.PorterStemFilterFactory"/>
       </analyzer>
-    </fieldtype>
-    <!-- fieldtype name="snowballfilt" class="solr.TextField">
+    </fieldType>
+    <!-- fieldType name="snowballfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.SnowballPorterFilterFactory"/>
       </analyzer>
-    </fieldtype -->
-    <fieldtype name="engporterfilt" class="solr.TextField">
+    </fieldType -->
+    <fieldType name="engporterfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.PorterStemFilterFactory"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="custengporterfilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="custengporterfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
         <filter class="solr.PorterStemFilterFactory"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="stopfilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="stopfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.StopFilterFactory" ignoreCase="true"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="custstopfilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="custstopfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.StopFilterFactory" words="stopwords.txt"/>
       </analyzer>
-    </fieldtype>
-    <fieldtype name="lengthfilt" class="solr.TextField">
+    </fieldType>
+    <fieldType name="lengthfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.LengthFilterFactory" min="2" max="5"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
-    <fieldtype name="subword" class="solr.TextField" multiValued="true" positionIncrementGap="100">
+    <fieldType name="subword" class="solr.TextField" multiValued="true" positionIncrementGap="100">
       <analyzer type="index">
           <tokenizer class="solr.MockTokenizerFactory"/>
           <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
@@ -230,10 +230,10 @@
           <filter class="solr.StopFilterFactory"/>
           <filter class="solr.PorterStemFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
     <!-- more flexible in matching skus, but more chance of a false match -->
-    <fieldtype name="skutype1" class="solr.TextField">
+    <fieldType name="skutype1" class="solr.TextField">
       <analyzer type="index">
           <tokenizer class="solr.MockTokenizerFactory"/>
           <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
@@ -244,10 +244,10 @@
           <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
           <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
     <!-- less flexible in matching skus, but less chance of a false match -->
-    <fieldtype name="skutype2" class="solr.TextField">
+    <fieldType name="skutype2" class="solr.TextField">
       <analyzer type="index">
           <tokenizer class="solr.MockTokenizerFactory"/>
           <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
@@ -258,20 +258,20 @@
           <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
           <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
     <!-- less flexible in matching skus, but less chance of a false match -->
-    <fieldtype name="syn" class="solr.TextField">
+    <fieldType name="syn" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
           <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
     
     <!-- Demonstrates How RemoveDuplicatesTokenFilter makes stemmed
          synonyms "better"
       -->
-    <fieldtype name="dedup" class="solr.TextField">
+    <fieldType name="dedup" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
           <filter class="solr.SynonymFilterFactory"
@@ -279,17 +279,17 @@
           <filter class="solr.PorterStemFilterFactory"/>
           <filter class="solr.RemoveDuplicatesTokenFilterFactory" />
       </analyzer>
-    </fieldtype>
+    </fieldType>
 
-    <fieldtype  name="unstored" class="solr.StrField" indexed="true" stored="false"/>
+    <fieldType  name="unstored" class="solr.StrField" indexed="true" stored="false"/>
 
 
-  <fieldtype name="textgap" class="solr.TextField" multiValued="true" positionIncrementGap="100">
+  <fieldType name="textgap" class="solr.TextField" multiValued="true" positionIncrementGap="100">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
           <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-  </fieldtype>
+  </fieldType>
 
  </types>
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-custom-field.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-custom-field.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-custom-field.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-custom-field.xml Sun Dec 14 22:50:20 2014
@@ -20,12 +20,12 @@
   <types>
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
     <fieldType class="org.apache.solr.schema.SortableBinaryField" name="sortable_binary"/>
   </types>
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-customfield.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-customfield.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-customfield.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-customfield.xml Sun Dec 14 22:50:20 2014
@@ -31,7 +31,7 @@
     <!-- field type definitions... note that the "name" attribute is
          just a label to be used by field definitions.  The "class"
          attribute and any other attributes determine the real type and
-         behavior of the fieldtype.
+         behavior of the fieldType.
       -->
 
     <!--

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml Sun Dec 14 22:50:20 2014
@@ -23,8 +23,8 @@
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="date" class="solr.TrieDateField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
-    <fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-distributed-missing-sort.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-distributed-missing-sort.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-distributed-missing-sort.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-distributed-missing-sort.xml Sun Dec 14 22:50:20 2014
@@ -30,19 +30,19 @@
     <fieldType name="double"    class="solr.TrieDoubleField"  precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="tdouble"   class="solr.TrieDoubleField"  precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
 
-    <fieldtype name="date"      class="solr.TrieDateField"    precisionStep="0"/>
-    <fieldtype name="tdate"     class="solr.TrieDateField"    precisionStep="6"/>
+    <fieldType name="date"      class="solr.TrieDateField"    precisionStep="0"/>
+    <fieldType name="tdate"     class="solr.TrieDateField"    precisionStep="6"/>
 
-    <fieldtype name="boolean"   class="solr.BoolField"        sortMissingLast="true"/>
+    <fieldType name="boolean"   class="solr.BoolField"        sortMissingLast="true"/>
 
-    <fieldtype name="int_ml"   class="solr.TrieIntField" precisionStep="0" sortMissingLast="true"/>
-    <fieldtype name="int_mf"   class="solr.TrieIntField" precisionStep="0" sortMissingFirst="true"/>
+    <fieldType name="int_ml"   class="solr.TrieIntField" precisionStep="0" sortMissingLast="true"/>
+    <fieldType name="int_mf"   class="solr.TrieIntField" precisionStep="0" sortMissingFirst="true"/>
 
     <fieldType name="long_ml"   class="solr.TrieLongField"    precisionStep="0" omitNorms="true" positionIncrementGap="0" sortMissingLast="true"/>
     <fieldType name="long_mf"   class="solr.TrieLongField"    precisionStep="0" omitNorms="true" positionIncrementGap="0" sortMissingFirst="true"/>
 
-    <fieldtype name="string_ml" class="solr.StrField"         sortMissingLast="true"/>
-    <fieldtype name="string_mf" class="solr.StrField"         sortMissingFirst="true"/>
+    <fieldType name="string_ml" class="solr.StrField"         sortMissingLast="true"/>
+    <fieldType name="string_mf" class="solr.StrField"         sortMissingFirst="true"/>
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml Sun Dec 14 22:50:20 2014
@@ -31,7 +31,7 @@
     <!-- field type definitions... note that the "name" attribute is
          just a label to be used by field definitions.  The "class"
          attribute and any other attributes determine the real type and
-         behavior of the fieldtype.
+         behavior of the fieldType.
       -->
 
     <!-- numeric field types that store and index the text
@@ -46,10 +46,10 @@
     <!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
          seconds part (.999) is optional.
       -->
-    <fieldtype name="date" class="solr.TrieDateField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+    <fieldType name="date" class="solr.TrieDateField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
-    <fieldtype name="boolean" class="solr.BoolField" />
-    <fieldtype name="string" class="solr.StrField" />
+    <fieldType name="boolean" class="solr.BoolField" />
+    <fieldType name="string" class="solr.StrField" />
 
     <fieldType name="uuid" class="solr.UUIDField" />
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml Sun Dec 14 22:50:20 2014
@@ -23,7 +23,7 @@
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="date" class="solr.TrieDateField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesMissing.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesMissing.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesMissing.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesMissing.xml Sun Dec 14 22:50:20 2014
@@ -24,8 +24,8 @@
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-    <fieldtype name="date" class="solr.TrieDateField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-    <fieldtype name="string" class="solr.StrField" />
+    <fieldType name="date" class="solr.TrieDateField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+    <fieldType name="string" class="solr.StrField" />
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesMulti.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesMulti.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesMulti.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-docValuesMulti.xml Sun Dec 14 22:50:20 2014
@@ -26,10 +26,10 @@
     <!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
          seconds part (.999) is optional.
       -->
-    <fieldtype name="date" class="solr.TrieDateField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+    <fieldType name="date" class="solr.TrieDateField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
-    <fieldtype name="boolean" class="solr.BoolField" />
-    <fieldtype name="string" class="solr.StrField" />
+    <fieldType name="boolean" class="solr.BoolField" />
+    <fieldType name="string" class="solr.StrField" />
 
     <fieldType name="uuid" class="solr.UUIDField" />
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-enums.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-enums.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-enums.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-enums.xml Sun Dec 14 22:50:20 2014
@@ -26,12 +26,12 @@
   <uniqueKey>id</uniqueKey>
 
   <types>
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.WhitespaceTokenizerFactory"/>
         <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
     <!-- note: you cannot change the order/existing values in enum without reindexing.
          but you can always add new values to the end. -->
     <fieldType name="severityType" class="solr.EnumField" enumsConfig="enumsConfig.xml" enumName="severity"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-field-sort-values.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-field-sort-values.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-field-sort-values.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-field-sort-values.xml Sun Dec 14 22:50:20 2014
@@ -20,12 +20,12 @@
   <types>
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
     <fieldType class="org.apache.solr.schema.WrappedIntField" name="wrapped_int"/>
   </types>
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml Sun Dec 14 22:50:20 2014
@@ -17,7 +17,7 @@
 
 <schema name="test" version="1.0">
   <types>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true" multiValued="false"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true" multiValued="false"/>
 
     <fieldType name="text" class="solr.TextField" multiValued="false">
       <analyzer>
@@ -221,8 +221,8 @@
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="double" class="solr.TrieDoubleField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
-    <fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
-    <fieldtype name="date" class="solr.TrieDateField" precisionStep="0"/>
+    <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
+    <fieldType name="date" class="solr.TrieDateField" precisionStep="0"/>
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-id-and-version-fields-only.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-id-and-version-fields-only.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-id-and-version-fields-only.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-id-and-version-fields-only.xml Sun Dec 14 22:50:20 2014
@@ -19,7 +19,7 @@
 <schema name="id-and-version-fields-only" version="1.5">
   <types>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" positionIncrementGap="0"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
   </types>
   <fields>
     <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="true"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-luceneMatchVersion.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-luceneMatchVersion.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-luceneMatchVersion.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-luceneMatchVersion.xml Sun Dec 14 22:50:20 2014
@@ -17,9 +17,9 @@
 -->
 <schema name="luceneMatchVersionTest" version="1.1">
  <types>
-  <fieldtype name="long" class="solr.TrieLongField"/>
-  <fieldtype name="string" class="solr.StrField"/>
-  <fieldtype name="text40" class="solr.TextField">
+  <fieldType name="long" class="solr.TrieLongField"/>
+  <fieldType name="string" class="solr.StrField"/>
+  <fieldType name="text40" class="solr.TextField">
     <analyzer>
       <tokenizer class="solr.StandardTokenizerFactory" luceneMatchVersion="LUCENE_40"/>
       <filter class="solr.StandardFilterFactory"/>
@@ -27,8 +27,8 @@
       <filter class="solr.StopFilterFactory" luceneMatchVersion="5.0"/>
       <filter class="solr.PorterStemFilterFactory"/>
     </analyzer>
-  </fieldtype>
-  <fieldtype name="textDefault" class="solr.TextField">
+  </fieldType>
+  <fieldType name="textDefault" class="solr.TextField">
     <analyzer>
       <tokenizer class="solr.StandardTokenizerFactory"/>
       <filter class="solr.StandardFilterFactory"/>
@@ -36,13 +36,13 @@
       <filter class="solr.StopFilterFactory"/>
       <filter class="solr.PorterStemFilterFactory"/>
     </analyzer>
-  </fieldtype>
-  <fieldtype name="textTurkishAnalyzer40" class="solr.TextField">
+  </fieldType>
+  <fieldType name="textTurkishAnalyzer40" class="solr.TextField">
     <analyzer class="org.apache.lucene.analysis.tr.TurkishAnalyzer" luceneMatchVersion="LUCENE_4_0"/>
-  </fieldtype>
-  <fieldtype name="textTurkishAnalyzerDefault" class="solr.TextField">
+  </fieldType>
+  <fieldType name="textTurkishAnalyzerDefault" class="solr.TextField">
     <analyzer class="org.apache.lucene.analysis.tr.TurkishAnalyzer"/>
-  </fieldtype>
+  </fieldType>
  </types>
  <fields>
    <field name="id" type="string" indexed="true" stored="true"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-not-required-unique-key.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-not-required-unique-key.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-not-required-unique-key.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-not-required-unique-key.xml Sun Dec 14 22:50:20 2014
@@ -24,13 +24,13 @@
 <schema name="test" version="1.0">
   <types>
     <fieldType name="string" class="solr.StrField"/>
-    <fieldtype name="int" class="solr.TrieIntField" precisionStep="0"/>
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="int" class="solr.TrieIntField" precisionStep="0"/>
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
         <filter class="solr.StandardFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
  </types>
 
 

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-numeric.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-numeric.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-numeric.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-numeric.xml Sun Dec 14 22:50:20 2014
@@ -27,7 +27,7 @@
 
 <schema name="test" version="1.0">
   <types>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
 
     <fieldType name="int"          class="solr.TrieIntField"    precisionStep="0" sortMissingLast="false" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="float"        class="solr.TrieFloatField"  precisionStep="0" sortMissingLast="false" omitNorms="true" positionIncrementGap="0"/>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-phrasesuggest.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-phrasesuggest.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-phrasesuggest.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-phrasesuggest.xml Sun Dec 14 22:50:20 2014
@@ -23,14 +23,14 @@
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
     <!-- basic text field -->
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.StandardTokenizerFactory"/>
         <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
     
-    <fieldtype name="phrase_suggest" class="solr.TextField">
+    <fieldType name="phrase_suggest" class="solr.TextField">
 	  <analyzer>
 	    <tokenizer class="solr.KeywordTokenizerFactory"/>
 	    <filter class="solr.PatternReplaceFilterFactory"
@@ -39,15 +39,15 @@
 	    <filter class="solr.LowerCaseFilterFactory"/>
 	    <filter class="solr.TrimFilterFactory"/>
 	  </analyzer>
-	</fieldtype>
+	</fieldType>
 	
-	<fieldtype name="ja_suggest" class="solr.TextField">
+	<fieldType name="ja_suggest" class="solr.TextField">
 	  <analyzer>
 	    <tokenizer class="solr.JapaneseTokenizerFactory" mode="normal"/>
 	    <filter class="solr.CJKWidthFilterFactory"/>
 	    <filter class="solr.JapaneseReadingFormFilterFactory" useRomaji="true"/>
 	  </analyzer>
-	</fieldtype>
+	</fieldType>
   </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-postingshighlight.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-postingshighlight.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-postingshighlight.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-postingshighlight.xml Sun Dec 14 22:50:20 2014
@@ -23,20 +23,20 @@
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
     <!-- basic text field: no offsets! -->
-    <fieldtype name="text" class="solr.TextField">
+    <fieldType name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
     
     <!-- text field with offsets -->
-    <fieldtype name="text_offsets" class="solr.TextField" storeOffsetsWithPositions="true">
+    <fieldType name="text_offsets" class="solr.TextField" storeOffsetsWithPositions="true">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
         <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
-    </fieldtype>
+    </fieldType>
    </types>
 
   <fields>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-preanalyzed.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-preanalyzed.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-preanalyzed.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-preanalyzed.xml Sun Dec 14 22:50:20 2014
@@ -18,7 +18,7 @@
 <schema name="tiny" version="1.1">
 
   <types>
-    <fieldtype name="preanalyzed" class="solr.PreAnalyzedField" parserImpl="json"/>
+    <fieldType name="preanalyzed" class="solr.PreAnalyzedField" parserImpl="json"/>
     <fieldType name="string" class="solr.StrField"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
   </types>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-replication1.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-replication1.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-replication1.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-replication1.xml Sun Dec 14 22:50:20 2014
@@ -29,7 +29,7 @@
   <types>
 
     <fieldType name="integer" class="solr.TrieIntField" precisionStep="0"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
 
 
   </types>

Modified: lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-replication2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-replication2.xml?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-replication2.xml (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test-files/solr/collection1/conf/schema-replication2.xml Sun Dec 14 22:50:20 2014
@@ -29,7 +29,7 @@
   <types>
 
     <fieldType name="integer" class="solr.TrieIntField" precisionStep="0"/>
-    <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
 
   </types>