You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2018/05/09 19:43:36 UTC

lucene-solr:master: SOLR-12170: fix date format exceptions for terms facet on date field

Repository: lucene-solr
Updated Branches:
  refs/heads/master 144f00a1e -> 1067cabc1


SOLR-12170: fix date format exceptions for terms facet on date field


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

Branch: refs/heads/master
Commit: 1067cabc192020db01e5e0ebc039f712e2f1c8ff
Parents: 144f00a
Author: yonik <yo...@apache.org>
Authored: Wed May 9 15:42:58 2018 -0400
Committer: yonik <yo...@apache.org>
Committed: Wed May 9 15:42:58 2018 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../solr/search/facet/FacetFieldProcessor.java  |   3 +-
 .../facet/FacetFieldProcessorByArray.java       |   4 +-
 .../apache/solr/search/facet/FacetModule.java   |   4 +-
 .../solr/collection1/conf/schema_latest.xml     |   2 +-
 .../search/facet/TestJsonFacetRefinement.java   | 496 ++++++++++---------
 .../solr/search/facet/TestJsonFacets.java       |   5 +
 7 files changed, 276 insertions(+), 241 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1067cabc/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8941f56..474df84 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -229,6 +229,9 @@ Bug Fixes
   Now we sync with ZooKeeper and try one more time.  V1 partially did this but only for aliases; now it does both.
   (David Smiley)
 
+* SOLR-12170: JSON Facet API: Terms facet on a date field sometimes failed with an exception complaining
+  about "Invalid Date String". (yonik)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1067cabc/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
index 9b47d66..f872db3 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
@@ -18,7 +18,6 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
-import java.time.Instant;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
@@ -708,7 +707,7 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
     bucket.add("val", bucketVal);
 
     // fieldQuery currently relies on a string input of the value...
-    String bucketStr = bucketVal instanceof Date ? Instant.ofEpochMilli(((Date)bucketVal).getTime()).toString() : bucketVal.toString();
+    String bucketStr = bucketVal instanceof Date ? ((Date)bucketVal).toInstant().toString() : bucketVal.toString();
     Query domainQ = ft.getFieldQuery(null, sf, bucketStr);
 
     fillBucket(bucket, domainQ, null, skip, facetInfo);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1067cabc/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
index 228678b..43bafd4 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
@@ -18,6 +18,7 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
+import java.util.Date;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -115,7 +116,8 @@ abstract class FacetFieldProcessorByArray extends FacetFieldProcessor {
             throw new RuntimeException(e);
           }
         },
-        Object::toString); // getFieldQueryVal
+        obj -> obj instanceof Date ? ((Date)obj).toInstant().toString() : obj.toString()
+    );
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1067cabc/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
index 9d94281..0e4266e 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
@@ -17,7 +17,6 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
-import java.time.Instant;
 import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
