You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2016/10/17 05:44:11 UTC

[3/3] lucene-solr:jira/solr-8396: Added FloatPointField.

Added FloatPointField.

Most test pass, but "enum" facet doesn't work. Modified TestPointFields to isolate the failure, but still didn't investigate the cause.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/f4c68cdd
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/f4c68cdd
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/f4c68cdd

Branch: refs/heads/jira/solr-8396
Commit: f4c68cddb2005f6ccb100524dc715d0e25917547
Parents: 5fa22ce
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Sun Oct 16 22:41:48 2016 -0700
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Sun Oct 16 22:41:48 2016 -0700

----------------------------------------------------------------------
 .../org/apache/solr/response/DocsStreamer.java  |    4 +-
 .../org/apache/solr/schema/FloatPointField.java |  179 +++
 .../DocumentExpressionDictionaryFactory.java    |   12 +-
 .../solr/collection1/conf/schema-point.xml      |   26 +-
 .../test-files/solr/collection1/conf/schema.xml |    9 +-
 .../solr/collection1/conf/schema11.xml          |   12 +-
 .../solr/collection1/conf/schema12.xml          |    7 +-
 .../solr/collection1/conf/schema_latest.xml     |    7 +-
 .../test/org/apache/solr/TestPointFields.java   | 1452 ++++++++++--------
 .../handler/admin/LukeRequestHandlerTest.java   |    2 +
 .../java/org/apache/solr/SolrTestCaseJ4.java    |    6 +-
 11 files changed, 1026 insertions(+), 690 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f4c68cdd/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
index bee09fc..ef0b0c7 100644
--- a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
+++ b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
@@ -33,6 +33,7 @@ import org.apache.solr.schema.BinaryField;
 import org.apache.solr.schema.BoolField;
 import org.apache.solr.schema.DoublePointField;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.FloatPointField;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.IntPointField;
 import org.apache.solr.schema.LongPointField;
