You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2017/03/14 06:40:40 UTC

[03/43] lucene-solr:feature/autoscaling: SOLR-9986: Implement DatePointField

SOLR-9986: Implement DatePointField


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

Branch: refs/heads/feature/autoscaling
Commit: 3131ec2d99401c1fd1fc33a00343a59a78ab6445
Parents: 6df17c8
Author: Cao Manh Dat <da...@apache.org>
Authored: Tue Mar 7 10:11:47 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Tue Mar 7 10:11:47 2017 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../handler/component/RangeFacetRequest.java    |   5 +-
 .../component/SortedDateStatsValues.java        |  89 +++
 .../handler/component/StatsValuesFactory.java   |   8 +-
 .../org/apache/solr/request/NumericFacets.java  |   5 +-
 .../org/apache/solr/response/DocsStreamer.java  |   2 +
 .../org/apache/solr/schema/DatePointField.java  | 219 ++++++++
 .../apache/solr/search/SolrIndexSearcher.java   |   2 +-
 .../conf/schema-docValuesFaceting.xml           |   5 +-
 .../solr/collection1/conf/schema-point.xml      |  11 +
 .../solr/collection1/conf/schema-sorts.xml      |  11 +-
 .../test-files/solr/collection1/conf/schema.xml |  11 +-
 .../solr/collection1/conf/schema11.xml          |   3 +-
 .../solr/collection1/conf/schema12.xml          |   9 +-
 .../solr/collection1/conf/schema_latest.xml     |  13 +-
 ...lrconfig-parsing-update-processor-chains.xml |   7 +
 .../conf/solrconfig-update-processor-chains.xml |   2 +
 .../handler/admin/LukeRequestHandlerTest.java   |   4 +-
 .../org/apache/solr/schema/DateFieldTest.java   |  10 +-
 .../org/apache/solr/schema/TestPointFields.java | 561 ++++++++++++++++++-
 .../apache/solr/search/TestSolrQueryParser.java |  12 +-
 .../update/processor/AtomicUpdatesTest.java     |  25 +-
 .../ParsingFieldUpdateProcessorsTest.java       |  11 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   2 +
 24 files changed, 966 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4cfcb72..db721da 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -154,6 +154,8 @@ New Features
 * SOLR-9999: Instrument DirectUpdateHandler2. This registers existing statistics under metrics API and adds
   more metrics to track the rates of update and delete commands. (ab)
 
+* SOLR-9986: Implement DatePointField (Cao Manh Dat, Tom�s Fern�ndez L�bbe)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java b/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java
index aa3e3cb..3ac7300 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java
@@ -31,6 +31,7 @@ import org.apache.solr.common.params.RequiredSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.DatePointField;
 import org.apache.solr.schema.DateRangeField;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