@@ -245,7 +244,7 @@ public class FacetModule extends SearchComponent {
         public void handleUnknownClass(Object o) {
           // handle date formatting correctly
           if (o instanceof Date) {
-            String s = Instant.ofEpochMilli(((Date)o).getTime()).toString();
+            String s = ((Date)o).toInstant().toString();
             writeString(s);
             return;
           }
@@ -308,6 +307,7 @@ public class FacetModule extends SearchComponent {
         return;
       }
 
+      // System.err.println("MERGING FACET RESULT FROM SHARD = " + facet);
       facetState.mcontext.root = facet;
       facetState.mcontext.newShard(shardRsp.getShard());
       facetState.merger.merge(facet , facetState.mcontext);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1067cabc/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 934451f..70abb39 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
@@ -382,6 +382,7 @@
   <fieldType name="float" class="${solr.tests.FloatFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
   <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
   <fieldType name="double" class="${solr.tests.DoubleFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
+  <fieldType name="date" class="${solr.tests.DateFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
 
   <!--
    Numeric field types that index each value at various levels of precision
@@ -420,7 +421,6 @@
 
        Note: For faster range queries, consider the tdate type
     -->
-  <fieldType name="date" class="${solr.tests.DateFieldType}" precisionStep="0" positionIncrementGap="0"/>
 
   <!-- A Trie based date field for faster date range queries and date faceting. -->
   <fieldType name="tdate" class="${solr.tests.DateFieldType}" precisionStep="6" positionIncrementGap="0"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1067cabc/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
index c8bf5ce..1e62cb2 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
@@ -239,18 +239,20 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
 
   @Test
   public void testBasicRefinement() throws Exception {
-    ModifiableSolrParams p = params("cat_s", "cat_s", "cat_i", "cat_i", "xy_s", "xy_s", "num_d", "num_d", "qw_s", "qw_s", "er_s", "er_s");
+    ModifiableSolrParams p;
+    p = params("cat_s", "cat_s", "cat_i", "cat_i", "date","cat_dt", "xy_s", "xy_s", "num_d", "num_d", "qw_s", "qw_s", "er_s", "er_s");
     doBasicRefinement(p);
 
-    p.set("terms", "method:dvhash,");
+    // multi-valued (except num_d)
+    p = params("cat_s", "cat_ss", "cat_i", "cat_is", "date","cat_dts", "xy_s", "xy_ss", "num_d", "num_d", "qw_s", "qw_ss", "er_s", "er_ss");
     doBasicRefinement(p);
 
-    // multi-valued
-    p = params("cat_s", "cat_ss", "cat_i", "cat_is", "xy_s", "xy_ss", "num_d", "num_d", "qw_s", "qw_ss", "er_s", "er_ss");
+    // single valued docvalues
+    p = params("cat_s", "cat_sd", "cat_i", "cat_id", "date","cat_dtd", "xy_s", "xy_sd", "num_d", "num_dd", "qw_s", "qw_sd", "er_s", "er_sd");
     doBasicRefinement(p);
 
-    // single valued docvalues
-    p = params("cat_s", "cat_sd", "cat_i", "cat_id", "xy_s", "xy_sd", "num_d", "num_dd", "qw_s", "qw_sd", "er_s", "er_sd");
+    // multi valued docvalues (except num_d)
+    p = params("cat_s", "cat_sds", "cat_i", "cat_ids", "date","cat_dtds", "xy_s", "xy_sds", "num_d", "num_dd", "qw_s", "qw_sds", "er_s", "er_sds");
     doBasicRefinement(p);
   }
 
@@ -270,17 +272,18 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
     String qw_s = p.get("qw_s");
     String er_s = p.get("er_s");  // this field is designed to test numBuckets refinement... the first phase will only have a single bucket returned for the top count bucket of cat_s
     String num_d = p.get("num_d");
+    String date = p.get("date");
 
-    clients.get(0).add(sdoc("id", "01", "all_s", "all", cat_s, "A", cat_i, 1, xy_s, "X", num_d, -1, qw_s, "Q", er_s, "E")); // A wins count tie
-    clients.get(0).add(sdoc("id", "02", "all_s", "all", cat_s, "B", cat_i, 2, xy_s, "Y", num_d, 3));
+    clients.get(0).add(sdoc("id", "01", "all_s", "all", cat_s, "A", cat_i, 1, date, "2001-01-01T01:01:01Z", xy_s, "X", num_d, -1, qw_s, "Q", er_s, "E")); // A wins count tie
+    clients.get(0).add(sdoc("id", "02", "all_s", "all", cat_s, "B", cat_i, 2, date, "2002-02-02T02:02:02Z", xy_s, "Y", num_d, 3));
 
-    clients.get(1).add(sdoc("id", "11", "all_s", "all", cat_s, "B", cat_i, 2, xy_s, "X", num_d, -5, er_s, "E")); // B highest count
-    clients.get(1).add(sdoc("id", "12", "all_s", "all", cat_s, "B", cat_i, 2, xy_s, "Y", num_d, -11, qw_s, "W"));
-    clients.get(1).add(sdoc("id", "13", "all_s", "all", cat_s, "A", cat_i, 1, xy_s, "X", num_d, 7, er_s, "R"));       // "R" will only be picked up via refinement when parent facet is cat_s
+    clients.get(1).add(sdoc("id", "11", "all_s", "all", cat_s, "B", cat_i, 2, date, "2002-02-02T02:02:02Z", xy_s, "X", num_d, -5, er_s, "E")); // B highest count
+    clients.get(1).add(sdoc("id", "12", "all_s", "all", cat_s, "B", cat_i, 2, date, "2002-02-02T02:02:02Z", xy_s, "Y", num_d, -11, qw_s, "W"));
+    clients.get(1).add(sdoc("id", "13", "all_s", "all", cat_s, "A", cat_i, 1, date, "2001-01-01T01:01:01Z", xy_s, "X", num_d, 7, er_s, "R"));       // "R" will only be picked up via refinement when parent facet is cat_s
 
-    clients.get(2).add(sdoc("id", "21", "all_s", "all", cat_s, "A", cat_i, 1, xy_s, "X", num_d, 17, qw_s, "W", er_s, "E")); // A highest count
-    clients.get(2).add(sdoc("id", "22", "all_s", "all", cat_s, "A", cat_i, 1, xy_s, "Y", num_d, -19));
-    clients.get(2).add(sdoc("id", "23", "all_s", "all", cat_s, "B", cat_i, 2, xy_s, "X", num_d, 11));
+    clients.get(2).add(sdoc("id", "21", "all_s", "all", cat_s, "A", cat_i, 1, date, "2001-01-01T01:01:01Z", xy_s, "X", num_d, 17, qw_s, "W", er_s, "E")); // A highest count
+    clients.get(2).add(sdoc("id", "22", "all_s", "all", cat_s, "A", cat_i, 1, date, "2001-01-01T01:01:01Z", xy_s, "Y", num_d, -19));
+    clients.get(2).add(sdoc("id", "23", "all_s", "all", cat_s, "B", cat_i, 2, date, "2002-02-02T02:02:02Z", xy_s, "X", num_d, 11));
 
     client.commit();
 
@@ -298,229 +301,252 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
      , "facets=={foo:555}"
      );
      ****/
+    for (String method : new String[]{"","dvhash","stream","uif","enum","stream","smart"}) {
+      if (method.equals("")) {
+        p.remove("terms");
+      } else {
+        p.set("terms", "method:" + method+", ");
+      }
 
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:false}" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", cat0:{ buckets:[ {val:A,count:3} ] }" +  // w/o overrequest and refinement, count is lower than it should be (we don't see the A from the middle shard)
-            "}"
-    );
-
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", cat0:{ buckets:[ {val:A,count:4} ] }" +  // w/o overrequest, we need refining to get the correct count.
-            "}"
-    );
-
-    // same as above, but with an integer field instead of a string
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "cat0:{${terms} type:terms, field:${cat_i}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", cat0:{ buckets:[ {val:1,count:4} ] }" +  // w/o overrequest, we need refining to get the correct count.
-            "}"
-    );
-
-    // basic refining test through/under a query facet
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "q1 : { type:query, q:'*:*', facet:{" +
-            "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
-            "}}" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", q1:{ count:8, cat0:{ buckets:[ {val:A,count:4} ] }   }" +
-            "}"
-    );
-
-    // basic refining test through/under a range facet
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "r1 : { type:range, field:${num_d} start:-20, end:20, gap:40   , facet:{" +
-            "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
-            "}}" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", r1:{ buckets:[{val:-20.0,count:8,  cat0:{buckets:[{val:A,count:4}]}  }]   }" +
-            "}"
-    );
-
-    // test that basic stats work for refinement
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true, facet:{ stat1:'sum(${num_d})'}   }" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", cat0:{ buckets:[ {val:A,count:4, stat1:4.0} ] }" +
-            "}"
-    );
-
-    // test sorting buckets by a different stat
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            " cat0:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:false, facet:{ min1:'min(${num_d})'}   }" +
-            ",cat1:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true,  facet:{ min1:'min(${num_d})'}   }" +
-            ",qfacet:{type:query, q:'*:*', facet:{  cat2:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true,  facet:{ min1:'min(${num_d})'}   }  }}" +  // refinement needed through a query facet
-            ",allf:{${terms} type:terms, field:all_s,  facet:{  cat3:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true,  facet:{ min1:'min(${num_d})'}   }  }}" +  // refinement needed through field facet
-            ",sum1:'sum(${num_d})'" +  // make sure that root bucket stats aren't affected by refinement
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", cat0:{ buckets:[ {val:A,count:3, min1:-19.0} ] }" +  // B wins in shard2, so we're missing the "A" count for that shard w/o refinement.
-            ", cat1:{ buckets:[ {val:A,count:4, min1:-19.0} ] }" +  // with refinement, we get the right count
-            ", qfacet:{ count:8,  cat2:{ buckets:[ {val:A,count:4, min1:-19.0} ] }    }" +  // just like the previous response, just nested under a query facet
-            ", allf:{ buckets:[  {cat3:{ buckets:[ {val:A,count:4, min1:-19.0} ] }  ,count:8,val:all   }]  }" +  // just like the previous response, just nested under a field facet
-            ", sum1:2.0" +
-            "}"
-    );
-
-    // test partial buckets (field facet within field facet)
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "ab:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true,  facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true   }  }}" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", ab:{ buckets:[  {val:A, count:4, xy:{buckets:[ {val:X,count:3}]}  }]  }" +  // just like the previous response, just nested under a field facet
-            "}"
-    );
-
-    // test that sibling facets and stats are included for _p buckets, but skipped for _s buckets
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            " ab :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true,  facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})'  }}" +
-            ",ab2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})'  }}" + // top level refine=false shouldn't matter
-            ",allf :{${terms} type:terms, field:all_s, limit:1, overrequest:0, refine:true,  facet:{cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})'  }}" +
-            ",allf2:{${terms} type:terms, field:all_s, limit:1, overrequest:0, refine:false, facet:{cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})'  }}" + // top level refine=false shouldn't matter
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", ab:{ buckets:[  {val:A, count:4, xy:{buckets:[ {val:X,count:3}]}    ,qq:{count:4}, ww:4.0 }]  }" +  // make sure qq and ww are included for _p buckets
-            ", allf:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }]  }" +  // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets
-            ", ab2:{ buckets:[  {val:A, count:4, xy:{buckets:[ {val:X,count:3}]}    ,qq:{count:4}, ww:4.0 }]  }" +  // make sure qq and ww are included for _p buckets
-            ", allf2:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }]  }" +  // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets
-            "}"
-    );
-
-    // test refining under the special "missing" bucket of a field facet
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "f:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, missing:true, refine:true,  facet:{  cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true   }  }}" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", f:{ buckets:[], missing:{count:8, cat:{buckets:[{val:A,count:4}]}  }  }" +  // just like the previous response, just nested under a field facet
-            "}"
-    );
-
-    // test filling in "missing" bucket for partially refined facets
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            // test all values missing in sub-facet
-            " ab :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false,  facet:{  zz:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, refine:false, missing:true}  }}" +
-            ",ab2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true ,  facet:{  zz:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, refine:true , missing:true}  }}" +
-            // test some values missing in sub-facet (and test that this works with normal partial bucket refinement)
-            ", cd :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false,  facet:{  qw:{${terms} type:terms, field:${qw_s}, limit:1, overrequest:0, refine:false, missing:true,   facet:{qq:{query:'*:*'}}   }  }}" +
-            ", cd2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true ,  facet:{  qw:{${terms} type:terms, field:${qw_s}, limit:1, overrequest:0, refine:true , missing:true,   facet:{qq:{query:'*:*'}}   }  }}" +
-
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", ab:{ buckets:[  {val:A, count:3, zz:{buckets:[], missing:{count:3}}}]  }" +
-            ",ab2:{ buckets:[  {val:A, count:4, zz:{buckets:[], missing:{count:4}}}]  }" +
-            ", cd:{ buckets:[  {val:A, count:3,  qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:1,qq:{count:1}}}}]  }" +
-            ",cd2:{ buckets:[  {val:A, count:4,  qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:2,qq:{count:2}}}}]  }" +
-            "}"
-    );
-
-    // test filling in missing "allBuckets"
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "  cat :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, allBuckets:true, facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:false}  }  }" +
-            ", cat2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , allBuckets:true, facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:true }  }  }" +
-            ", cat3:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , allBuckets:true, facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:true , facet:{f:'sum(${num_d})'}   }  }  }" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", cat:{ allBuckets:{count:8}, buckets:[  {val:A, count:3, xy:{buckets:[{count:2, val:X}], allBuckets:{count:3}}}]  }" +
-            ",cat2:{ allBuckets:{count:8}, buckets:[  {val:A, count:4, xy:{buckets:[{count:3, val:X}], allBuckets:{count:4}}}]  }" +
-            ",cat3:{ allBuckets:{count:8}, buckets:[  {val:A, count:4, xy:{buckets:[{count:3, val:X, f:23.0}], allBuckets:{count:4, f:4.0}}}]  }" +
-            "}"
-    );
-
-    // test filling in missing numBuckets
-    client.testJQ(params(p, "q", "*:*",
-        "json.facet", "{" +
-            "  cat :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, numBuckets:true, facet:{  er:{${terms} type:terms, field:${er_s}, limit:1, overrequest:0, numBuckets:true, refine:false}  }  }" +
-            ", cat2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , numBuckets:true, facet:{  er:{${terms} type:terms, field:${er_s}, limit:1, overrequest:0, numBuckets:true, refine:true }  }  }" +
-            "}"
-        )
-        , "facets=={ count:8" +
-            ", cat:{ numBuckets:2, buckets:[  {val:A, count:3, er:{numBuckets:1,buckets:[{count:2, val:E}]  }}]  }" +  // the "R" bucket will not be seen w/o refinement
-            ",cat2:{ numBuckets:2, buckets:[  {val:A, count:4, er:{numBuckets:2,buckets:[{count:2, val:E}]  }}]  }" +
-            "}"
-    );
-
-    final String sort_limit_over = "sort:'count desc', limit:1, overrequest:0, ";
-    // simplistic join domain testing: no refinement == low count
-    client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
-        "json.facet", "{" +
-            "  cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:false," +
-            // self join on all_s ensures every doc on every shard included in facets
-            "        domain: { join: { from:all_s, to:all_s } } }" +
-            "}"
-        )
-        ,
-        "/response/numFound==3",
-        "facets=={ count:3, " +
-            // w/o overrequest and refinement, count for 'A' is lower than it should be
-            // (we don't see the A from the middle shard)
-            "          cat0:{ buckets:[ {val:A,count:3} ] } }");
-    // simplistic join domain testing: refinement == correct count
-    client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
-        "json.facet", "{" +
-            "  cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:true," +
-            // self join on all_s ensures every doc on every shard included in facets
-            "        domain: { join: { from:all_s, to:all_s } } }" +
-            "}"
-        )
-        ,
-        "/response/numFound==3",
-        "facets=={ count:3," +
-            // w/o overrequest, we need refining to get the correct count for 'A'.
-            "          cat0:{ buckets:[ {val:A,count:4} ] } }");
-
-    // contrived join domain + refinement (at second level) + testing
-    client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
-        "json.facet", "{" +
-            // top level facet has a single term
-            "  all:{${terms} type:terms, field:all_s, " + sort_limit_over + " refine:true, " +
-            "       facet:{  " +
-            // subfacet will facet on cat after joining on all (so all docs should be included in subfacet)
-            "         cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:true," +
-            "               domain: { join: { from:all_s, to:all_s } } } } }" +
-            "}"
-        )
-        ,
-        "/response/numFound==3",
-        "facets=={ count:3," +
-            // all 3 docs matching base query have same 'all' value in top facet
-            "          all:{ buckets:[ { val:all, count:3, " +
-            // sub facet has refinement, so count for 'A' should be correct
-            "                            cat0:{ buckets: [{val:A,count:4}] } } ] } }");
 
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:false}" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", cat0:{ buckets:[ {val:A,count:3} ] }" +  // w/o overrequest and refinement, count is lower than it should be (we don't see the A from the middle shard)
+              "}"
+      );
+
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", cat0:{ buckets:[ {val:A,count:4} ] }" +  // w/o overrequest, we need refining to get the correct count.
+              "}"
+      );
+
+      // same as above, but with an integer field instead of a string
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "cat0:{${terms} type:terms, field:${cat_i}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", cat0:{ buckets:[ {val:1,count:4} ] }" +  // w/o overrequest, we need refining to get the correct count.
+              "}"
+      );
+
+      // same as above, but with a date field
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "cat0:{${terms} type:terms, field:${date}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", cat0:{ buckets:[ {val:'2001-01-01T01:01:01Z',count:4} ] }" +  // w/o overrequest, we need refining to get the correct count.
+              "}"
+      );
+
+      // basic refining test through/under a query facet
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "q1 : { type:query, q:'*:*', facet:{" +
+              "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
+              "}}" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", q1:{ count:8, cat0:{ buckets:[ {val:A,count:4} ] }   }" +
+              "}"
+      );
+
+      // basic refining test through/under a range facet
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "r1 : { type:range, field:${num_d} start:-20, end:20, gap:40   , facet:{" +
+              "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" +
+              "}}" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", r1:{ buckets:[{val:-20.0,count:8,  cat0:{buckets:[{val:A,count:4}]}  }]   }" +
+              "}"
+      );
+
+      // test that basic stats work for refinement
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true, facet:{ stat1:'sum(${num_d})'}   }" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", cat0:{ buckets:[ {val:A,count:4, stat1:4.0} ] }" +
+              "}"
+      );
+
+      // test sorting buckets by a different stat
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              " cat0:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:false, facet:{ min1:'min(${num_d})'}   }" +
+              ",cat1:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true,  facet:{ min1:'min(${num_d})'}   }" +
+              ",qfacet:{type:query, q:'*:*', facet:{  cat2:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true,  facet:{ min1:'min(${num_d})'}   }  }}" +  // refinement needed through a query facet
+              ",allf:{${terms} type:terms, field:all_s,  facet:{  cat3:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true,  facet:{ min1:'min(${num_d})'}   }  }}" +  // refinement needed through field facet
+              ",sum1:'sum(${num_d})'" +  // make sure that root bucket stats aren't affected by refinement
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", cat0:{ buckets:[ {val:A,count:3, min1:-19.0} ] }" +  // B wins in shard2, so we're missing the "A" count for that shard w/o refinement.
+              ", cat1:{ buckets:[ {val:A,count:4, min1:-19.0} ] }" +  // with refinement, we get the right count
+              ", qfacet:{ count:8,  cat2:{ buckets:[ {val:A,count:4, min1:-19.0} ] }    }" +  // just like the previous response, just nested under a query facet
+              ", allf:{ buckets:[  {cat3:{ buckets:[ {val:A,count:4, min1:-19.0} ] }  ,count:8,val:all   }]  }" +  // just like the previous response, just nested under a field facet
+              ", sum1:2.0" +
+              "}"
+      );
+
+      // test partial buckets (field facet within field facet)
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              " ab:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true,  facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true   }  }}" +
+              ",cd:{${terms} type:terms, field:${cat_i}, limit:1, overrequest:0, refine:true,  facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true   }  }}" +
+              ",ef:{${terms} type:terms, field:${date},  limit:1, overrequest:0, refine:true,  facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true   }  }}" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", ab:{ buckets:[  {val:A, count:4, xy:{buckets:[ {val:X,count:3}]}  }]  }" +  // just like the previous response, just nested under a field facet
+              ", cd:{ buckets:[  {val:1, count:4, xy:{buckets:[ {val:X,count:3}]}  }]  }" +  // just like the previous response, just nested under a field facet (int type)
+              ", ef:{ buckets:[  {val:'2001-01-01T01:01:01Z', count:4, xy:{buckets:[ {val:X,count:3}]}  }]  }" +  // just like the previous response, just nested under a field facet (date type)
+              "}"
+      );
+
+      // test that sibling facets and stats are included for _p buckets, but skipped for _s buckets
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              " ab :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true,  facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})'  }}" +
+              ",ab2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})'  }}" + // top level refine=false shouldn't matter
+              ",allf :{${terms} type:terms, field:all_s, limit:1, overrequest:0, refine:true,  facet:{cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})'  }}" +
+              ",allf2:{${terms} type:terms, field:all_s, limit:1, overrequest:0, refine:false, facet:{cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})'  }}" + // top level refine=false shouldn't matter
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", ab:{ buckets:[  {val:A, count:4, xy:{buckets:[ {val:X,count:3}]}    ,qq:{count:4}, ww:4.0 }]  }" +  // make sure qq and ww are included for _p buckets
+              ", allf:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }]  }" +  // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets
+              ", ab2:{ buckets:[  {val:A, count:4, xy:{buckets:[ {val:X,count:3}]}    ,qq:{count:4}, ww:4.0 }]  }" +  // make sure qq and ww are included for _p buckets
+              ", allf2:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }]  }" +  // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets
+              "}"
+      );
+
+      // test refining under the special "missing" bucket of a field facet
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "f:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, missing:true, refine:true,  facet:{  cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true   }  }}" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", f:{ buckets:[], missing:{count:8, cat:{buckets:[{val:A,count:4}]}  }  }" +  // just like the previous response, just nested under a field facet
+              "}"
+      );
+
+      // test filling in "missing" bucket for partially refined facets
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              // test all values missing in sub-facet
+              " ab :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false,  facet:{  zz:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, refine:false, missing:true}  }}" +
+              ",ab2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true ,  facet:{  zz:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, refine:true , missing:true}  }}" +
+              // test some values missing in sub-facet (and test that this works with normal partial bucket refinement)
+              ", cd :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false,  facet:{  qw:{${terms} type:terms, field:${qw_s}, limit:1, overrequest:0, refine:false, missing:true,   facet:{qq:{query:'*:*'}}   }  }}" +
+              ", cd2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true ,  facet:{  qw:{${terms} type:terms, field:${qw_s}, limit:1, overrequest:0, refine:true , missing:true,   facet:{qq:{query:'*:*'}}   }  }}" +
+
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", ab:{ buckets:[  {val:A, count:3, zz:{buckets:[], missing:{count:3}}}]  }" +
+              ",ab2:{ buckets:[  {val:A, count:4, zz:{buckets:[], missing:{count:4}}}]  }" +
+              ", cd:{ buckets:[  {val:A, count:3,  qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:1,qq:{count:1}}}}]  }" +
+              ",cd2:{ buckets:[  {val:A, count:4,  qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:2,qq:{count:2}}}}]  }" +
+              "}"
+      );
+
+      // test filling in missing "allBuckets"
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "  cat :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, allBuckets:true, facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:false}  }  }" +
+              ", cat2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , allBuckets:true, facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:true }  }  }" +
+              ", cat3:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , allBuckets:true, facet:{  xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:true , facet:{f:'sum(${num_d})'}   }  }  }" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", cat:{ allBuckets:{count:8}, buckets:[  {val:A, count:3, xy:{buckets:[{count:2, val:X}], allBuckets:{count:3}}}]  }" +
+              ",cat2:{ allBuckets:{count:8}, buckets:[  {val:A, count:4, xy:{buckets:[{count:3, val:X}], allBuckets:{count:4}}}]  }" +
+              ",cat3:{ allBuckets:{count:8}, buckets:[  {val:A, count:4, xy:{buckets:[{count:3, val:X, f:23.0}], allBuckets:{count:4, f:4.0}}}]  }" +
+              "}"
+      );
+
+      // test filling in missing numBuckets
+      client.testJQ(params(p, "q", "*:*",
+          "json.facet", "{" +
+              "  cat :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, numBuckets:true, facet:{  er:{${terms} type:terms, field:${er_s}, limit:1, overrequest:0, numBuckets:true, refine:false}  }  }" +
+              ", cat2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , numBuckets:true, facet:{  er:{${terms} type:terms, field:${er_s}, limit:1, overrequest:0, numBuckets:true, refine:true }  }  }" +
+              "}"
+          )
+          , "facets=={ count:8" +
+              ", cat:{ numBuckets:2, buckets:[  {val:A, count:3, er:{numBuckets:1,buckets:[{count:2, val:E}]  }}]  }" +  // the "R" bucket will not be seen w/o refinement
+              ",cat2:{ numBuckets:2, buckets:[  {val:A, count:4, er:{numBuckets:2,buckets:[{count:2, val:E}]  }}]  }" +
+              "}"
+      );
+
+      final String sort_limit_over = "sort:'count desc', limit:1, overrequest:0, ";
+      // simplistic join domain testing: no refinement == low count
+      client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
+          "json.facet", "{" +
+              "  cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:false," +
+              // self join on all_s ensures every doc on every shard included in facets
+              "        domain: { join: { from:all_s, to:all_s } } }" +
+              "}"
+          )
+          ,
+          "/response/numFound==3",
+          "facets=={ count:3, " +
+              // w/o overrequest and refinement, count for 'A' is lower than it should be
+              // (we don't see the A from the middle shard)
+              "          cat0:{ buckets:[ {val:A,count:3} ] } }");
+      // simplistic join domain testing: refinement == correct count
+      client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
+          "json.facet", "{" +
+              "  cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:true," +
+              // self join on all_s ensures every doc on every shard included in facets
+              "        domain: { join: { from:all_s, to:all_s } } }" +
+              "}"
+          )
+          ,
+          "/response/numFound==3",
+          "facets=={ count:3," +
+              // w/o overrequest, we need refining to get the correct count for 'A'.
+              "          cat0:{ buckets:[ {val:A,count:4} ] } }");
+
+      // contrived join domain + refinement (at second level) + testing
+      client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
+          "json.facet", "{" +
+              // top level facet has a single term
+              "  all:{${terms} type:terms, field:all_s, " + sort_limit_over + " refine:true, " +
+              "       facet:{  " +
+              // subfacet will facet on cat after joining on all (so all docs should be included in subfacet)
+              "         cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:true," +
+              "               domain: { join: { from:all_s, to:all_s } } } } }" +
+              "}"
+          )
+          ,
+          "/response/numFound==3",
+          "facets=={ count:3," +
+              // all 3 docs matching base query have same 'all' value in top facet
+              "          all:{ buckets:[ { val:all, count:3, " +
+              // sub facet has refinement, so count for 'A' should be correct
+              "                            cat0:{ buckets: [{val:A,count:4}] } } ] } }");
+
+    } // end method loop
   }
 
   // Unlike solrconfig.xml this test using solrconfig-tlog.xml should not fail with too-many-exceptions (see TestSolrQueryParser.testManyClauses)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1067cabc/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