@@ -217,8 +218,9 @@ public class DocsStreamer implements Iterator<SolrDocument> {
     KNOWN_TYPES.add(TrieDateField.class);
     KNOWN_TYPES.add(BinaryField.class);
     KNOWN_TYPES.add(IntPointField.class);
-    KNOWN_TYPES.add(DoublePointField.class);
     KNOWN_TYPES.add(LongPointField.class);
+    KNOWN_TYPES.add(DoublePointField.class);
+    KNOWN_TYPES.add(FloatPointField.class);
     // We do not add UUIDField because UUID object is not a supported type in JavaBinCodec
     // and if we write UUIDField.toObject, we wouldn't know how to handle it in the client side
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f4c68cdd/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FloatPointField.java b/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
new file mode 100644
index 0000000..2098e37
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.schema;
+
+import java.lang.invoke.MethodHandles;
+
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.legacy.LegacyNumericType;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FloatPointField extends PointField implements FloatValueFieldType {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public FloatPointField() {
+    super(Float.BYTES);
+  }
+
+  {
+    type = PointTypes.FLOAT;
+  }
+
+  @Override
+  public Object toNativeType(Object val) {
+    if (val == null) return null;
+    if (val instanceof Number) return ((Number) val).floatValue();
+    if (val instanceof String) return Float.parseFloat((String) val);
+    return super.toNativeType(val);
+  }
+
+  public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive,
+      boolean maxInclusive) {
+    float actualMin, actualMax;
+    if (min == null) {
+      actualMin = Float.NEGATIVE_INFINITY;
+    } else {
+      actualMin = Float.parseFloat(min);
+      if (!minInclusive) {
+        actualMin = Math.nextUp(actualMin);
+      }
+    }
+    if (max == null) {
+      actualMax = Float.POSITIVE_INFINITY;
+    } else {
+      actualMax = Float.parseFloat(max);
+      if (!maxInclusive) {
+        actualMax = Math.nextDown(actualMax);
+      }
+    }
+    return FloatPoint.newRangeQuery(field.getName(), actualMin, actualMax);
+  }
+
+  @Override
+  public Object toObject(SchemaField sf, BytesRef term) {
+    return FloatPoint.decodeDimension(term.bytes, term.offset);
+  }
+  
+  @Override
+  public Object toObject(IndexableField f) {
+    final Number val = f.numericValue();
+    if (val != null) {
+      if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.NUMERIC) {
+        return Float.intBitsToFloat(val.intValue());
+      } else {
+        return val;
+      }
+    } else {
+      throw new AssertionError("Unexpected state. Field: '" + f + "'");
+    }
+  }
+
+  @Override
+  protected Query getExactQuery(QParser parser, SchemaField field, String externalVal) {
+    return FloatPoint.newExactQuery(field.getName(), Float.parseFloat(externalVal));
+  }
+
+  @Override
+  protected String indexedToReadable(BytesRef indexedForm) {
+    return Float.toString(FloatPoint.decodeDimension(indexedForm.bytes, indexedForm.offset));
+  }
+
+  @Override
+  public void readableToIndexed(CharSequence val, BytesRefBuilder result) {
+    result.grow(Float.BYTES);
+    result.setLength(Float.BYTES);
+    FloatPoint.encodeDimension(Float.parseFloat(val.toString()), result.bytes(), 0);
+  }
+  
+  @Override
+  protected BytesRef storedToIndexedByteRef(IndexableField f) {
+    BytesRef bytes = new BytesRef(new byte[Float.BYTES], 0, Float.BYTES);
+    FloatPoint.encodeDimension(f.numericValue().floatValue(), bytes.bytes, 0);
+    return bytes;
+  }
+  
+
+  @Override
+  public SortField getSortField(SchemaField field, boolean top) {
+    field.checkSortability();
+
+    Object missingValue = null;
+    boolean sortMissingLast = field.sortMissingLast();
+    boolean sortMissingFirst = field.sortMissingFirst();
+
+    if (sortMissingLast) {
+      missingValue = top ? Float.NEGATIVE_INFINITY : Float.POSITIVE_INFINITY;
+    } else if (sortMissingFirst) {
+      missingValue = top ? Float.POSITIVE_INFINITY : Float.NEGATIVE_INFINITY;
+    }
+    SortField sf = new SortField(field.getName(), SortField.Type.FLOAT, top);
+    sf.setMissingValue(missingValue);
+    return sf;
+  }
+
+  @Override
+  public Type getUninversionType(SchemaField sf) {
+    if (sf.multiValued()) {
+      throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
+//      return Type.SORTED_FLOAT;
+    } else {
+      return Type.FLOAT_POINT;
+    }
+  }
+
+  @Override
+  public ValueSource getValueSource(SchemaField field, QParser qparser) {
+    field.checkFieldCacheSource();
+    return new FloatFieldSource(field.getName());
+  }
+
+  @Override
+  public LegacyNumericType getNumericType() {
+    // TODO: refactor this to not use LegacyNumericType
+    return LegacyNumericType.FLOAT;
+  }
+
+  @Override
+  public IndexableField createField(SchemaField field, Object value, float boost) {
+    if (!isFieldUsed(field)) return null;
+
+    if (boost != 1.0 && log.isTraceEnabled()) {
+      log.trace("Can't use document/field boost for PointField. Field: " + field.getName() + ", boost: " + boost);
+    }
+    float floatValue = (value instanceof Number) ? ((Number) value).floatValue() : Float.parseFloat(value.toString());
+    return new FloatPoint(field.getName(), floatValue);
+  }
+
+  @Override
+  protected StoredField getStoredField(SchemaField sf, Object value) {
+    return new StoredField(sf.getName(), (Float) this.toNativeType(value));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f4c68cdd/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java b/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
index 24f1553..77f9105 100644
--- a/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
+++ b/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
@@ -28,7 +28,11 @@ import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.search.suggest.DocumentValueSourceDictionary;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.schema.DoublePointField;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.FloatPointField;
+import org.apache.solr.schema.IntPointField;
+import org.apache.solr.schema.LongPointField;
 import org.apache.solr.schema.TrieDoubleField;
 import org.apache.solr.schema.TrieFloatField;
 import org.apache.solr.schema.TrieIntField;
@@ -111,13 +115,13 @@ public class DocumentExpressionDictionaryFactory extends DictionaryFactory {
     SortField.Type type = null;
     String fieldTypeName = core.getLatestSchema().getField(sortFieldName).getType().getTypeName();
     FieldType ft = core.getLatestSchema().getFieldTypes().get(fieldTypeName);
-    if (ft instanceof TrieFloatField) {
+    if (ft instanceof TrieFloatField || ft instanceof FloatPointField) {
       type = SortField.Type.FLOAT;
-    } else if (ft instanceof TrieIntField) {
+    } else if (ft instanceof TrieIntField || ft instanceof IntPointField) {
       type = SortField.Type.INT;
-    } else if (ft instanceof TrieLongField) {
+    } else if (ft instanceof TrieLongField || ft instanceof LongPointField) {
       type = SortField.Type.LONG;
-    } else if (ft instanceof TrieDoubleField) {
+    } else if (ft instanceof TrieDoubleField || ft instanceof DoublePointField) {
       type = SortField.Type.DOUBLE;
     }
     return type;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f4c68cdd/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
index 48c1059..08c0d96 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
@@ -57,8 +57,10 @@
     <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true" omitNorms="true"/>
 
     <fieldType name="pint" class="solr.IntPointField"/>
-    <fieldType name="pdouble" class="solr.DoublePointField"/>
     <fieldType name="plong" class="solr.LongPointField"/>
+    <fieldType name="pdouble" class="solr.DoublePointField"/>
+    <fieldType name="pfloat" class="solr.FloatPointField"/>
+    
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
@@ -101,6 +103,13 @@
    <dynamicField name="*_p_i_ni_dv"  type="pint"    indexed="false"  stored="true" docValues="true"/>
    <dynamicField name="*_p_i_ni_mv_dv"  type="pint"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
    
+   <dynamicField name="*_p_l"  type="plong"    indexed="true"  stored="true"/>
+   <dynamicField name="*_p_l_dv"  type="plong"    indexed="true"  stored="true" docValues="true"/>
+   <dynamicField name="*_p_l_mv"  type="plong"    indexed="true"  stored="true" multiValued="true"/>
+   <dynamicField name="*_p_l_mv_dv"  type="plong"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
+   <dynamicField name="*_p_l_ni_dv"  type="plong"    indexed="false"  stored="true" docValues="true"/>
+   <dynamicField name="*_p_l_ni_mv_dv"  type="plong"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
+   
    <dynamicField name="*_p_d"  type="pdouble"    indexed="true"  stored="true"/>
    <dynamicField name="*_p_d_dv"  type="pdouble"    indexed="true"  stored="true" docValues="true"/>
    <dynamicField name="*_p_d_mv"  type="pdouble"    indexed="true"  stored="true" multiValued="true"/>
@@ -108,17 +117,18 @@
    <dynamicField name="*_p_d_ni_dv"  type="pdouble"    indexed="false"  stored="true" docValues="true"/>
    <dynamicField name="*_p_d_ni_mv_dv"  type="pdouble"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
    
-   <dynamicField name="*_p_l"  type="plong"    indexed="true"  stored="true"/>
-   <dynamicField name="*_p_l_dv"  type="plong"    indexed="true"  stored="true" docValues="true"/>
-   <dynamicField name="*_p_l_mv"  type="plong"    indexed="true"  stored="true" multiValued="true"/>
-   <dynamicField name="*_p_l_mv_dv"  type="plong"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_p_l_ni_dv"  type="plong"    indexed="false"  stored="true" docValues="true"/>
-   <dynamicField name="*_p_l_ni_mv_dv"  type="plong"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
+   <dynamicField name="*_p_f"  type="pfloat"    indexed="true"  stored="true"/>
+   <dynamicField name="*_p_f_dv"  type="pfloat"    indexed="true"  stored="true" docValues="true"/>
+   <dynamicField name="*_p_f_mv"  type="pfloat"    indexed="true"  stored="true" multiValued="true"/>
+   <dynamicField name="*_p_f_mv_dv"  type="pfloat"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
+   <dynamicField name="*_p_f_ni_dv"  type="pfloat"    indexed="false"  stored="true" docValues="true"/>
+   <dynamicField name="*_p_f_ni_mv_dv"  type="pfloat"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
    
    <!-- return DV fields as  -->
    <dynamicField name="*_p_i_dv_ns"  type="pint"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
-   <dynamicField name="*_p_d_dv_ns"  type="pdouble"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_l_dv_ns"  type="plong"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
+   <dynamicField name="*_p_d_dv_ns"  type="pdouble"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
+   <dynamicField name="*_p_f_dv_ns"  type="pfloat"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    
    <!--  dynamicField name="*_l"  type="plong"   indexed="true"  stored="true"/>
    <dynamicField name="*_f"  type="pfloat"  indexed="true"  stored="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f4c68cdd/solr/core/src/test-files/solr/collection1/conf/schema.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema.xml b/solr/core/src/test-files/solr/collection1/conf/schema.xml
index 6a84eb9..be1b6f5 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema.xml
@@ -46,8 +46,9 @@
   
   <!-- Point Fields -->
   <fieldType name="pint" class="solr.IntPointField" docValues="true"/>
-  <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
+  <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
+  <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
 
   <!-- Field type demonstrating an Analyzer failure -->
   <fieldType name="failtype1" class="solr.TextField">
@@ -555,7 +556,7 @@
   <field name="dateRange" type="dateRange" multiValued="true"/>
 
   <field name="cat" type="string" indexed="true" stored="true" multiValued="true"/>
-  <field name="price" type="float" indexed="true" stored="true" multiValued="false"/>
+  <field name="price" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="false"/>
   <field name="inStock" type="boolean" indexed="true" stored="true"/>
 
   <field name="subword" type="subword" indexed="true" stored="true"/>
@@ -615,7 +616,7 @@
   <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
   <dynamicField name="*_b1" type="boolean" indexed="true" stored="true" multiValued="false"/>
   <dynamicField name="*_f" type="float" indexed="true" stored="true"/>
-  <dynamicField name="*_f1" type="float" indexed="true" stored="true" multiValued="false"/>
+  <dynamicField name="*_f1" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="false"/>
   <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
   <dynamicField name="*_d1" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="false"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
@@ -681,7 +682,7 @@
   <dynamicField name="*_f_dv" type="float" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_d_dv" type="double" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_dt_dv" type="date" indexed="true" stored="true" docValues="true"/>
-  <dynamicField name="*_f1_dv" type="float" indexed="true" stored="true" docValues="true" multiValued="false"/>
+  <dynamicField name="*_f1_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true" multiValued="false"/>
 
   <!--  Non-stored, DocValues=true -->
   <dynamicField name="*_i_dvo" multiValued="false" type="${solr.tests.intClass:pint}" docValues="true" indexed="true" stored="false"

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f4c68cdd/solr/core/src/test-files/solr/collection1/conf/schema11.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema11.xml b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
index 0d90ad6..53f18b8 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
@@ -80,9 +80,9 @@
   
   <!-- Point Fields -->
   <fieldType name="pint" class="solr.IntPointField" docValues="true"/>
-  <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
-
+  <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
+  <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
 
     <!-- The format for this date field is of the form 1995-12-31T23:59:59Z, and
          is a more restricted form of the canonical representation of dateTime
@@ -348,7 +348,7 @@ valued. -->
    <dynamicField name="*_i"    type="${solr.tests.intClass:pint}"    indexed="true"  stored="true"/>
    <dynamicField name="*_ii"   type="int"    indexed="true"  stored="true" multiValued="true"/>
    <dynamicField name="*_l"    type="${solr.tests.longClass:plong}"   indexed="true"  stored="true"/>
-   <dynamicField name="*_f"    type="float"  indexed="true"  stored="true"/>
+   <dynamicField name="*_f"    type="${solr.tests.floatClass:pfloat}"  indexed="true"  stored="true"/>
    <dynamicField name="*_d"    type="${solr.tests.doubleClass:pdouble}" indexed="true"  stored="true"/>
 
    <dynamicField name="*_ti"      type="tint"    indexed="true"  stored="true"/>
@@ -357,9 +357,9 @@ valued. -->
    <dynamicField name="*_tl"      type="tlong"   indexed="true"  stored="true"/>
    <dynamicField name="*_tl_dv"    type="${solr.tests.longClass:plong}"   indexed="true"  stored="true" docValues="true"/>
    <dynamicField name="*_tl_ni_dv"   type="${solr.tests.longClass:plong}"   indexed="false"  stored="true" docValues="true"/>
-   <dynamicField name="*_tf"      type="tfloat"  indexed="true"  stored="true"/>
-   <dynamicField name="*_tf_dv"    type="tfloat"  indexed="true"  stored="true" docValues="true"/>
-   <dynamicField name="*_tf_ni_dv"   type="tfloat"  indexed="false"  stored="true" docValues="true"/>
+   <dynamicField name="*_tf"      type="${solr.tests.floatClass:pfloat}"  indexed="true"  stored="true"/>
+   <dynamicField name="*_tf_dv"    type="${solr.tests.floatClass:pfloat}"  indexed="true"  stored="true" docValues="true"/>
+   <dynamicField name="*_tf_ni_dv"   type="${solr.tests.floatClass:pfloat}"  indexed="false"  stored="true" docValues="true"/>
    <dynamicField name="*_td"      type="tdouble" indexed="true"  stored="true"/>
    <dynamicField name="*_td_dv"    type="${solr.tests.doubleClass:pdouble}" indexed="true"  stored="true" docValues="true"/>
    <dynamicField name="*_td_ni_dv"   type="${solr.tests.doubleClass:pdouble}" indexed="false"  stored="true" docValues="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f4c68cdd/solr/core/src/test-files/solr/collection1/conf/schema12.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema12.xml b/solr/core/src/test-files/solr/collection1/conf/schema12.xml
index 95f6581..6ccc193 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema12.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema12.xml
@@ -45,8 +45,9 @@
   
   <!-- Point Fields -->
   <fieldType name="pint" class="solr.IntPointField" docValues="true"/>
-  <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
+  <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
+  <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
 
   <!-- Field type demonstrating an Analyzer failure -->
   <fieldType name="failtype1" class="solr.TextField">
@@ -437,7 +438,7 @@
   <field name="text" type="text" indexed="true" stored="false"/>
   <field name="subject" type="text" indexed="true" stored="true"/>
   <field name="title" type="nametext" indexed="true" stored="true"/>
-  <field name="weight" type="float" indexed="true" stored="true"/>
+  <field name="weight" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
   <field name="bday" type="date" indexed="true" stored="true"/>
 
   <field name="text_np" type="text_np" indexed="true" stored="false"/>
@@ -565,7 +566,7 @@
   <dynamicField name="*_t" type="text" indexed="true" stored="true"/>
   <dynamicField name="*_tt" type="text" indexed="true" stored="true"/>
   <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
-  <dynamicField name="*_f" type="float" indexed="true" stored="true"/>
+  <dynamicField name="*_f" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
   <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f4c68cdd/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
index b74c07f..c6491eb 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
@@ -215,7 +215,7 @@
   <dynamicField name="*_lds" type="long" indexed="true" stored="false" multiValued="true" docValues="true"/>
   <dynamicField name="*_f" type="float" indexed="true" stored="true"/>
   <dynamicField name="*_fs" type="float" indexed="true" stored="true" multiValued="true"/>
-  <dynamicField name="*_fd" type="float" indexed="true" stored="false" docValues="true"/>
+  <dynamicField name="*_fd" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_fds" type="float" indexed="true" stored="false" multiValued="true" docValues="true"/>
   <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
   <dynamicField name="*_ds" type="double" indexed="true" stored="true" multiValued="true"/>
@@ -233,7 +233,7 @@
   <dynamicField name="*_sdsS" type="string" indexed="true" stored="true" multiValued="true" docValues="true"/>
   <dynamicField name="*_ldS" type="${solr.tests.longClass:plong}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_ldsS" type="long" indexed="true" stored="true" multiValued="true" docValues="true"/>
-  <dynamicField name="*_fdS" type="float" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_fdS" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_fdsS" type="float" indexed="true" stored="true" multiValued="true" docValues="true"/>
   <dynamicField name="*_ddS" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_ddsS" type="double" indexed="true" stored="true" multiValued="true" docValues="true"/>
@@ -397,8 +397,9 @@
   
   <!-- Point Fields -->
   <fieldType name="pint" class="solr.IntPointField"/>
-  <fieldType name="pdouble" class="solr.DoublePointField"/>
   <fieldType name="plong" class="solr.LongPointField"/>
+  <fieldType name="pdouble" class="solr.DoublePointField"/>
+  <fieldType name="pfloat" class="solr.FloatPointField"/>
   
 
   <!-- The format for this date field is of the form 1995-12-31T23:59:59Z, and