@@ -725,7 +726,9 @@ public class RangeFacetRequest extends FacetComponent.FacetBase {
                                        final Date now) {
       super(rangeFacetRequest);
       this.now = now;
-      if (!(field.getType() instanceof TrieDateField) && !(field.getType() instanceof DateRangeField)) {
+      if (!(field.getType() instanceof TrieDateField)
+          && !(field.getType() instanceof DateRangeField)
+          && !(field.getType() instanceof DatePointField)) {
         throw new IllegalArgumentException(TYPE_ERR_MSG);
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/handler/component/SortedDateStatsValues.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SortedDateStatsValues.java b/solr/core/src/java/org/apache/solr/handler/component/SortedDateStatsValues.java
new file mode 100644
index 0000000..0df45c7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/component/SortedDateStatsValues.java
@@ -0,0 +1,89 @@
+/*
+ * 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.handler.component;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.Map;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.util.NamedList;
+
+public class SortedDateStatsValues implements StatsValues {
+
+  private final DateStatsValues dsv;
+  private final String fieldName;
+  private SortedNumericDocValues sndv;
+
+
+  public SortedDateStatsValues(DateStatsValues dsv, StatsField field) {
+    this.dsv = dsv;
+    this.fieldName = field.getSchemaField().getName();
+  }
+
+  @Override
+  public void accumulate(NamedList stv) {
+    dsv.accumulate(stv);
+  }
+
+  @Override
+  public void accumulate(int docId) throws IOException {
+    if (!sndv.advanceExact(docId)) {
+      missing();
+    } else {
+      for (int i = 0 ; i < sndv.docValueCount(); i++) {
+        dsv.accumulate(new Date(sndv.nextValue()), 1);
+      }
+    }
+
+  }
+
+  @Override
+  public void accumulate(BytesRef value, int count) {
+    dsv.accumulate(value, count);
+  }
+
+  @Override
+  public void missing() {
+    dsv.missing();
+  }
+
+  @Override
+  public void addMissing(int count) {
+    dsv.addMissing(count);
+  }
+
+  @Override
+  public void addFacet(String facetName, Map<String,StatsValues> facetValues) {
+    dsv.addFacet(facetName, facetValues);
+  }
+
+  @Override
+  public NamedList<?> getStatsValues() {
+    return dsv.getStatsValues();
+  }
+
+  @Override
+  public void setNextReader(LeafReaderContext ctx) throws IOException {
+    sndv = DocValues.getSortedNumeric(ctx.reader(), fieldName);
+    assert sndv != null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
index 2a6e795..d39ada2 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
@@ -63,8 +63,12 @@ public class StatsValuesFactory {
     
     final FieldType fieldType = sf.getType(); // TODO: allow FieldType to provide impl.
     
-    if (TrieDateField.class.isInstance(fieldType)) {
-      return new DateStatsValues(statsField);
+    if (TrieDateField.class.isInstance(fieldType) || DatePointField.class.isInstance(fieldType)) {
+      DateStatsValues statsValues = new DateStatsValues(statsField);
+      if (sf.multiValued()) {
+        return new SortedDateStatsValues(statsValues, statsField);
+      }
+      return statsValues;
     } else if (TrieField.class.isInstance(fieldType) || PointField.class.isInstance(fieldType)) {
       
       NumericStatsValues statsValue = new NumericStatsValues(statsField);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/request/NumericFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/NumericFacets.java b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
index a72eeee..c3bcb9f 100644
--- a/solr/core/src/java/org/apache/solr/request/NumericFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
@@ -19,6 +19,7 @@ package org.apache.solr.request;
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Collections;
+import java.util.Date;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -51,6 +52,7 @@ import org.apache.solr.schema.TrieField;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.DateMathParser;
 
 /** Utility class to compute facets on numeric fields. */
 final class NumericFacets {
@@ -512,7 +514,8 @@ final class NumericFacets {
         return String.valueOf(NumericUtils.sortableIntToFloat((int)bits));
       case DOUBLE:
         return String.valueOf(NumericUtils.sortableLongToDouble(bits));
-        //TODO: DATE
+      case DATE:
+        return new Date(bits).toInstant().toString();
       default:
         throw new AssertionError("Unsupported NumberType: " + fieldType.getNumberType());
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/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 ef0b0c7..bdea9ec 100644
--- a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
+++ b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
@@ -31,6 +31,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.response.transform.DocTransformer;
 import org.apache.solr.schema.BinaryField;
 import org.apache.solr.schema.BoolField;
+import org.apache.solr.schema.DatePointField;
 import org.apache.solr.schema.DoublePointField;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.FloatPointField;
@@ -221,6 +222,7 @@ public class DocsStreamer implements Iterator<SolrDocument> {
     KNOWN_TYPES.add(LongPointField.class);
     KNOWN_TYPES.add(DoublePointField.class);
     KNOWN_TYPES.add(FloatPointField.class);
+    KNOWN_TYPES.add(DatePointField.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/3131ec2d/solr/core/src/java/org/apache/solr/schema/DatePointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DatePointField.java b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
new file mode 100644
index 0000000..18bf651
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
@@ -0,0 +1,219 @@
+/*
+ * 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 java.time.Instant;
+import java.util.Collection;
+import java.util.Date;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.legacy.LegacyNumericRangeQuery;
+import org.apache.lucene.legacy.LegacyNumericType;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+import org.apache.lucene.queries.function.valuesource.MultiValuedLongFieldSource;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortedNumericSelector;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.mutable.MutableValueDate;
+import org.apache.lucene.util.mutable.MutableValueLong;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.DateMathParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DatePointField extends PointField implements DateValueFieldType {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public DatePointField() {
+    type = NumberType.DATE;
+  }
+
+
+  @Override
+  public Object toNativeType(Object val) {
+    if (val instanceof String) {
+      return DateMathParser.parseMath(null, (String) val);
+    }
+    return super.toNativeType(val);
+  }
+
+  @Override
+  public Query getPointRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive, boolean maxInclusive) {
+    long actualMin, actualMax;
+    if (min == null) {
+      actualMin = Long.MIN_VALUE;
+    } else {
+      actualMin = DateMathParser.parseMath(null, min).getTime();
+      if (!minInclusive) {
+        actualMin++;
+      }
+    }
+    if (max == null) {
+      actualMax = Long.MAX_VALUE;
+    } else {
+      actualMax = DateMathParser.parseMath(null, max).getTime();
+      if (!maxInclusive) {
+        actualMax--;
+      }
+    }
+    return LongPoint.newRangeQuery(field.getName(), actualMin, actualMax);
+  }
+
+  @Override
+  public Object toObject(SchemaField sf, BytesRef term) {
+    return new Date(LongPoint.decodeDimension(term.bytes, term.offset));
+  }
+
+  @Override
+  public Object toObject(IndexableField f) {
+    final Number val = f.numericValue();
+    if (val != null) {
+      return new Date(val.longValue());
+    } else {
+      throw new AssertionError("Unexpected state. Field: '" + f + "'");
+    }
+  }
+
+  @Override
+  protected Query getExactQuery(SchemaField field, String externalVal) {
+    return LongPoint.newExactQuery(field.getName(), DateMathParser.parseMath(null, externalVal).getTime());
+  }
+
+  @Override
+  public Query getSetQuery(QParser parser, SchemaField field, Collection<String> externalVals) {
+    assert externalVals.size() > 0;
+    long[] values = new long[externalVals.size()];
+    int i = 0;
+    for (String val:externalVals) {
+      values[i] = DateMathParser.parseMath(null, val).getTime();
+      i++;
+    }
+    return LongPoint.newSetQuery(field.getName(), values);
+  }
+
+  @Override
+  protected String indexedToReadable(BytesRef indexedForm) {
+    return Instant.ofEpochMilli(LongPoint.decodeDimension(indexedForm.bytes, indexedForm.offset)).toString();
+  }
+
+  @Override
+  public void readableToIndexed(CharSequence val, BytesRefBuilder result) {
+    Date date = (Date) toNativeType(val.toString());
+    result.grow(Long.BYTES);
+    result.setLength(Long.BYTES);
+    LongPoint.encodeDimension(date.getTime(), result.bytes(), 0);
+  }
+
+  @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 ? Long.MIN_VALUE : Long.MAX_VALUE;
+    } else if (sortMissingFirst) {
+      missingValue = top ? Long.MAX_VALUE : Long.MIN_VALUE;
+    }
+    SortField sf = new SortField(field.getName(), SortField.Type.LONG, top);
+    sf.setMissingValue(missingValue);
+    return sf;
+  }
+
+  @Override
+  public UninvertingReader.Type getUninversionType(SchemaField sf) {
+    if (sf.multiValued()) {
+      return UninvertingReader.Type.SORTED_LONG;
+    } else {
+      return UninvertingReader.Type.LONG_POINT;
+    }
+  }
+
+  @Override
+  public ValueSource getValueSource(SchemaField field, QParser parser) {
+    field.checkFieldCacheSource();
+    return new DatePointFieldSource(field.getName());
+  }
+
+  @Override
+  protected ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField field) {
+    return new MultiValuedLongFieldSource(field.getName(), choice);
+  }
+
+  @Override
+  public LegacyNumericType getNumericType() {
+    return LegacyNumericType.LONG;
+  }
+
+  @Override
+  public IndexableField createField(SchemaField field, Object value) {
+    if (!isFieldUsed(field)) return null;
+
+    Date date = (value instanceof Date)
+        ? ((Date)value)
+        : DateMathParser.parseMath(null, value.toString());
+    return new LongPoint(field.getName(), date.getTime());
+  }
+
+  @Override
+  protected StoredField getStoredField(SchemaField sf, Object value) {
+    return new StoredField(sf.getName(), ((Date) this.toNativeType(value)).getTime());
+  }
+}
+
+class DatePointFieldSource extends LongFieldSource {
+
+  public DatePointFieldSource(String field) {
+    super(field);
+  }
+
+  @Override
+  public String description() {
+    return "date(" + field + ')';
+  }
+
+  @Override
+  protected MutableValueLong newMutableValueLong() {
+    return new MutableValueDate();
+  }
+
+  @Override
+  public Date longToObject(long val) {
+    return new Date(val);
+  }
+
+  @Override
+  public String longToString(long val) {
+    return longToObject(val).toInstant().toString();
+  }
+
+  @Override
+  public long externalToLong(String extVal) {
+    return DateMathParser.parseMath(null, extVal).getTime();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index c650845..521324a 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -923,7 +923,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
                     outValues.add(NumericUtils.sortableLongToDouble(number));
                     break;
                   case DATE:
-                    newVal = new Date(number);
+                    outValues.add(new Date(number));
                     break;
                   default:
                     throw new AssertionError("Unexpected PointType: " + type);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
index 597f2c3..0917ff5 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
@@ -27,6 +27,7 @@
   <fieldType name="plong" class="solr.LongPointField"/>
   <fieldType name="pdouble" class="solr.DoublePointField"/>
   <fieldType name="pfloat" class="solr.FloatPointField"/>
+  <fieldType name="pdate" class="solr.DatePointField"/>
 
   <field name="id" type="string" indexed="true" stored="true" docValues="false" multiValued="false" required="true"/>
   <field name="id_dv" type="string" indexed="false" stored="false" docValues="true" multiValued="false"
@@ -60,8 +61,8 @@
   <dynamicField name="*_ds_dv" type="double" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_ds_p" type="pdouble" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="false" docValues="false"/>
-  <dynamicField name="*_dt_dv" type="date" indexed="true" stored="false" docValues="true"/>
-  <dynamicField name="*_dts_dv" type="date" indexed="true" stored="false" docValues="true" multiValued="true"/>
+  <dynamicField name="*_dt_dv" type="${solr.tests.dateClass:pdate}" indexed="true" stored="false" docValues="true"/>
+  <dynamicField name="*_dts_dv" type="${solr.tests.dateClass:pdate}" indexed="true" stored="false" docValues="true" multiValued="true"/>
 
   <defaultSearchField>id</defaultSearchField>
   <uniqueKey>id</uniqueKey>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/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 3561013..ed169a1 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
@@ -24,6 +24,7 @@
     <fieldType name="plong" class="solr.LongPointField"/>
     <fieldType name="pdouble" class="solr.DoublePointField"/>
     <fieldType name="pfloat" class="solr.FloatPointField"/>
+    <fieldType name="pdate" class="solr.DatePointField"/>
     
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
@@ -73,21 +74,31 @@
    <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"/>
+
+   <dynamicField name="*_p_dt"  type="pdate"    indexed="true"  stored="true"/>
+   <dynamicField name="*_p_dt_dv"  type="pdate"    indexed="true"  stored="true" docValues="true"/>
+   <dynamicField name="*_p_dt_mv"  type="pdate"    indexed="true"  stored="true" multiValued="true"/>
+   <dynamicField name="*_p_dt_mv_dv"  type="pdate"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
+   <dynamicField name="*_p_dt_ni_dv"  type="pdate"    indexed="false"  stored="true" docValues="true"/>
+   <dynamicField name="*_p_dt_ni_mv_dv"  type="pdate"    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_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="*_p_dt_dv_ns"  type="pdate"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_i_ni_ns_dv" type="pint"    indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_l_ni_ns_dv" type="plong"   indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_d_ni_ns_dv" type="pdouble" indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_f_ni_ns_dv" type="pfloat"  indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
+   <dynamicField name="*_p_dt_ni_ns_dv" type="pdate"   indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
 
    <dynamicField name="*_p_i_dv_ns_mv"  type="pint"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
    <dynamicField name="*_p_d_dv_ns_mv"  type="pdouble"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
    <dynamicField name="*_p_l_dv_ns_mv"  type="plong"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
    <dynamicField name="*_p_f_dv_ns_mv"  type="pfloat"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
+   <dynamicField name="*_p_dt_dv_ns_mv"  type="pdate"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
 
 
  </fields>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml b/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
index f68841c..8497318 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
@@ -73,9 +73,9 @@ NOTE: Tests expect every field in this schema to be sortable.
   <field name="date" type="date"/>
   <field name="date_last" type="date_last"/>
   <field name="date_first" type="date_first"/>
-  <field name="date_dv" type="date_dv"/>
-  <field name="date_dv_last" type="date_dv_last"/>
-  <field name="date_dv_first" type="date_dv_first"/>
+  <field name="date_dv" type="${solr.tests.dateClass:pdate}_dv"/>
+  <field name="date_dv_last" type="${solr.tests.dateClass:pdate}_dv_last"/>
+  <field name="date_dv_first" type="${solr.tests.dateClass:pdate}_dv_first"/>
 
   <field name="uuid" type="uuid"/>
   <field name="uuid_last" type="uuid_last"/>
@@ -276,6 +276,11 @@ NOTE: Tests expect every field in this schema to be sortable.
              sortMissingLast="true"/>
   <fieldType name="date_dv_first" class="solr.TrieDateField" stored="true" indexed="false" docValues="true"
              sortMissingFirst="true"/>
+  <fieldType name="pdate_dv" class="solr.DatePointField" stored="true" indexed="false" docValues="true"/>
+  <fieldType name="pdate_dv_last" class="solr.DatePointField" stored="true" indexed="false" docValues="true"
+             sortMissingLast="true"/>
+  <fieldType name="pdate_dv_first" class="solr.DatePointField" stored="true" indexed="false" docValues="true"
+             sortMissingFirst="true"/>
 
   <fieldType name="uuid" class="solr.UUIDField" stored="true" indexed="true"/>
   <fieldType name="uuid_last" class="solr.UUIDField" stored="true" indexed="true" sortMissingLast="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/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 c53be9b..8c549a3 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema.xml
@@ -49,6 +49,7 @@
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
   <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
+  <fieldType name="pdate" class="solr.DatePointField" docValues="true"/>
 
   <!-- Field type demonstrating an Analyzer failure -->
   <fieldType name="failtype1" class="solr.TextField">
@@ -569,13 +570,13 @@
 
   <field name="textgap" type="textgap" indexed="true" stored="true"/>
 
-  <field name="timestamp" type="date" indexed="true" stored="true" docValues="true" default="NOW" multiValued="false"/>
+  <field name="timestamp" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" docValues="true" default="NOW" multiValued="false"/>
   <field name="multiDefault" type="string" indexed="true" stored="true" default="muLti-Default" multiValued="true"/>
   <field name="intDefault" type="${solr.tests.intClass:pint}" indexed="true" stored="true" default="42" multiValued="false"/>
   <field name="intDvoDefault" type="${solr.tests.intClass:pint}" indexed="false" stored="false" multiValued="false"
          useDocValuesAsStored="true" docValues="true" default="42" />
   <field name="intRemove" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true"/>
-  <field name="dateRemove" type="date" indexed="true" stored="true" multiValued="true"/>
+  <field name="dateRemove" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" multiValued="true"/>
   <field name="floatRemove" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="true"/>
 
   <field name="nopositionstext" type="nopositions" indexed="true" stored="true"/>
@@ -621,8 +622,8 @@
   <dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
   <dynamicField name="*_d1" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="false"/>
   <dynamicField name="*_d1_ndv" type="${solr.tests.doubleClass:pdouble}" indexed="true" docValues="false" stored="true" multiValued="false"/>
-  <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
-  <dynamicField name="*_dt1" type="date" indexed="true" stored="true" multiValued="false"/>
+  <dynamicField name="*_dt" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true"/>
+  <dynamicField name="*_dt1" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" multiValued="false"/>
 
 
   <!-- some trie-coded dynamic fields for faster range queries -->
@@ -683,7 +684,7 @@
   <dynamicField name="*_l_dv" type="${solr.tests.longClass:plong}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_f_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_d_dv" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" docValues="true"/>
-  <dynamicField name="*_dt_dv" type="date" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_dt_dv" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_f1_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true" multiValued="false"/>
 
   <!--  Non-stored, DocValues=true -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/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 24129ae..7591c96 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
@@ -83,6 +83,7 @@
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
   <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
+  <fieldType name="pdate" class="solr.DatePointField" 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
@@ -403,7 +404,7 @@ valued. -->
 
    <dynamicField name="*_t"  type="text"    indexed="true"  stored="true"/>
    <dynamicField name="*_b"  type="boolean" indexed="true"  stored="true"/>
-   <dynamicField name="*_dt" type="date"    indexed="true"  stored="true"/>
+   <dynamicField name="*_dt" type="${solr.tests.dateClass:pdate}"    indexed="true"  stored="true"/>
    <dynamicField name="*_ws" type="text_ws" indexed="true"  stored="true"/>
    
    <!-- for testing tfidf functions, see TestFunctionQuery.testTFIDFFunctions -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/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 2d0615c..8577440 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema12.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema12.xml
@@ -42,12 +42,13 @@
   <fieldType name="tfloat" class="solr.TrieFloatField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
   <fieldType name="tlong" class="solr.TrieLongField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
   <fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
-  
+
   <!-- Point Fields -->
   <fieldType name="pint" class="solr.IntPointField" 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"/>
+  <fieldType name="pdate" class="solr.DatePointField" docValues="true"/>
 
   <!-- Field type demonstrating an Analyzer failure -->
   <fieldType name="failtype1" class="solr.TextField">
@@ -361,7 +362,7 @@
     </analyzer>
   </fieldType>
 
-  <!-- a text field with the stop filter only on the query analyzer 
+  <!-- a text field with the stop filter only on the query analyzer
    -->
   <fieldType name="text_sw" class="solr.TextField" positionIncrementGap="100">
     <analyzer type="index">
@@ -568,8 +569,8 @@
   <dynamicField name="*_b" type="boolean" 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="*_dt" type="date" indexed="true" stored="true"/>
-  
+  <dynamicField name="*_dt" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true"/>
+
   <dynamicField name="*_pi" type="pint" indexed="true" stored="true" docValues="false" multiValued="false"/>
 
   <!-- some trie-coded dynamic fields for faster range queries -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/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 e4747d8..dfeac32 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
@@ -221,10 +221,10 @@
   <dynamicField name="*_ds" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="true"/>
   <dynamicField name="*_dd" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_dds" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" multiValued="true" docValues="true"/>
-  <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
-  <dynamicField name="*_dts" type="date" indexed="true" stored="true" multiValued="true"/>
-  <dynamicField name="*_dtd" type="date" indexed="true" stored="false" docValues="true"/>
-  <dynamicField name="*_dtds" type="date" indexed="true" stored="false" multiValued="true" docValues="true"/>
+  <dynamicField name="*_dt" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true"/>
+  <dynamicField name="*_dts" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" multiValued="true"/>
+  <dynamicField name="*_dtd" type="${solr.tests.dateClass:pdate}" indexed="true" stored="false" docValues="true"/>
+  <dynamicField name="*_dtds" type="${solr.tests.dateClass:pdate}" indexed="true" stored="false" multiValued="true" docValues="true"/>
 
   <!-- docvalues and stored (S suffix) -->
   <dynamicField name="*_idS" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true"/>
@@ -237,8 +237,8 @@
   <dynamicField name="*_fdsS" type="${solr.tests.floatClass:pfloat}" 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="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="true" docValues="true"/>
-  <dynamicField name="*_dtdS" type="date" indexed="true" stored="true" docValues="true"/>
-  <dynamicField name="*_dtdsS" type="date" indexed="true" stored="true" multiValued="true" docValues="true"/>
+  <dynamicField name="*_dtdS" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_dtdsS" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" multiValued="true" docValues="true"/>
 
 
   <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
@@ -400,6 +400,7 @@
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
   <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
+  <fieldType name="pdate" class="solr.DatePointField" docValues="true"/>
   
 
   <!-- The format for this date field is of the form 1995-12-31T23:59:59Z, and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml
index 7078da7..f83df6c 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml
@@ -47,6 +47,13 @@
     </processor>
   </updateRequestProcessorChain>
 
+  <updateRequestProcessorChain name="parse-date-explicit-typeclass-point-selector-no-run-processor">
+    <processor class="solr.ParseDateFieldUpdateProcessorFactory">
+      <str name="typeClass">solr.DatePointField</str>
+      <str name="format">yyyy-MM-dd'T'HH:mm:ss.SSSZ</str>
+    </processor>
+  </updateRequestProcessorChain>
+
   <updateRequestProcessorChain name="parse-date-explicit-typeclass-selector-no-run-processor">
     <processor class="solr.ParseDateFieldUpdateProcessorFactory">
       <str name="typeClass">solr.TrieDateField</str>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
index a38bc04..426f3c0 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
@@ -94,6 +94,7 @@
       <!-- each set of exclusions is checked independently -->
       <lst name="exclude">
         <str name="typeClass">solr.TrieDateField</str>
+        <str name="typeClass">solr.DatePointField</str>
       </lst>
       <lst name="exclude">
         <str name="fieldRegex">.*HOSS.*</str>
@@ -144,6 +145,7 @@
   <updateRequestProcessorChain name="trim-classes">
     <processor class="solr.TrimFieldUpdateProcessorFactory">
       <str name="typeClass">solr.TrieDateField</str>
+      <str name="typeClass">solr.DatePointField</str>
       <str name="typeClass">solr.StrField</str>
     </processor>
   </updateRequestProcessorChain>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
index 92b4943..d253ef2 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
@@ -117,7 +117,7 @@ public class LukeRequestHandlerTest extends AbstractSolrTestCase {
     // only valid for fields that are indexed & stored
     for (String f : Arrays.asList("solr_t","solr_s","solr_ti",
         "solr_td","solr_dt","solr_b")) {
-
+      if (h.getCore().getLatestSchema().getField(f).getType().isPointField()) continue;
       final String xp = getFieldXPathPrefix(f);
       assertQ("Not as many index flags as expected ("+numFlags+") for " + f,
           req("qt","/admin/luke", "fl", f),
@@ -166,7 +166,7 @@ public class LukeRequestHandlerTest extends AbstractSolrTestCase {
       response = h.query(req);
       for (String f : Arrays.asList("solr_t", "solr_s", "solr_ti",
           "solr_td", "solr_dt", "solr_b")) {
-
+        if (h.getCore().getLatestSchema().getField(f).getType().isPointField()) continue;
         assertNull(TestHarness.validateXPath(response,
             getFieldXPathPrefix(f) + "[@name='index']"));
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java b/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
index ac451bf..d42d6dd 100644
--- a/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
@@ -29,7 +29,7 @@ import org.apache.solr.core.SolrResourceLoader;
 public class DateFieldTest extends SolrTestCaseJ4 {
   private final String testInstanceDir = TEST_HOME() + File.separator + "collection1";
   private final String testConfHome = testInstanceDir + File.separator + "conf"+ File.separator;
-  private TrieDateField f = null;
+  private FieldType f = null;
 
   @Override
   public void setUp()  throws Exception {
@@ -40,7 +40,7 @@ public class DateFieldTest extends SolrTestCaseJ4 {
     SolrConfig config = new SolrConfig
         (new SolrResourceLoader(Paths.get(testInstanceDir)), testConfHome + "solrconfig.xml", null);
     IndexSchema schema = IndexSchemaFactory.buildIndexSchema(testConfHome + "schema.xml", config);
-    f = new TrieDateField();
+    f = random().nextBoolean()? new TrieDateField() : new DatePointField();
     f.init(schema, Collections.<String,String>emptyMap());
   }
 
@@ -51,13 +51,13 @@ public class DateFieldTest extends SolrTestCaseJ4 {
     SchemaField sf = new SchemaField( "test", f, props, null );
     // String
     IndexableField out = f.createField(sf, "1995-12-31T23:59:59Z" );
-    assertEquals(820454399000L, f.toObject( out ).getTime() );
+    assertEquals(820454399000L, ((Date) f.toObject( out )).getTime() );
     // Date obj
     out = f.createField(sf, new Date(820454399000L) );
-    assertEquals(820454399000L, f.toObject( out ).getTime() );
+    assertEquals(820454399000L, ((Date) f.toObject( out )).getTime() );
     // Date math
     out = f.createField(sf, "1995-12-31T23:59:59.99Z+5MINUTES");
-    assertEquals(820454699990L, f.toObject( out ).getTime() );
+    assertEquals(820454699990L, ((Date) f.toObject( out )).getTime() );
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
index b3d0b97..3c1f0b3 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
@@ -25,6 +25,7 @@ import org.apache.lucene.search.IndexOrDocValuesQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DateMathParser;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -587,6 +588,126 @@ public class TestPointFields extends SolrTestCaseJ4 {
     doTestSetQueries("number_p_l_mv", getRandomStringArrayWithLongs(10, false), true);
     doTestSetQueries("number_p_l_ni_dv", getRandomStringArrayWithLongs(10, false), false);
   }
+
+  // Date
+
+  @Test
+  public void testDatePointFieldExactQuery() throws Exception {
+    doTestDatePointFieldExactQuery("number_p_dt", "1995-12-31T23:59:59Z");
+    doTestDatePointFieldExactQuery("number_p_dt_mv", "2015-12-31T23:59:59Z-1DAY");
+    doTestDatePointFieldExactQuery("number_p_dt_ni_dv", "2000-12-31T23:59:59Z+3DAYS");
+    doTestDatePointFieldExactQuery("number_p_dt_ni_ns_dv", "1995-12-31T23:59:59Z-1MONTH");
+    doTestDatePointFieldExactQuery("number_p_dt_ni_mv_dv", "1995-12-31T23:59:59Z+2MONTHS");
+  }
+
+  @Test
+  public void testDatePointFieldReturn() throws Exception {
+    testPointFieldReturn("number_p_dt", "date",
+        new String[]{"1995-12-31T23:59:59Z", "1994-02-28T23:59:59Z",
+            "2015-12-31T23:59:59Z", "2000-10-31T23:59:59Z", "1999-12-31T12:59:59Z"});
+    clearIndex();
+    assertU(commit());
+    testPointFieldReturn("number_p_dt_dv_ns", "date",
+        new String[]{"1995-12-31T23:59:59Z", "1994-02-28T23:59:59Z",
+            "2015-12-31T23:59:59Z", "2000-10-31T23:59:59Z", "1999-12-31T12:59:59Z"});
+  }
+
+  @Test
+  public void testDatePointFieldRangeQuery() throws Exception {
+    doTestDatePointFieldRangeQuery("number_p_dt");
+    doTestDatePointFieldRangeQuery("number_p_dt_ni_ns_dv");
+  }
+
+  @Test
+  public void testDatePointFieldSort() throws Exception {
+    doTestPointFieldSort("number_p_dt", "number_p_dt_dv", getSequentialStringArrayWithDates(10));
+  }
+
+  @Test
+  public void testDatePointFieldFacetField() throws Exception {
+    testPointFieldFacetField("number_p_dt", "number_p_dt_dv", getSequentialStringArrayWithDates(10));
+    clearIndex();
+    assertU(commit());
+    testPointFieldFacetField("number_p_dt", "number_p_dt_dv", getSequentialStringArrayWithDates(10));
+  }
+
+  @Test
+  public void testDatePointFieldRangeFacet() throws Exception {
+    doTestDatePointFieldRangeFacet("number_p_dt_dv", "number_p_dt");
+  }
+
+  @Test
+  public void testDatePointFunctionQuery() throws Exception {
+    doTestDatePointFunctionQuery("number_p_dt_dv", "number_p_dt", "date");
+  }
+
+  @Test
+  public void testDatePointStats() throws Exception {
+    testDatePointStats("number_p_dt", "number_p_dt_dv", getSequentialStringArrayWithDates(10));
+    testDatePointStats("number_p_dt_mv", "number_p_dt_mv_dv", getSequentialStringArrayWithDates(10));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedExactQuery() throws Exception {
+    testPointFieldMultiValuedExactQuery("number_p_dt_mv", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedExactQuery("number_p_dt_ni_mv_dv", getSequentialStringArrayWithDates(20));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedReturn() throws Exception {
+    testPointFieldMultiValuedReturn("number_p_dt_mv", "date", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedReturn("number_p_dt_ni_mv_dv", "date", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedReturn("number_p_dt_dv_ns_mv", "date", getSequentialStringArrayWithDates(20));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedRangeQuery() throws Exception {
+    testPointFieldMultiValuedRangeQuery("number_p_dt_mv", "date", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedRangeQuery("number_p_dt_ni_mv_dv", "date", getSequentialStringArrayWithDates(20));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedFacetField() throws Exception {
+    testPointFieldMultiValuedFacetField("number_p_dt_mv", "number_p_dt_mv_dv", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedFacetField("number_p_dt_mv", "number_p_dt_mv_dv", getRandomStringArrayWithDates(20, false));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedRangeFacet() throws Exception {
+    doTestDatePointFieldMultiValuedRangeFacet("number_p_dt_mv_dv", "number_p_dt_mv");
+  }
+
+  @Test
+  public void testDatePointMultiValuedFunctionQuery() throws Exception {
+    testPointMultiValuedFunctionQuery("number_p_dt_mv", "number_p_dt_mv_dv", "date", getSequentialStringArrayWithDates(20));
+  }
+
+  @Test
+  public void testDatePointFieldsAtomicUpdates() throws Exception {
+    if (!Boolean.getBoolean("enable.update.log")) {
+      return;
+    }
+    testDatePointFieldsAtomicUpdates("number_p_dt", "date");
+    testDatePointFieldsAtomicUpdates("number_p_dt_dv", "date");
+    testDatePointFieldsAtomicUpdates("number_p_dt_dv_ns", "date");
+  }
+
+  @Test
+  public void testMultiValuedDatePointFieldsAtomicUpdates() throws Exception {
+    if (!Boolean.getBoolean("enable.update.log")) {
+      return;
+    }
+    testMultiValuedDatePointFieldsAtomicUpdates("number_p_dt_mv", "date");
+    testMultiValuedDatePointFieldsAtomicUpdates("number_p_dt_ni_mv_dv", "date");
+    testMultiValuedDatePointFieldsAtomicUpdates("number_p_dt_dv_ns_mv", "date");
+  }
+
+  @Test
+  public void testDatePointSetQuery() throws Exception {
+    doTestSetQueries("number_p_dt", getRandomStringArrayWithDates(10, false), false);
+    doTestSetQueries("number_p_dt_mv", getRandomStringArrayWithDates(10, false), true);
+    doTestSetQueries("number_p_dt_ni_dv", getRandomStringArrayWithDates(10, false), false);
+  }
   
   @Test
   public void testIndexOrDocValuesQuery() throws Exception {
@@ -664,6 +785,15 @@ public class TestPointFields extends SolrTestCaseJ4 {
     }
     return arr;
   }
+
+  private String[] getSequentialStringArrayWithDates(int length) {
+    assert length < 60;
+    String[] arr = new String[length];
+    for (int i = 0; i < length; i++) {
+      arr[i] = String.format(Locale.ROOT, "1995-12-11T19:59:%02dZ", i);
+    }
+    return arr;
+  }
   
   private String[] getSequentialStringArrayWithDoubles(int length) {
     String[] arr = new String[length];
@@ -718,6 +848,27 @@ public class TestPointFields extends SolrTestCaseJ4 {
     }
     return stringArr;
   }
+
+  private String[] getRandomStringArrayWithDates(int length, boolean sorted) {
+    assert length < 60;
+    Set<Integer> set;
+    if (sorted) {
+      set = new TreeSet<>();
+    } else {
+      set = new HashSet<>();
+    }
+    while (set.size() < length) {
+      int number = random().nextInt(60);
+      set.add(number);
+    }
+    String[] stringArr = new String[length];
+    int i = 0;
+    for (int val:set) {
+      stringArr[i] = String.format(Locale.ROOT, "1995-12-11T19:59:%02dZ", val);
+      i++;
+    }
+    return stringArr;
+  }
   
   private void doTestIntPointFieldExactQuery(String field, boolean testLong) throws Exception {
     for (int i=0; i < 10; i++) {
@@ -1037,12 +1188,21 @@ public class TestPointFields extends SolrTestCaseJ4 {
     }
     assertU(commit());
     for (int i = 0; i < 20; i++) {
-      assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-")), 
-          "//*[@numFound='1']");
+      if (h.getCore().getLatestSchema().getField(fieldName).getType() instanceof DatePointField) {
+        assertQ(req("q", fieldName + ":\"" + numbers[i] + "\""),
+            "//*[@numFound='1']");
+      } else {
+        assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-")),
+            "//*[@numFound='1']");
+      }
     }
     
     for (int i = 0; i < 20; i++) {
-      assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-") + " OR " + fieldName + ":" + numbers[(i+1)%10].replace("-", "\\-")), "//*[@numFound='2']");
+      if (h.getCore().getLatestSchema().getField(fieldName).getType() instanceof DatePointField) {
+        assertQ(req("q", fieldName + ":\"" + numbers[i] + "\"" + " OR " + fieldName + ":\"" + numbers[(i+1)%10]+"\""), "//*[@numFound='2']");
+      } else {
+        assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-") + " OR " + fieldName + ":" + numbers[(i+1)%10].replace("-", "\\-")), "//*[@numFound='2']");
+      }
     }
   }
   
@@ -1089,10 +1249,10 @@ public class TestPointFields extends SolrTestCaseJ4 {
     assertTrue(h.getCore().getLatestSchema().getField(fieldName).multiValued());
     assertTrue(h.getCore().getLatestSchema().getField(fieldName).getType() instanceof PointField);
     for (int i=0; i < 10; i++) {
-      assertU(adoc("id", String.valueOf(i), fieldName, String.valueOf(i), fieldName, String.valueOf(i+10)));
+      assertU(adoc("id", String.valueOf(i), fieldName, numbers[i], fieldName, numbers[i+10]));
     }
     assertU(commit());
-    assertQ(req("q", fieldName + ":[0 TO 3]", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s]", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='4']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[10] + "']",
@@ -1103,36 +1263,36 @@ public class TestPointFields extends SolrTestCaseJ4 {
         "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[3] + "']",
         "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[13] + "']");
     
-    assertQ(req("q", fieldName + ":{0 TO 3]", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO %s]", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='3']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']",
         "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']",
         "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[3] + "']");
     
-    assertQ(req("q", fieldName + ":[0 TO 3}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s}", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='3']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']",
         "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']",
         "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']");
     
-    assertQ(req("q", fieldName + ":{0 TO 3}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO %s}", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='2']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']",
         "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']");
-    
-    assertQ(req("q", fieldName + ":{0 TO *}", "fl", "id, " + fieldName), 
+
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO *}", fieldName, numbers[0]), "fl", "id, " + fieldName),
         "//*[@numFound='10']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']");
     
-    assertQ(req("q", fieldName + ":{10 TO *}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO *}", fieldName, numbers[10]), "fl", "id, " + fieldName),
         "//*[@numFound='9']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']");
     
-    assertQ(req("q", fieldName + ":{* TO 3}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{* TO %s}", fieldName, numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='3']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']");
     
-    assertQ(req("q", fieldName + ":[* TO 3}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[* TO %s}", fieldName, numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='3']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']");
     
@@ -1141,14 +1301,14 @@ public class TestPointFields extends SolrTestCaseJ4 {
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']",
         "//result/doc[10]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[9] + "']");
     
-    assertQ(req("q", fieldName + ":[0 TO 1] OR " + fieldName + ":[8 TO 9]", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s] OR %s:[%s TO %s]", fieldName, numbers[0], numbers[1], fieldName, numbers[8], numbers[9]), "fl", "id, " + fieldName),
         "//*[@numFound='4']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']",
         "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']",
         "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[8] + "']",
         "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[9] + "']");
     
-    assertQ(req("q", fieldName + ":[0 TO 0] AND " + fieldName + ":[10 TO 10]", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s] OR %s:[%s TO %s]", fieldName, numbers[0], numbers[0], fieldName, numbers[10], numbers[10]), "fl", "id, " + fieldName),
         "//*[@numFound='1']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']");
   }
@@ -1238,12 +1398,22 @@ public class TestPointFields extends SolrTestCaseJ4 {
         larger = numbers[1];
       }
     } catch (NumberFormatException e) {
-      if (Double.valueOf(numbers[1]) < Double.valueOf(numbers[2])) {
-        smaller = numbers[1];
-        larger = numbers[2];
-      } else {
-        smaller = numbers[2];
-        larger = numbers[1];
+      try {
+        if (Double.valueOf(numbers[1]) < Double.valueOf(numbers[2])) {
+          smaller = numbers[1];
+          larger = numbers[2];
+        } else {
+          smaller = numbers[2];
+          larger = numbers[1];
+        }
+      } catch (NumberFormatException e2) {
+        if (DateMathParser.parseMath(null, numbers[1]).getTime() < DateMathParser.parseMath(null, numbers[2]).getTime()) {
+          smaller = numbers[1];
+          larger = numbers[2];
+        } else {
+          smaller = numbers[2];
+          larger = numbers[1];
+        }
       }
     }
     
@@ -1818,4 +1988,353 @@ public class TestPointFields extends SolrTestCaseJ4 {
         "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
         "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
   }
+
+
+  private void doTestDatePointFieldExactQuery(String field, String baseDate) throws Exception {
+    for (int i=0; i < 10; i++) {
+      assertU(adoc("id", String.valueOf(i), field, String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1)));
+    }
+    assertU(commit());
+    for (int i = 0; i < 10; i++) {
+      String date = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1);
+      assertQ(req("q", field + ":\""+date+"\"", "fl", "id, " + field),
+          "//*[@numFound='1']");
+    }
+
+    for (int i = 0; i < 10; i++) {
+      String date1 = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1);
+      String date2 = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, ((i+1)%10 + 1));
+      assertQ(req("q", field + ":\"" + date1 + "\""
+          + " OR " + field + ":\"" + date2 + "\""), "//*[@numFound='2']");
+    }
+
+    clearIndex();
+    assertU(commit());
+  }
+
+  private void doTestDatePointFieldRangeQuery(String fieldName) throws Exception {
+    String baseDate = "1995-12-31T10:59:59Z";
+    for (int i = 0; i < 10; i++) {
+      assertU(adoc("id", String.valueOf(i), fieldName, String.format(Locale.ROOT, "%s+%dHOURS", baseDate, i)));
+    }
+    assertU(commit());
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[%s+0HOURS TO %s+3HOURS]", baseDate, baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='4']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']",
+        "//result/doc[4]/date[@name='" + fieldName + "'][.='1995-12-31T13:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{%s+0HOURS TO %s+3HOURS]", baseDate, baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='3']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']",
+        "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T13:59:59Z']");
+
+    assertQ(req("q", fieldName + ":"+ String.format(Locale.ROOT, "[%s+0HOURS TO %s+3HOURS}",baseDate,baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='3']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']");
+
+    assertQ(req("q", fieldName + ":"+ String.format(Locale.ROOT, "{%s+0HOURS TO %s+3HOURS}",baseDate,baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='2']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{%s+0HOURS TO *}",baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='9']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{* TO %s+3HOURS}",baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='3']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[* TO %s+3HOURS}",baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='3']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']");
+
+    assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName),
+        "//*[@numFound='10']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']",
+        "//result/doc[10]/date[@name='" + fieldName + "'][.='1995-12-31T19:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[%s+0HOURS TO %s+1HOURS]",baseDate,baseDate)
+            +" OR " + fieldName + ":" + String.format(Locale.ROOT, "[%s+8HOURS TO %s+9HOURS]",baseDate,baseDate) , "fl", "id, " + fieldName),
+        "//*[@numFound='4']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T18:59:59Z']",
+        "//result/doc[4]/date[@name='" + fieldName + "'][.='1995-12-31T19:59:59Z']");
+
+    assertQ(req("q", fieldName + ":"+String.format(Locale.ROOT, "[%s+0HOURS TO %s+1HOURS]",baseDate,baseDate)
+            +" AND " + fieldName + ":"+String.format(Locale.ROOT, "[%s+1HOURS TO %s+2HOURS]",baseDate,baseDate) , "fl", "id, " + fieldName),
+        "//*[@numFound='1']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']");
+
+    assertQ(req("q", fieldName + ":"+String.format(Locale.ROOT, "[%s+0HOURS TO %s+1HOURS]",baseDate,baseDate)
+            +" AND NOT " + fieldName + ":"+String.format(Locale.ROOT, "[%s+1HOURS TO %s+2HOURS]",baseDate,baseDate) , "fl", "id, " + fieldName),
+        "//*[@numFound='1']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']");
+
+    clearIndex();
+    assertU(commit());
+  }
+
+  private void doTestDatePointFieldRangeFacet(String docValuesField, String nonDocValuesField) throws Exception {
+    String baseDate = "1995-01-10T10:59:59Z";
+    for (int i = 0; i < 10; i++) {
+      String date = String.format(Locale.ROOT, "%s+%dDAYS", baseDate, i);
+      assertU(adoc("id", String.valueOf(i), docValuesField, date, nonDocValuesField, date));
+    }
+    assertU(commit());
+    assertTrue(h.getCore().getLatestSchema().getField(docValuesField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof PointField);
+    assertQ(req("q", "*:*", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+10DAYS", "facet.range.gap", "+2DAYS"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-08T10:59:59Z'][.='0']");
+
+    assertQ(req("q", "*:*", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+10DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "dv"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-08T10:59:59Z'][.='0']");
+
+    assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
+    // Range Faceting with method = filter should work
+    assertQ(req("q", "*:*", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+10DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "filter"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-08T10:59:59Z'][.='0']");
+
+    // this should actually use filter method instead of dv
+    assertQ(req("q", "*:*", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+10DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "dv"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-08T10:59:59Z'][.='0']");
+  }
+
+  private void doTestDatePointFieldMultiValuedRangeFacet(String docValuesField, String nonDocValuesField) throws Exception {
+    String baseDate = "1995-01-10T10:59:59Z";
+    for (int i = 0; i < 10; i++) {
+      String date1 = String.format(Locale.ROOT, "%s+%dDAYS", baseDate, i);
+      String date2 = String.format(Locale.ROOT, "%s+%dDAYS", baseDate, i+10);
+      assertU(adoc("id", String.valueOf(i), docValuesField, date1, docValuesField, date2,
+          nonDocValuesField, date1, nonDocValuesField, date2));
+    }
+    assertU(commit());
+    assertTrue(h.getCore().getLatestSchema().getField(docValuesField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof PointField);
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+2DAYS"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-20T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-22T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-24T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-26T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-28T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1994-12-31T10:59:59Z'][.='0']");
+
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "dv"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-20T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-22T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-24T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-26T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-28T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1994-12-31T10:59:59Z'][.='0']");
+
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+100DAYS"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='10']");
+
+    assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
+    // Range Faceting with method = filter should work
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "filter"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-20T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-22T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-24T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-26T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-28T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1994-12-31T10:59:59Z'][.='0']");
+
+    // this should actually use filter method instead of dv
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "dv"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-20T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-22T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-24T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-26T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-28T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1994-12-31T10:59:59Z'][.='0']");
+  }
+
+  private void doTestDatePointFunctionQuery(String dvFieldName, String nonDvFieldName, String type) throws Exception {
+    String baseDate = "1995-01-10T10:59:10Z";
+    for (int i = 0; i < 10; i++) {
+      String date = String.format(Locale.ROOT, "%s+%dSECONDS", baseDate, i+1);
+      assertU(adoc("id", String.valueOf(i), dvFieldName, date, nonDvFieldName, date));
+    }
+    assertU(commit());
+    assertTrue(h.getCore().getLatestSchema().getField(dvFieldName).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(dvFieldName).getType() instanceof PointField);
+    assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "sort", "product(-1," + dvFieldName + ") asc"),
+        "//*[@numFound='10']",
+        "//result/doc[1]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:11Z']",
+        "//result/doc[2]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:12Z']",
+        "//result/doc[3]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:13Z']",
+        "//result/doc[10]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:20Z']");
+
+    assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", ms(" + dvFieldName + ","+baseDate+")"),
+        "//*[@numFound='10']",
+        "//result/doc[1]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='1000.0']",
+        "//result/doc[2]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='2000.0']",
+        "//result/doc[3]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='3000.0']",
+        "//result/doc[10]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='10000.0']");
+
+    assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", field(" + dvFieldName + ")"),
+        "//*[@numFound='10']",
+        "//result/doc[1]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:11Z']",
+        "//result/doc[2]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:12Z']",
+        "//result/doc[3]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:13Z']",
+        "//result/doc[10]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:20Z']");
+
+    assertFalse(h.getCore().getLatestSchema().getField(nonDvFieldName).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(nonDvFieldName).getType() instanceof PointField);
+
+    assertQEx("Expecting Exception",
+        "sort param could not be parsed as a query",
+        req("q", "*:*", "fl", "id, " + nonDvFieldName, "sort", "product(-1," + nonDvFieldName + ") asc"),
+        SolrException.ErrorCode.BAD_REQUEST);
+  }
+
+  private void testDatePointStats(String field, String dvField, String[] dates) {
+    for (int i = 0; i < dates.length; i++) {
+      assertU(adoc("id", String.valueOf(i), dvField, dates[i], field, dates[i]));
+    }
+    assertU(adoc("id", String.valueOf(dates.length)));
+    assertU(commit());
+    assertTrue(h.getCore().getLatestSchema().getField(dvField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(dvField).getType() instanceof PointField);
+    assertQ(req("q", "*:*", "fl", "id, " + dvField, "stats", "true", "stats.field", dvField),
+        "//*[@numFound='11']",
+        "//lst[@name='stats']/lst[@name='stats_fields']/lst[@name='" + dvField+ "']/date[@name='min'][.='" + dates[0] + "']",
+        "//lst[@name='stats']/lst[@name='stats_fields']/lst[@name='" + dvField+ "']/date[@name='max'][.='" + dates[dates.length-1] + "']",
+        "//lst[@name='stats']/lst[@name='stats_fields']/lst[@name='" + dvField+ "']/long[@name='count'][.='" + dates.length + "']",
+        "//lst[@name='stats']/lst[@name='stats_fields']/lst[@name='" + dvField+ "']/long[@name='missing'][.='1']");
+
+    assertFalse(h.getCore().getLatestSchema().getField(field).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(field).getType() instanceof PointField);
+    assertQEx("Expecting Exception",
+        "Can't calculate stats on a PointField without docValues",
+        req("q", "*:*", "fl", "id, " + field, "stats", "true", "stats.field", field),
+        SolrException.ErrorCode.BAD_REQUEST);
+  }
+
+  private void testDatePointFieldsAtomicUpdates(String field, String type) throws Exception {
+    String date = "1995-01-10T10:59:10Z";
+    assertU(adoc(sdoc("id", "1", field, date)));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/" + type + "[@name='" + field + "'][.='"+date+"']");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("set", date+"+2DAYS"))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/" + type + "[@name='" + field + "'][.='1995-01-12T10:59:10Z']");
+  }
+
+  private void testMultiValuedDatePointFieldsAtomicUpdates(String field, String type) throws Exception {
+    String date1 = "1995-01-10T10:59:10Z";
+    String date2 = "1995-01-11T10:59:10Z";
+    String date3 = "1995-01-12T10:59:10Z";
+    assertU(adoc(sdoc("id", "1", field, date1)));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date1+"']",
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=1");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("add", date2))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date1+"']",
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date2+"']",
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=2");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("remove", date1))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date2+"']",
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=1");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("set", ImmutableList.of(date1, date2, date3)))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date1+"']",
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date2+"']",
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date3+"']",
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=3");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("removeregex", ".*"))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=0");
+
+  }
+
+
 }