index 65d4e75..b6afdb8 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
@@ -760,10 +760,15 @@ public class TestJsonFacets extends SolrTestCaseHS {
     client.testJQ(params(p, "q", "*:*"
         , "json.facet", "{" +
             " f1:{${terms}  type:field, field:${date}}" +
+            ",f2:{${terms}  type:field, field:${date} sort:'index asc'}" +
+            ",f3:{${terms}  type:field, field:${date} sort:'index desc'}" +
+            // ",f4:{${terms}  type:field, field:${date}, facet:{x:{type:field,field:${num_is},limit:1}}     }" +
             "}"
         )
         , "facets=={count:6 " +
             ",f1:{ buckets:[ {val:'2001-01-01T01:01:01Z', count:1},{val:'2001-02-03T01:02:03Z', count:1},{val:'2002-02-02T02:02:02Z', count:1},{val:'2002-03-01T03:02:01Z', count:1},{val:'2003-03-03T03:03:03Z', count:1} ] }" +
+            ",f2:{ buckets:[ {val:'2001-01-01T01:01:01Z', count:1},{val:'2001-02-03T01:02:03Z', count:1},{val:'2002-02-02T02:02:02Z', count:1},{val:'2002-03-01T03:02:01Z', count:1},{val:'2003-03-03T03:03:03Z', count:1} ] }" +
+            ",f3:{ buckets:[ {val:'2003-03-03T03:03:03Z', count:1},{val:'2002-03-01T03:02:01Z', count:1},{val:'2002-02-02T02:02:02Z', count:1},{val:'2001-02-03T01:02:03Z', count:1},{val:'2001-01-01T01:01:01Z', count:1} ] }" +
             "}"
     );