You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/07/07 13:42:59 UTC

[18/32] lucene-solr:jira/solr-10996: SOLR-10123: Fix to better support numeric PointFields. Unit tests now use randomized numeric fields.

SOLR-10123: Fix to better support numeric PointFields. Unit tests now use randomized numeric fields.

Closes #215


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

Branch: refs/heads/jira/solr-10996
Commit: 88b7ed1d463f396932071102814363a3a300294f
Parents: b200ba5
Author: Houston Putman <hp...@bloomberg.net>
Authored: Thu Jun 29 15:52:08 2017 -0400
Committer: Dennis Gove <dp...@gmail.com>
Committed: Thu Jul 6 06:40:47 2017 -0400

----------------------------------------------------------------------
 .../solr/analytics/ExpressionFactory.java       |  10 +-
 .../function/field/DateMultiField.java          |  47 -------
 .../function/field/DateMultiTrieField.java      |  47 +++++++
 .../function/field/DoubleMultiField.java        |  85 ------------
 .../function/field/DoubleMultiTrieField.java    |  85 ++++++++++++
 .../function/field/FloatMultiField.java         |  91 -------------
 .../function/field/FloatMultiTrieField.java     |  91 +++++++++++++
 .../analytics/function/field/IntMultiField.java | 100 ---------------
 .../function/field/IntMultiTrieField.java       | 100 +++++++++++++++
 .../function/field/LongMultiField.java          |  89 -------------
 .../function/field/LongMultiTrieField.java      |  89 +++++++++++++
 .../analytics/util/FacetRangeGenerator.java     |  12 +-
 .../solr/collection1/conf/schema-analytics.xml  |  10 +-
 .../configsets/cloud-analytics/conf/schema.xml  |  17 +--
 .../analytics/facet/FieldFacetCloudTest.java    |  86 ++++++-------
 .../solr/analytics/facet/FieldFacetTest.java    | 128 +++++++++++--------
 16 files changed, 548 insertions(+), 539 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/ExpressionFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/ExpressionFactory.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/ExpressionFactory.java
index b59469f..1d7a793 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/ExpressionFactory.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/ExpressionFactory.java
@@ -449,7 +449,7 @@ public class ExpressionFactory {
       }
     } else if (fieldType instanceof TrieIntField) {
       if (multivalued) {
-        aField = new IntMultiField(fieldName);
+        aField = new IntMultiTrieField(fieldName);
       } else {
         aField = new IntField(fieldName);
       }
@@ -461,7 +461,7 @@ public class ExpressionFactory {
       }
     } else if (fieldType instanceof TrieLongField) {
       if (multivalued) {
-        aField = new LongMultiField(fieldName);
+        aField = new LongMultiTrieField(fieldName);
       } else {
         aField = new LongField(fieldName);
       }
@@ -473,7 +473,7 @@ public class ExpressionFactory {
       }
     } else if (fieldType instanceof TrieFloatField) {
       if (multivalued) {
-        aField = new FloatMultiField(fieldName);
+        aField = new FloatMultiTrieField(fieldName);
       } else {
         aField = new FloatField(fieldName);
       }
@@ -485,7 +485,7 @@ public class ExpressionFactory {
       }
     } else if (fieldType instanceof TrieDoubleField) {
       if (multivalued) {
-        aField = new DoubleMultiField(fieldName);
+        aField = new DoubleMultiTrieField(fieldName);
       } else {
         aField = new DoubleField(fieldName);
       }
@@ -497,7 +497,7 @@ public class ExpressionFactory {
       }
     } else if (fieldType instanceof TrieDateField) {
       if (multivalued) {
-        aField = new DateMultiField(fieldName);
+        aField = new DateMultiTrieField(fieldName);
       } else {
         aField = new DateField(fieldName);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DateMultiField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DateMultiField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DateMultiField.java
deleted file mode 100644
index 64ee489..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DateMultiField.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.analytics.function.field;
-
-import java.time.Instant;
-import java.util.Date;
-import java.util.function.Consumer;
-
-import org.apache.solr.analytics.value.DateValueStream.CastingDateValueStream;
-import org.apache.solr.schema.TrieDateField;
-
-/**
- * An analytics wrapper for a multi-valued {@link TrieDateField} with DocValues enabled.
- */
-public class DateMultiField extends LongMultiField implements CastingDateValueStream {
-
-  public DateMultiField(String fieldName) {
-    super(fieldName);
-  }
-  
-  @Override
-  public void streamDates(Consumer<Date> cons) {
-    streamLongs(value -> cons.accept(new Date(value)));
-  }
-  @Override
-  public void streamStrings(Consumer<String> cons) {
-    streamLongs(value -> cons.accept(Instant.ofEpochMilli(value).toString()));
-  }
-  @Override
-  public void streamObjects(Consumer<Object> cons) {
-    streamLongs(value -> cons.accept(new Date(value)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DateMultiTrieField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DateMultiTrieField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DateMultiTrieField.java
new file mode 100644
index 0000000..20c1a09
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DateMultiTrieField.java
@@ -0,0 +1,47 @@
+/*
+ * 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.analytics.function.field;
+
+import java.time.Instant;
+import java.util.Date;
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.value.DateValueStream.CastingDateValueStream;
+import org.apache.solr.schema.TrieDateField;
+
+/**
+ * An analytics wrapper for a multi-valued {@link TrieDateField} with DocValues enabled.
+ */
+public class DateMultiTrieField extends LongMultiTrieField implements CastingDateValueStream {
+
+  public DateMultiTrieField(String fieldName) {
+    super(fieldName);
+  }
+  
+  @Override
+  public void streamDates(Consumer<Date> cons) {
+    streamLongs(value -> cons.accept(new Date(value)));
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    streamLongs(value -> cons.accept(Instant.ofEpochMilli(value).toString()));
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    streamLongs(value -> cons.accept(new Date(value)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DoubleMultiField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DoubleMultiField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DoubleMultiField.java
deleted file mode 100644
index 3d58634..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DoubleMultiField.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.analytics.function.field;
-
-import java.io.IOException;
-import java.util.function.Consumer;
-import java.util.function.DoubleConsumer;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.util.NumericUtils;
-import org.apache.solr.analytics.value.DoubleValueStream.CastingDoubleValueStream;
-import org.apache.solr.legacy.LegacyNumericUtils;
-import org.apache.solr.schema.TrieDoubleField;
-
-/**
- * An analytics wrapper for a multi-valued {@link TrieDoubleField} with DocValues enabled.
- */
-public class DoubleMultiField extends AnalyticsField implements CastingDoubleValueStream {
-  private SortedSetDocValues docValues;
-  private int count;
-  private double[] values;
-
-  public DoubleMultiField(String fieldName) {
-    super(fieldName);
-    count = 0;
-    values = new double[initialArrayLength];
-  }
-  
-  @Override
-  public void doSetNextReader(LeafReaderContext context) throws IOException {
-    docValues = DocValues.getSortedSet(context.reader(), fieldName);
-  }
-  @Override
-  public void collect(int doc) throws IOException {
-    count = 0;
-    if (docValues.advanceExact(doc)) {
-      int term;
-      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-        if (count == values.length) {
-          resizeValues();
-        }
-        values[count++] = NumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(docValues.lookupOrd(term)));
-      }
-    }
-  }
-  
-  private void resizeValues() {
-    double[] newValues = new double[values.length*2];
-    for (int i = 0; i < count; ++i) {
-      newValues[i] = values[i];
-    }
-    values = newValues;
-  }
-  
-  @Override
-  public void streamDoubles(DoubleConsumer cons) {
-    for (int i = 0; i < count; ++i) {
-      cons.accept(values[i]);
-    }
-  }
-  @Override
-  public void streamStrings(Consumer<String> cons) {
-    streamDoubles(value -> cons.accept(Double.toString(value)));
-  }
-  @Override
-  public void streamObjects(Consumer<Object> cons) {
-    streamDoubles(value -> cons.accept(value));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DoubleMultiTrieField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DoubleMultiTrieField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DoubleMultiTrieField.java
new file mode 100644
index 0000000..a79d652
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/DoubleMultiTrieField.java
@@ -0,0 +1,85 @@
+/*
+ * 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.analytics.function.field;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.DoubleConsumer;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.solr.analytics.value.DoubleValueStream.CastingDoubleValueStream;
+import org.apache.solr.legacy.LegacyNumericUtils;
+import org.apache.solr.schema.TrieDoubleField;
+
+/**
+ * An analytics wrapper for a multi-valued {@link TrieDoubleField} with DocValues enabled.
+ */
+public class DoubleMultiTrieField extends AnalyticsField implements CastingDoubleValueStream {
+  private SortedSetDocValues docValues;
+  private int count;
+  private double[] values;
+
+  public DoubleMultiTrieField(String fieldName) {
+    super(fieldName);
+    count = 0;
+    values = new double[initialArrayLength];
+  }
+  
+  @Override
+  public void doSetNextReader(LeafReaderContext context) throws IOException {
+    docValues = DocValues.getSortedSet(context.reader(), fieldName);
+  }
+  @Override
+  public void collect(int doc) throws IOException {
+    count = 0;
+    if (docValues.advanceExact(doc)) {
+      int term;
+      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        if (count == values.length) {
+          resizeValues();
+        }
+        values[count++] = NumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(docValues.lookupOrd(term)));
+      }
+    }
+  }
+  
+  private void resizeValues() {
+    double[] newValues = new double[values.length*2];
+    for (int i = 0; i < count; ++i) {
+      newValues[i] = values[i];
+    }
+    values = newValues;
+  }
+  
+  @Override
+  public void streamDoubles(DoubleConsumer cons) {
+    for (int i = 0; i < count; ++i) {
+      cons.accept(values[i]);
+    }
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    streamDoubles(value -> cons.accept(Double.toString(value)));
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    streamDoubles(value -> cons.accept(value));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/FloatMultiField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/FloatMultiField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/FloatMultiField.java
deleted file mode 100644
index acfce18..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/FloatMultiField.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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.analytics.function.field;
-
-import java.io.IOException;
-import java.util.function.Consumer;
-import java.util.function.DoubleConsumer;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.util.NumericUtils;
-import org.apache.solr.analytics.util.function.FloatConsumer;
-import org.apache.solr.analytics.value.FloatValueStream.CastingFloatValueStream;
-import org.apache.solr.legacy.LegacyNumericUtils;
-import org.apache.solr.schema.TrieFloatField;
-
-/**
- * An analytics wrapper for a multi-valued {@link TrieFloatField} with DocValues enabled.
- */
-public class FloatMultiField extends AnalyticsField implements CastingFloatValueStream {
-  private SortedSetDocValues docValues;
-  private int count;
-  private float[] values;
-
-  public FloatMultiField(String fieldName) {
-    super(fieldName);
-    count = 0;
-    values = new float[initialArrayLength];
-  }
-  
-  @Override
-  public void doSetNextReader(LeafReaderContext context) throws IOException {
-    docValues = DocValues.getSortedSet(context.reader(), fieldName);
-  }
-  
-  @Override
-  public void collect(int doc) throws IOException {
-    count = 0;
-    if (docValues.advanceExact(doc)) {
-      int term;
-      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-        if (count == values.length) {
-          resizeValues();
-        }
-        values[count++] = NumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(docValues.lookupOrd(term)));
-      }
-    }
-  }
-  
-  private void resizeValues() {
-    float[] newValues = new float[values.length*2];
-    for (int i = 0; i < count; ++i) {
-      newValues[i] = values[i];
-    }
-    values = newValues;
-  }
-  
-  @Override
-  public void streamFloats(FloatConsumer cons) {
-    for (int i = 0; i < count; ++i) {
-      cons.accept(values[i]);
-    }
-  }
-  @Override
-  public void streamDoubles(DoubleConsumer cons) {
-    streamFloats(value -> cons.accept((double)value));
-  }
-  @Override
-  public void streamStrings(Consumer<String> cons) {
-    streamFloats(value -> cons.accept(Float.toString(value)));
-  }
-  @Override
-  public void streamObjects(Consumer<Object> cons) {
-    streamFloats(value -> cons.accept(value));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/FloatMultiTrieField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/FloatMultiTrieField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/FloatMultiTrieField.java
new file mode 100644
index 0000000..7f6896f
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/FloatMultiTrieField.java
@@ -0,0 +1,91 @@
+/*
+ * 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.analytics.function.field;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.DoubleConsumer;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.solr.analytics.util.function.FloatConsumer;
+import org.apache.solr.analytics.value.FloatValueStream.CastingFloatValueStream;
+import org.apache.solr.legacy.LegacyNumericUtils;
+import org.apache.solr.schema.TrieFloatField;
+
+/**
+ * An analytics wrapper for a multi-valued {@link TrieFloatField} with DocValues enabled.
+ */
+public class FloatMultiTrieField extends AnalyticsField implements CastingFloatValueStream {
+  private SortedSetDocValues docValues;
+  private int count;
+  private float[] values;
+
+  public FloatMultiTrieField(String fieldName) {
+    super(fieldName);
+    count = 0;
+    values = new float[initialArrayLength];
+  }
+  
+  @Override
+  public void doSetNextReader(LeafReaderContext context) throws IOException {
+    docValues = DocValues.getSortedSet(context.reader(), fieldName);
+  }
+  
+  @Override
+  public void collect(int doc) throws IOException {
+    count = 0;
+    if (docValues.advanceExact(doc)) {
+      int term;
+      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        if (count == values.length) {
+          resizeValues();
+        }
+        values[count++] = NumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(docValues.lookupOrd(term)));
+      }
+    }
+  }
+  
+  private void resizeValues() {
+    float[] newValues = new float[values.length*2];
+    for (int i = 0; i < count; ++i) {
+      newValues[i] = values[i];
+    }
+    values = newValues;
+  }
+  
+  @Override
+  public void streamFloats(FloatConsumer cons) {
+    for (int i = 0; i < count; ++i) {
+      cons.accept(values[i]);
+    }
+  }
+  @Override
+  public void streamDoubles(DoubleConsumer cons) {
+    streamFloats(value -> cons.accept((double)value));
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    streamFloats(value -> cons.accept(Float.toString(value)));
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    streamFloats(value -> cons.accept(value));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/IntMultiField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/IntMultiField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/IntMultiField.java
deleted file mode 100644
index 657b1f3..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/IntMultiField.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.analytics.function.field;
-
-import java.io.IOException;
-import java.util.function.Consumer;
-import java.util.function.DoubleConsumer;
-import java.util.function.IntConsumer;
-import java.util.function.LongConsumer;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.solr.analytics.util.function.FloatConsumer;
-import org.apache.solr.analytics.value.IntValueStream.CastingIntValueStream;
-import org.apache.solr.legacy.LegacyNumericUtils;
-import org.apache.solr.schema.TrieIntField;
-
-/**
- * An analytics wrapper for a multi-valued {@link TrieIntField} with DocValues enabled.
- */
-public class IntMultiField extends AnalyticsField implements CastingIntValueStream {
-  private SortedSetDocValues docValues;
-  private int count;
-  private int[] values;
-
-  public IntMultiField(String fieldName) {
-    super(fieldName);
-    count = 0;
-    values = new int[initialArrayLength];
-  }
-  
-  @Override
-  public void doSetNextReader(LeafReaderContext context) throws IOException {
-    docValues = DocValues.getSortedSet(context.reader(), fieldName);
-  }
-  
-  @Override
-  public void collect(int doc) throws IOException {
-    count = 0;
-    if (docValues.advanceExact(doc)) {
-      int term;
-      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-        if (count == values.length) {
-          resizeValues();
-        }
-        values[count++] = LegacyNumericUtils.prefixCodedToInt(docValues.lookupOrd(term));
-      }
-    }
-  }
-  
-  private void resizeValues() {
-    int[] newValues = new int[values.length*2];
-    for (int i = 0; i < count; ++i) {
-      newValues[i] = values[i];
-    }
-    values = newValues;
-  }
-  
-  @Override
-  public void streamInts(IntConsumer cons) {
-    for (int i = 0; i < count; ++i) {
-      cons.accept(values[i]);
-    }
-  }
-  @Override
-  public void streamLongs(LongConsumer cons) {
-    streamInts(value -> cons.accept((long)value));
-  }
-  @Override
-  public void streamFloats(FloatConsumer cons) {
-    streamInts(value -> cons.accept((float)value));
-  }
-  @Override
-  public void streamDoubles(DoubleConsumer cons) {
-    streamInts(value -> cons.accept((double)value));
-  }
-  @Override
-  public void streamStrings(Consumer<String> cons) {
-    streamInts(value -> cons.accept(Integer.toString(value)));
-  }
-  @Override
-  public void streamObjects(Consumer<Object> cons) {
-    streamInts(value -> cons.accept(value));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/IntMultiTrieField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/IntMultiTrieField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/IntMultiTrieField.java
new file mode 100644
index 0000000..da62b60
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/IntMultiTrieField.java
@@ -0,0 +1,100 @@
+/*
+ * 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.analytics.function.field;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.DoubleConsumer;
+import java.util.function.IntConsumer;
+import java.util.function.LongConsumer;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.solr.analytics.util.function.FloatConsumer;
+import org.apache.solr.analytics.value.IntValueStream.CastingIntValueStream;
+import org.apache.solr.legacy.LegacyNumericUtils;
+import org.apache.solr.schema.TrieIntField;
+
+/**
+ * An analytics wrapper for a multi-valued {@link TrieIntField} with DocValues enabled.
+ */
+public class IntMultiTrieField extends AnalyticsField implements CastingIntValueStream {
+  private SortedSetDocValues docValues;
+  private int count;
+  private int[] values;
+
+  public IntMultiTrieField(String fieldName) {
+    super(fieldName);
+    count = 0;
+    values = new int[initialArrayLength];
+  }
+  
+  @Override
+  public void doSetNextReader(LeafReaderContext context) throws IOException {
+    docValues = DocValues.getSortedSet(context.reader(), fieldName);
+  }
+  
+  @Override
+  public void collect(int doc) throws IOException {
+    count = 0;
+    if (docValues.advanceExact(doc)) {
+      int term;
+      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        if (count == values.length) {
+          resizeValues();
+        }
+        values[count++] = LegacyNumericUtils.prefixCodedToInt(docValues.lookupOrd(term));
+      }
+    }
+  }
+  
+  private void resizeValues() {
+    int[] newValues = new int[values.length*2];
+    for (int i = 0; i < count; ++i) {
+      newValues[i] = values[i];
+    }
+    values = newValues;
+  }
+  
+  @Override
+  public void streamInts(IntConsumer cons) {
+    for (int i = 0; i < count; ++i) {
+      cons.accept(values[i]);
+    }
+  }
+  @Override
+  public void streamLongs(LongConsumer cons) {
+    streamInts(value -> cons.accept((long)value));
+  }
+  @Override
+  public void streamFloats(FloatConsumer cons) {
+    streamInts(value -> cons.accept((float)value));
+  }
+  @Override
+  public void streamDoubles(DoubleConsumer cons) {
+    streamInts(value -> cons.accept((double)value));
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    streamInts(value -> cons.accept(Integer.toString(value)));
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    streamInts(value -> cons.accept(value));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/LongMultiField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/LongMultiField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/LongMultiField.java
deleted file mode 100644
index dc2a953..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/LongMultiField.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.analytics.function.field;
-
-import java.io.IOException;
-import java.util.function.Consumer;
-import java.util.function.DoubleConsumer;
-import java.util.function.LongConsumer;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.solr.analytics.value.LongValueStream.CastingLongValueStream;
-import org.apache.solr.legacy.LegacyNumericUtils;
-import org.apache.solr.schema.TrieLongField;
-
-/**
- * An analytics wrapper for a multi-valued {@link TrieLongField} with DocValues enabled.
- */
-public class LongMultiField extends AnalyticsField implements CastingLongValueStream {
-  private SortedSetDocValues docValues;
-  private int count;
-  private long[] values;
-
-  public LongMultiField(String fieldName) {
-    super(fieldName);
-    count = 0;
-    values = new long[initialArrayLength];
-  }
-  
-  @Override
-  public void doSetNextReader(LeafReaderContext context) throws IOException {
-    docValues = DocValues.getSortedSet(context.reader(), fieldName);
-  }
-  @Override
-  public void collect(int doc) throws IOException {
-    count = 0;
-    if (docValues.advanceExact(doc)) {
-      int term;
-      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-        if (count == values.length) {
-          resizeValues();
-        }
-        values[count++] = LegacyNumericUtils.prefixCodedToLong(docValues.lookupOrd(term));
-      }
-    }
-  }
-  
-  private void resizeValues() {
-    long[] newValues = new long[values.length*2];
-    for (int i = 0; i < count; ++i) {
-      newValues[i] = values[i];
-    }
-    values = newValues;
-  }
-  
-  @Override
-  public void streamLongs(LongConsumer cons) {
-    for (int i = 0; i < count; ++i) {
-      cons.accept(values[i]);
-    }
-  }
-  @Override
-  public void streamDoubles(DoubleConsumer cons) {
-    streamLongs(value -> cons.accept((double)value));
-  }
-  @Override
-  public void streamStrings(Consumer<String> cons) {
-    streamLongs(value -> cons.accept(Long.toString(value)));
-  }
-  @Override
-  public void streamObjects(Consumer<Object> cons) {
-    streamLongs(value -> cons.accept(value));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/LongMultiTrieField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/LongMultiTrieField.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/LongMultiTrieField.java
new file mode 100644
index 0000000..53b57ad
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/field/LongMultiTrieField.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.analytics.function.field;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.DoubleConsumer;
+import java.util.function.LongConsumer;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.solr.analytics.value.LongValueStream.CastingLongValueStream;
+import org.apache.solr.legacy.LegacyNumericUtils;
+import org.apache.solr.schema.TrieLongField;
+
+/**
+ * An analytics wrapper for a multi-valued {@link TrieLongField} with DocValues enabled.
+ */
+public class LongMultiTrieField extends AnalyticsField implements CastingLongValueStream {
+  private SortedSetDocValues docValues;
+  private int count;
+  private long[] values;
+
+  public LongMultiTrieField(String fieldName) {
+    super(fieldName);
+    count = 0;
+    values = new long[initialArrayLength];
+  }
+  
+  @Override
+  public void doSetNextReader(LeafReaderContext context) throws IOException {
+    docValues = DocValues.getSortedSet(context.reader(), fieldName);
+  }
+  @Override
+  public void collect(int doc) throws IOException {
+    count = 0;
+    if (docValues.advanceExact(doc)) {
+      int term;
+      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        if (count == values.length) {
+          resizeValues();
+        }
+        values[count++] = LegacyNumericUtils.prefixCodedToLong(docValues.lookupOrd(term));
+      }
+    }
+  }
+  
+  private void resizeValues() {
+    long[] newValues = new long[values.length*2];
+    for (int i = 0; i < count; ++i) {
+      newValues[i] = values[i];
+    }
+    values = newValues;
+  }
+  
+  @Override
+  public void streamLongs(LongConsumer cons) {
+    for (int i = 0; i < count; ++i) {
+      cons.accept(values[i]);
+    }
+  }
+  @Override
+  public void streamDoubles(DoubleConsumer cons) {
+    streamLongs(value -> cons.accept((double)value));
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    streamLongs(value -> cons.accept(Long.toString(value)));
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    streamLongs(value -> cons.accept(value));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/FacetRangeGenerator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/FacetRangeGenerator.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/FacetRangeGenerator.java
index 0576096..2aec283 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/FacetRangeGenerator.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/FacetRangeGenerator.java
@@ -27,9 +27,8 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.FacetParams.FacetRangeInclude;
 import org.apache.solr.common.params.FacetParams.FacetRangeOther;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.NumericFieldType;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.schema.TrieDateField;
-import org.apache.solr.schema.TrieField;
 import org.apache.solr.util.DateMathParser;
 
 
@@ -249,7 +248,7 @@ public abstract class FacetRangeGenerator<T extends Comparable<T>> {
     final SchemaField sf = rangeFacet.getField();
     final FieldType ft = sf.getType();
     final FacetRangeGenerator<?> calc;
-    if (ft instanceof TrieField) {
+    if (ft instanceof NumericFieldType) {
       switch (ft.getNumberType()) {
         case FLOAT:
           calc = new FloatFacetRangeGenerator(rangeFacet);
@@ -267,10 +266,10 @@ public abstract class FacetRangeGenerator<T extends Comparable<T>> {
           calc = new DateFacetRangeGenerator(rangeFacet, null);
           break;
         default:
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to range facet on tried field of unexpected type:" + sf.getName());
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to range facet on numeric field of unexpected type: " + sf.getName());
       }
     } else {
-      throw new SolrException (SolrException.ErrorCode.BAD_REQUEST, "Unable to range facet on field:" + sf);
+      throw new SolrException (SolrException.ErrorCode.BAD_REQUEST, "Unable to range facet on non-numeric field: " + sf);
     } 
     return calc;
   }
@@ -330,9 +329,6 @@ class DateFacetRangeGenerator extends FacetRangeGenerator<Date> {
   public DateFacetRangeGenerator(final RangeFacet rangeFacet, final Date now) { 
     super(rangeFacet); 
     this.now = now;
-    if (! (field.getType() instanceof TrieDateField) ) {
-      throw new IllegalArgumentException("SchemaField must use field type extending TrieDateField");
-    }
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/test-files/solr/collection1/conf/schema-analytics.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/test-files/solr/collection1/conf/schema-analytics.xml b/solr/contrib/analytics/src/test-files/solr/collection1/conf/schema-analytics.xml
index 82d408a..527e66a 100644
--- a/solr/contrib/analytics/src/test-files/solr/collection1/conf/schema-analytics.xml
+++ b/solr/contrib/analytics/src/test-files/solr/collection1/conf/schema-analytics.xml
@@ -39,14 +39,14 @@
        These are provided more for backward compatability, allowing one
        to create a schema that matches an existing lucene index.
   -->
-  <fieldType name="int" class="${solr.tests.IntegerFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-  <fieldType name="float" class="${solr.tests.FloatFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-  <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-  <fieldType name="double" class="${solr.tests.DoubleFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="int" class="${solr.tests.IntegerFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="float" class="${solr.tests.FloatFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="long" class="${solr.tests.LongFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="double" class="${solr.tests.DoubleFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
   <!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
        seconds part (.999) is optional.
     -->
-  <fieldType name="date" class="${solr.tests.DateFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="date" class="${solr.tests.DateFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
   <fieldType name="boolean" class="solr.BoolField"/>
   <fieldType name="string" class="solr.StrField"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/test-files/solr/configsets/cloud-analytics/conf/schema.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/test-files/solr/configsets/cloud-analytics/conf/schema.xml b/solr/contrib/analytics/src/test-files/solr/configsets/cloud-analytics/conf/schema.xml
index d446437..0fac17f 100644
--- a/solr/contrib/analytics/src/test-files/solr/configsets/cloud-analytics/conf/schema.xml
+++ b/solr/contrib/analytics/src/test-files/solr/configsets/cloud-analytics/conf/schema.xml
@@ -20,21 +20,16 @@
 
 <schema name="cloud-analytics" version="1.0">
 
-  <fieldType name="int" class="solr.TrieIntField" precisionStep="8" positionIncrementGap="0"/>
-  <fieldType name="float" class="solr.TrieFloatField" precisionStep="8" positionIncrementGap="0"/>
-  <fieldType name="long" class="solr.TrieLongField" precisionStep="8" positionIncrementGap="0"/>
-  <fieldType name="double" class="solr.TrieDoubleField" precisionStep="8" positionIncrementGap="0"/>
   <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
   <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
-
-  <!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
-       seconds part (.999) is optional.
-    -->
-  <fieldType name="date" class="solr.TrieDateField" sortMissingLast="true" precisionStep="6"/>
-
+  
+  <fieldType name="int" class="${solr.tests.IntegerFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0" />
+  <fieldType name="float" class="${solr.tests.FloatFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0" />
+  <fieldType name="long" class="${solr.tests.LongFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="double" class="${solr.tests.DoubleFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="date" class="${solr.tests.DateFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0" />
 
   <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="true"/>
-  <field name="signatureField" type="string" indexed="true" stored="false"/>
 
   <!-- for versioning -->
   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetCloudTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetCloudTest.java b/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetCloudTest.java
index b3341b1..d4ffb40 100644
--- a/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetCloudTest.java
+++ b/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetCloudTest.java
@@ -28,7 +28,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 
-public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
+public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest {
   public static final int INT = 71;
   public static final int LONG = 36;
   public static final int LONGM = 50;
@@ -132,6 +132,8 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
     multiStringTestMissing = new ArrayList<>();
     multiDateTestStart = new ArrayList<>();
     multiDateTestMissing = new ArrayList<>();
+    
+    boolean multiCanHaveDuplicates = Boolean.getBoolean(NUMERIC_POINTS_SYSPROP);
 
     UpdateRequest req = new UpdateRequest();
     for (int j = 0; j < NUM_LOOPS; ++j) {
@@ -148,8 +150,6 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
       List<String> fields = new ArrayList<>();
       fields.add("id"); fields.add("1000"+j);
       
-      if (dt != 0) {
-      }
       if( i != 0 ) {
         fields.add("int_id"); fields.add("" + i);
       }
@@ -180,16 +180,13 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
       if ( sm != 0 ) {
         fields.add("string_sdm"); fields.add("str" + sm);
       }
-      if ( dtm != 0 ) {
-        fields.add("date_dtdm"); fields.add((1800+dtm) + "-12-31T23:59:59Z");
-      }
       req.add(fields.toArray(new String[0]));
       
-      if( dt != 0 ){
+      if( dt != 0 ) {
         //Dates
-        if (j-DATE<0) {
+        if ( j-DATE < 0 ) {
           ArrayList<Integer> list1 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list1.add(i);
             intDateTestMissing.add(0l);
           } else {
@@ -197,7 +194,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           intDateTestStart.add(list1);
           ArrayList<Long> list2 = new ArrayList<>();
-          if( l != 0l ){
+          if( l != 0l ) {
             list2.add(l);
             longDateTestMissing.add(0l);
           } else {
@@ -205,7 +202,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           longDateTestStart.add(list2);
           ArrayList<Float> list3 = new ArrayList<>();
-          if ( f != 0.0f ){
+          if ( f != 0.0f ) {
             list3.add(f);
             floatDateTestMissing.add(0l);
           } else {
@@ -214,7 +211,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           floatDateTestStart.add(list3);
           ArrayList<Double> list4 = new ArrayList<>();
-          if( d != 0.0d ){
+          if( d != 0.0d ) {
             list4.add(d);
             doubleDateTestMissing.add(0l);
           } else {
@@ -222,7 +219,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           doubleDateTestStart.add(list4);
           ArrayList<Integer> list5 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list5.add(i);
             multiDateTestMissing.add(0l);
           } else {
@@ -239,24 +236,26 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
         }
       }
       
-      if (j-DATEM<0 && dtm!=dt && dtm!=0) {
-        ArrayList<Integer> list1 = new ArrayList<>();
-        if( i != 0 ){
-          list1.add(i);
-          multiDateTestMissing.add(0l);
-        } else {
-          multiDateTestMissing.add(1l);
+      if ( dtm != 0 ) {
+        if ( j-DATEM < 0 && dtm != dt ) {
+          ArrayList<Integer> list1 = new ArrayList<>();
+          if( i != 0 ) {
+            list1.add(i);
+            multiDateTestMissing.add(0l);
+          } else {
+            multiDateTestMissing.add(1l);
+          }
+          multiDateTestStart.add(list1);
+        } else if ( dtm != dt || multiCanHaveDuplicates ) {
+          if( i != 0 ) multiDateTestStart.get(dtm-1).add(i); else increment(multiDateTestMissing,dtm-1);
         }
-        multiDateTestStart.add(list1);
-      } else if (dtm!=dt && dtm!=0) {
-        if( i != 0 ) multiDateTestStart.get(dtm-1).add(i);
       }
       
       if( s != 0 ){
         //Strings
-        if (j-STRING<0) {
+        if ( j-STRING < 0 ) {
           ArrayList<Integer> list1 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list1.add(i);
             intStringTestMissing.add(0l);
           } else {
@@ -264,7 +263,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           intStringTestStart.add(list1);
           ArrayList<Long> list2 = new ArrayList<>();
-          if( l != 0l ){
+          if( l != 0l ) {
             list2.add(l);
             longStringTestMissing.add(0l);
           } else {
@@ -280,7 +279,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           floatStringTestStart.add(list3);
           ArrayList<Double> list4 = new ArrayList<>();
-          if( d != 0.0d ){
+          if( d != 0.0d ) {
             list4.add(d);
             doubleStringTestMissing.add(0l);
           } else {
@@ -288,7 +287,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           doubleStringTestStart.add(list4);
           ArrayList<Integer> list5 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list5.add(i);
             multiStringTestMissing.add(0l);
           } else {
@@ -306,7 +305,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
       
       //Strings
       if( sm != 0 ){
-        if (j-STRINGM<0&&sm!=s) {
+        if ( j-STRINGM < 0 && sm != s ) {
           ArrayList<Integer> list1 = new ArrayList<>();
           if( i != 0 ){
             list1.add(i);
@@ -315,14 +314,14 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
             multiStringTestMissing.add(1l);
           }
           multiStringTestStart.add(list1);
-        } else if (sm!=s) {
+        } else if ( sm != s ) {
           if( i != 0 ) multiStringTestStart.get(sm-1).add(i); else increment(multiStringTestMissing,sm-1);
         }
       }
       
       //Int
-      if( i != 0 ){
-        if (j-INT<0) {
+      if( i != 0 ) {
+        if ( j-INT < 0 ) {
           ArrayList<String> list1 = new ArrayList<>();
           if( dt != 0 ){
             list1.add((1800+dt) + "-12-31T23:59:59Z");
@@ -332,7 +331,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           dateIntTestStart.add(list1);
           ArrayList<String> list2 = new ArrayList<>();
-          if( s != 0 ){
+          if( s != 0 ) {
             list2.add("str"+s);
             stringIntTestMissing.add(0l);
           } else {
@@ -346,8 +345,8 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
       }
       
       //Long
-      if( l != 0 ){
-        if (j-LONG<0) {
+      if( l != 0 ) {
+        if ( j-LONG < 0 ) {
           ArrayList<String> list1 = new ArrayList<>();
           if( dt != 0 ){
             list1.add((1800+dt) + "-12-31T23:59:59Z");
@@ -357,7 +356,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           dateLongTestStart.add(list1);
           ArrayList<String> list2 = new ArrayList<>();
-          if( s != 0 ){
+          if( s != 0 ) {
             list2.add("str"+s);
             stringLongTestMissing.add(0l);
           } else {
@@ -365,7 +364,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
           }
           stringLongTestStart.add(list2);
           ArrayList<Integer> list3 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list3.add(i);
             multiLongTestMissing.add(0l);
           } else {
@@ -380,17 +379,17 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
       }
       
       //Long
-      if( lm != 0 ){
-        if (j-LONGM<0&&lm!=l) {
+      if( lm != 0 ) {
+        if ( j-LONGM < 0 && lm != l ) {
           ArrayList<Integer> list1 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list1.add(i);
             multiLongTestMissing.add(0l);
           } else {
             multiLongTestMissing.add(1l);
           }
           multiLongTestStart.add(list1);
-        } else if (lm!=l) {
+        } else if ( lm != l || multiCanHaveDuplicates ) {
           if( i != 0 ) multiLongTestStart.get((int)lm-1).add(i); else increment( multiLongTestMissing,(int)lm-1);
         }
       }
@@ -848,7 +847,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
     //Int Date
     Collection<Integer> intDate = getValueList(response, "maxn", "fieldFacets", "date_dtd", "int", false);
     ArrayList<Integer> intDateTest = (ArrayList<Integer>)calculateStat(intDateTestStart, "max");
-    //assertEquals(responseStr,intDate,intDateTest);
+    assertEquals(responseStr,intDate,intDateTest);
     
     //Int String
     Collection<Integer> intString = getValueList(response, "maxn", "fieldFacets", "string_sd", "int", false);
@@ -1061,7 +1060,6 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
     assertEquals(responseStr,stringLong,stringLongTest);
   }
 
-  @SuppressWarnings("unchecked")
   @Test
   public void missingTest() throws Exception { 
     String[] params = new String[] {
@@ -1165,7 +1163,6 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
   @Test
   public void missingFacetTest() throws Exception { 
     String[] params = new String[] {
-        "o.func.facet_show_missing(a)", "fillmissing(a,\"(MISSING)\")",
         "o.missingf.s.mean", "mean(int_id)",
         "o.missingf.ff", "date_dtd",
         "o.missingf.ff", "string_sd",
@@ -1206,6 +1203,7 @@ public class FieldFacetCloudTest extends AbstractAnalyticsFacetCloudTest{
     }
   }
 
+  @SuppressWarnings({"unchecked", "rawtypes"})
   public static void assertEquals(String mes, Object actual, Object expected) {
     Collections.sort((List<Comparable>) actual);
     Collections.sort((List<Comparable>)  expected);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88b7ed1d/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetTest.java b/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetTest.java
index 59f4d88..d30413e 100644
--- a/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetTest.java
+++ b/solr/contrib/analytics/src/test/org/apache/solr/analytics/facet/FieldFacetTest.java
@@ -141,6 +141,8 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
     multiStringTestMissing = new ArrayList<>();
     multiDateTestStart = new ArrayList<>();
     multiDateTestMissing = new ArrayList<>();
+    
+    boolean multiCanHaveDuplicates = Boolean.getBoolean(NUMERIC_POINTS_SYSPROP);
 
     for (int j = 0; j < NUM_LOOPS; ++j) {
       int i = j%INT;
@@ -152,32 +154,48 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
       int dtm = j%DATEM;
       int s = j%STRING;
       int sm = j%STRINGM;
-      if (dt==0 && dtm == 0) {
-        assertU(adoc(filter("id", "1000" + j, "int_id", "" + i, "long_ld", "" + l, "float_fd", "" + f, 
-          "double_dd", "" + d,  "date_dtd", (1800+dt) + "-12-31T23:59:59Z", "string_sd", "str" + s,
-          "long_ldm", "" + l, "long_ldm", ""+lm, "string_sdm", "str" + s, "string_sdm", "str"+sm)));
-      } else if (dt == 0) {
-        assertU(adoc(filter("id", "1000" + j, "int_id", "" + i, "long_ld", "" + l, "float_fd", "" + f, 
-            "double_dd", "" + d,  "date_dtd", (1800+dt) + "-12-31T23:59:59Z", "string_sd", "str" + s,
-            "long_ldm", "" + l, "long_ldm", ""+lm, "string_sdm", "str" + s, "string_sdm", "str"+sm,
-            "date_dtdm", (1800+dtm) + "-12-31T23:59:59Z")));
-      } else if (dtm == 0) {
-        assertU(adoc(filter("id", "1000" + j, "int_id", "" + i, "long_ld", "" + l, "float_fd", "" + f, 
-            "double_dd", "" + d,  "date_dtd", (1800+dt) + "-12-31T23:59:59Z", "string_sd", "str" + s,
-            "long_ldm", "" + l, "long_ldm", ""+lm, "string_sdm", "str" + s, "string_sdm", "str"+sm,
-            "date_dtdm", (1800+dt) + "-12-31T23:59:59Z")));
-      } else {
-        assertU(adoc(filter("id", "1000" + j, "int_id", "" + i, "long_ld", "" + l, "float_fd", "" + f, 
-            "double_dd", "" + d,  "date_dtd", (1800+dt) + "-12-31T23:59:59Z", "string_sd", "str" + s,
-            "long_ldm", "" + l, "long_ldm", ""+lm, "string_sdm", "str" + s, "string_sdm", "str"+sm,
-            "date_dtdm", (1800+dt) + "-12-31T23:59:59Z", "date_dtdm", (1800+dtm) + "-12-31T23:59:59Z")));
+      
+
+      List<String> fields = new ArrayList<>();
+      fields.add("id"); fields.add("1000"+j);
+      
+      if( i != 0 ) {
+        fields.add("int_id"); fields.add("" + i);
+      }
+      if( l != 0l ) {
+        fields.add("long_ld"); fields.add("" + l);
+        fields.add("long_ldm"); fields.add("" + l);
+      }
+      if( lm != 0l ) {
+        fields.add("long_ldm"); fields.add("" + lm);
+      }
+      if( f != 0.0f ) {
+        fields.add("float_fd"); fields.add("" + f);
       }
+      if( d != 0.0d ) {
+        fields.add("double_dd"); fields.add("" + d);
+      }
+      if( dt != 0 ) {
+        fields.add("date_dtd"); fields.add((1800+dt) + "-12-31T23:59:59Z");
+        fields.add("date_dtdm"); fields.add((1800+dt) + "-12-31T23:59:59Z");
+      }
+      if ( dtm != 0 ) {
+        fields.add("date_dtdm"); fields.add((1800+dtm) + "-12-31T23:59:59Z");
+      }
+      if ( s != 0 ) {
+        fields.add("string_sd"); fields.add("str" + s);
+        fields.add("string_sdm"); fields.add("str" + s);
+      }
+      if ( sm != 0 ) {
+        fields.add("string_sdm"); fields.add("str" + sm);
+      }
+      assertU(adoc(fields.toArray(new String[0])));
       
-      if( dt != 0 ){
+      if( dt != 0 ) {
         //Dates
-        if (j-DATE<0) {
+        if ( j-DATE < 0 ) {
           ArrayList<Integer> list1 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list1.add(i);
             intDateTestMissing.add(0l);
           } else {
@@ -185,7 +203,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           intDateTestStart.add(list1);
           ArrayList<Long> list2 = new ArrayList<>();
-          if( l != 0l ){
+          if( l != 0l ) {
             list2.add(l);
             longDateTestMissing.add(0l);
           } else {
@@ -193,7 +211,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           longDateTestStart.add(list2);
           ArrayList<Float> list3 = new ArrayList<>();
-          if ( f != 0.0f ){
+          if ( f != 0.0f ) {
             list3.add(f);
             floatDateTestMissing.add(0l);
           } else {
@@ -202,7 +220,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           floatDateTestStart.add(list3);
           ArrayList<Double> list4 = new ArrayList<>();
-          if( d != 0.0d ){
+          if( d != 0.0d ) {
             list4.add(d);
             doubleDateTestMissing.add(0l);
           } else {
@@ -210,7 +228,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           doubleDateTestStart.add(list4);
           ArrayList<Integer> list5 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list5.add(i);
             multiDateTestMissing.add(0l);
           } else {
@@ -227,24 +245,26 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
         }
       }
       
-      if (j-DATEM<0 && dtm!=dt && dtm!=0) {
-        ArrayList<Integer> list1 = new ArrayList<>();
-        if( i != 0 ){
-          list1.add(i);
-          multiDateTestMissing.add(0l);
-        } else {
-          multiDateTestMissing.add(1l);
+      if ( dtm != 0 ) {
+        if ( j-DATEM < 0 && dtm != dt ) {
+          ArrayList<Integer> list1 = new ArrayList<>();
+          if( i != 0 ) {
+            list1.add(i);
+            multiDateTestMissing.add(0l);
+          } else {
+            multiDateTestMissing.add(1l);
+          }
+          multiDateTestStart.add(list1);
+        } else if ( dtm != dt || multiCanHaveDuplicates ) {
+          if( i != 0 ) multiDateTestStart.get(dtm-1).add(i); else increment(multiDateTestMissing,dtm-1);
         }
-        multiDateTestStart.add(list1);
-      } else if (dtm!=dt && dtm!=0) {
-        if( i != 0 ) multiDateTestStart.get(dtm-1).add(i);
       }
       
       if( s != 0 ){
         //Strings
-        if (j-STRING<0) {
+        if ( j-STRING < 0 ) {
           ArrayList<Integer> list1 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list1.add(i);
             intStringTestMissing.add(0l);
           } else {
@@ -252,7 +272,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           intStringTestStart.add(list1);
           ArrayList<Long> list2 = new ArrayList<>();
-          if( l != 0l ){
+          if( l != 0l ) {
             list2.add(l);
             longStringTestMissing.add(0l);
           } else {
@@ -268,7 +288,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           floatStringTestStart.add(list3);
           ArrayList<Double> list4 = new ArrayList<>();
-          if( d != 0.0d ){
+          if( d != 0.0d ) {
             list4.add(d);
             doubleStringTestMissing.add(0l);
           } else {
@@ -276,7 +296,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           doubleStringTestStart.add(list4);
           ArrayList<Integer> list5 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list5.add(i);
             multiStringTestMissing.add(0l);
           } else {
@@ -294,7 +314,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
       
       //Strings
       if( sm != 0 ){
-        if (j-STRINGM<0&&sm!=s) {
+        if ( j-STRINGM < 0 && sm != s ) {
           ArrayList<Integer> list1 = new ArrayList<>();
           if( i != 0 ){
             list1.add(i);
@@ -303,14 +323,14 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
             multiStringTestMissing.add(1l);
           }
           multiStringTestStart.add(list1);
-        } else if (sm!=s) {
+        } else if ( sm != s ) {
           if( i != 0 ) multiStringTestStart.get(sm-1).add(i); else increment(multiStringTestMissing,sm-1);
         }
       }
       
       //Int
-      if( i != 0 ){
-        if (j-INT<0) {
+      if( i != 0 ) {
+        if ( j-INT < 0 ) {
           ArrayList<String> list1 = new ArrayList<>();
           if( dt != 0 ){
             list1.add((1800+dt) + "-12-31T23:59:59Z");
@@ -320,7 +340,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           dateIntTestStart.add(list1);
           ArrayList<String> list2 = new ArrayList<>();
-          if( s != 0 ){
+          if( s != 0 ) {
             list2.add("str"+s);
             stringIntTestMissing.add(0l);
           } else {
@@ -334,8 +354,8 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
       }
       
       //Long
-      if( l != 0 ){
-        if (j-LONG<0) {
+      if( l != 0 ) {
+        if ( j-LONG < 0 ) {
           ArrayList<String> list1 = new ArrayList<>();
           if( dt != 0 ){
             list1.add((1800+dt) + "-12-31T23:59:59Z");
@@ -345,7 +365,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           dateLongTestStart.add(list1);
           ArrayList<String> list2 = new ArrayList<>();
-          if( s != 0 ){
+          if( s != 0 ) {
             list2.add("str"+s);
             stringLongTestMissing.add(0l);
           } else {
@@ -353,7 +373,7 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
           }
           stringLongTestStart.add(list2);
           ArrayList<Integer> list3 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list3.add(i);
             multiLongTestMissing.add(0l);
           } else {
@@ -368,17 +388,17 @@ public class FieldFacetTest extends AbstractAnalyticsFacetTest{
       }
       
       //Long
-      if( lm != 0 ){
-        if (j-LONGM<0&&lm!=l) {
+      if( lm != 0 ) {
+        if ( j-LONGM < 0 && lm != l ) {
           ArrayList<Integer> list1 = new ArrayList<>();
-          if( i != 0 ){
+          if( i != 0 ) {
             list1.add(i);
             multiLongTestMissing.add(0l);
           } else {
             multiLongTestMissing.add(1l);
           }
           multiLongTestStart.add(list1);
-        } else if (lm!=l) {
+        } else if ( lm != l || multiCanHaveDuplicates ) {
           if( i != 0 ) multiLongTestStart.get((int)lm-1).add(i); else increment( multiLongTestMissing,(int)lm-1);
         }
       }