You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ge...@apache.org on 2018/11/18 15:11:42 UTC

[1/2] lucene-solr:master: SOLR-12981: Introduce SolrJ type for JSON facet response parsing

Repository: lucene-solr
Updated Branches:
  refs/heads/master dc2c7784e -> d17c6d889


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java
index 7717f7f..344fdd2 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java
@@ -19,7 +19,9 @@ package org.apache.solr.client.solrj.request.json;
 
 import java.io.File;
 import java.io.IOException;
+import java.time.Instant;
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -27,12 +29,13 @@ import java.util.Map;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
+import org.apache.solr.client.solrj.response.json.BucketJsonFacet;
+import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.ExternalPaths;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -43,6 +46,7 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
   private static final String CONFIG_NAME = "techproducts_config";
   private static final int NUM_TECHPRODUCTS_DOCS = 32;
   private static final int NUM_IN_STOCK = 17;
+  private static final int NUM_CATEGORIES = 16;
   private static final int NUM_ELECTRONICS = 12;
   private static final int NUM_CURRENCY = 4;
   private static final int NUM_MEMORY = 3;
@@ -79,15 +83,53 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"top_cats",
-        new FacetBucket("electronics",NUM_ELECTRONICS),
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData, "top_cats",
+        new FacetBucket("electronics", NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
+  }
+
+  @Test
+  public void testTermsFacetWithNumBucketsRequested() throws Exception {
+    final TermsFacetMap categoriesFacetMap = new TermsFacetMap("cat")
+        .includeTotalNumBuckets(true)
+        .setLimit(3);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_cats", categoriesFacetMap);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData, "top_cats",
+        new FacetBucket("electronics", NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
+    assertEquals(NUM_CATEGORIES, topLevelFacetData.getBucketBasedFacets("top_cats").getNumBuckets());
+  }
+
+  @Test
+  public void testTermsFacetWithAllBucketsRequested() throws Exception {
+    final TermsFacetMap categoriesFacetMap = new TermsFacetMap("cat")
+        .includeAllBucketsUnionBucket(true)
+        .setLimit(3);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_cats", categoriesFacetMap);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData, "top_cats",
+        new FacetBucket("electronics", NUM_ELECTRONICS),
         new FacetBucket("currency", NUM_CURRENCY),
         new FacetBucket("memory", NUM_MEMORY));
+    assertEquals(37, topLevelFacetData.getBucketBasedFacets("top_cats").getAllBuckets());
   }
 
   @Test
@@ -106,13 +148,11 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"top_cats",
-        new FacetBucket("electronics",NUM_ELECTRONICS),
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData, "top_cats",
+        new FacetBucket("electronics", NUM_ELECTRONICS),
         new FacetBucket("currency", NUM_CURRENCY),
         new FacetBucket("memory", NUM_MEMORY));
   }
@@ -130,17 +170,15 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"top_cats",
-        new FacetBucket("electronics",NUM_ELECTRONICS),
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData, "top_cats",
+        new FacetBucket("electronics", NUM_ELECTRONICS),
         new FacetBucket("currency", NUM_CURRENCY),
         new FacetBucket("memory", NUM_MEMORY));
-    assertHasFacetWithBucketValues(rawResponse,"top_manufacturers",
-        new FacetBucket("corsair",NUM_CORSAIR),
+    assertHasFacetWithBucketValues(topLevelFacetData, "top_manufacturers",
+        new FacetBucket("corsair", NUM_CORSAIR),
         new FacetBucket("belkin", NUM_BELKIN),
         new FacetBucket("canon", NUM_CANON));
   }
@@ -154,12 +192,10 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"prices",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData,"prices",
         new FacetBucket(0.0f, 5),
         new FacetBucket(20.0f, 0),
         new FacetBucket(40.0f, 0),
@@ -168,6 +204,27 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testSingleDateRangeFacet() throws Exception {
+    final Date startDate = new Date(Instant.parse("2005-08-01T16:30:25Z").toEpochMilli());
+    final Date endDate = new Date(Instant.parse("2006-02-13T15:26:37Z").toEpochMilli());
+    final RangeFacetMap manufactureDateFacet = new RangeFacetMap("manufacturedate_dt", startDate, endDate, "+1MONTH")
+        .setMinCount(1);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("man_date", manufactureDateFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData,"man_date",
+        new FacetBucket(new Date(Instant.parse("2005-08-01T16:30:25Z").toEpochMilli()), 1),
+        new FacetBucket(new Date(Instant.parse("2005-10-01T16:30:25Z").toEpochMilli()), 1),
+        new FacetBucket(new Date(Instant.parse("2006-02-01T16:30:25Z").toEpochMilli()), 9));
+  }
+
+  @Test
   public void testMultiRangeFacet() throws Exception {
     final RangeFacetMap pricesFacet = new RangeFacetMap("price", 0, 100, 20);
     final RangeFacetMap shippingWeightFacet = new RangeFacetMap("weight", 0, 200, 50);
@@ -178,18 +235,16 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"prices",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData,"prices",
         new FacetBucket(0.0f, 5),
         new FacetBucket(20.0f, 0),
         new FacetBucket(40.0f, 0),
         new FacetBucket(60.0f, 1),
         new FacetBucket(80.0f, 1));
-    assertHasFacetWithBucketValues(rawResponse, "shipping_weights",
+    assertHasFacetWithBucketValues(topLevelFacetData, "shipping_weights",
         new FacetBucket(0.0f, 6),
         new FacetBucket(50.0f, 0),
         new FacetBucket(100.0f, 0),
@@ -204,12 +259,10 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasStatFacetWithValue(rawResponse,"sum_price", 5251.270030975342);
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasStatFacetWithValue(topLevelFacetData,"sum_price", 5251.270030975342);
   }
 
   @Test
@@ -221,13 +274,10 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasStatFacetWithValue(rawResponse,"sum_price", 5251.270030975342);
-    assertHasStatFacetWithValue(rawResponse,"avg_price", 328.20437693595886);
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasStatFacetWithValue(topLevelFacetData,"sum_price", 5251.270030975342);
+    assertHasStatFacetWithValue(topLevelFacetData,"avg_price", 328.20437693595886);
   }
 
   @Test
@@ -241,14 +291,13 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasStatFacetWithValue(rawResponse,"avg_price", 328.20437693595886);
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasStatFacetWithValue(topLevelFacetData,"avg_price", 328.20437693595886);
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
   }
 
   @Test
@@ -262,22 +311,21 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
     // Test top level facets
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
     // Test subfacet values for each top-level facet bucket
-    final List<NamedList<Object>> topLevelFacetResponse = (List<NamedList<Object>>) rawResponse.findRecursive("facets", "top_cats", "buckets");
-    final NamedList<Object> electronicsSubFacet = topLevelFacetResponse.get(0);
-    assertFacetResponseHasFacetWithBuckets(electronicsSubFacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
-    final NamedList<Object> currencySubfacet = topLevelFacetResponse.get(1);
-    assertFacetResponseHasFacetWithBuckets(currencySubfacet, "top_manufacturers_for_cat", new FacetBucket("boa", 1));
-    final NamedList<Object> memorySubfacet = topLevelFacetResponse.get(2);
-    assertFacetResponseHasFacetWithBuckets(memorySubfacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
+    final List<BucketJsonFacet> topCatsBuckets = topLevelFacetData.getBucketBasedFacets("top_cats").getBuckets();
+    final NestableJsonFacet electronicsFacet = topCatsBuckets.get(0);
+    assertHasFacetWithBucketValues(electronicsFacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
+    final NestableJsonFacet currencyFacet = topCatsBuckets.get(1);
+    assertHasFacetWithBucketValues(currencyFacet, "top_manufacturers_for_cat", new FacetBucket("boa", 1));
+    final NestableJsonFacet memoryFacet = topCatsBuckets.get(2);
+    assertHasFacetWithBucketValues(memoryFacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
   }
 
   @Test
@@ -293,22 +341,18 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
     // Test top level facets
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
     // Test subfacet values for each top-level facet bucket
-    final List<NamedList<Object>> topLevelFacetResponse = (List<NamedList<Object>>) rawResponse.findRecursive("facets", "top_cats", "buckets");
-    final NamedList<Object> electronicsSubFacet = topLevelFacetResponse.get(0);
-    assertFacetResponseHasStatFacetWithValue(electronicsSubFacet, subfacetName, 252.02909261530095);
-    final NamedList<Object> currencySubfacet = topLevelFacetResponse.get(1);
-    assertFacetResponseHasStatFacetWithValue(currencySubfacet, subfacetName, 0.0);
-    final NamedList<Object> memorySubfacet = topLevelFacetResponse.get(2);
-    assertFacetResponseHasStatFacetWithValue(memorySubfacet, subfacetName, 129.99499893188477);
+    final List<BucketJsonFacet> topCatsResultBuckets = topLevelFacetData.getBucketBasedFacets("top_cats").getBuckets();
+    assertHasStatFacetWithValue(topCatsResultBuckets.get(0), subfacetName, 252.02909261530095); // electronics
+    assertHasStatFacetWithValue(topCatsResultBuckets.get(1), subfacetName, 0.0); // currency
+    assertHasStatFacetWithValue(topCatsResultBuckets.get(2), subfacetName, 129.99499893188477); // memory
   }
 
   @Test
@@ -323,13 +367,12 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"top_popular_cats", new FacetBucket("electronics",9),
-        new FacetBucket("graphics card", 2), new FacetBucket("hard drive", 2));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_popular_cats",
+        new FacetBucket("electronics",9),
+        new FacetBucket("graphics card", 2),
+        new FacetBucket("hard drive", 2));
   }
 
   @Test
@@ -345,13 +388,12 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"top_popular_cats", new FacetBucket("electronics",9),
-        new FacetBucket("graphics card", 2), new FacetBucket("hard drive", 2));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_popular_cats",
+        new FacetBucket("electronics",9),
+        new FacetBucket("graphics card", 2),
+        new FacetBucket("hard drive", 2));
   }
 
   @Test
@@ -366,13 +408,12 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_ELECTRONICS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
   }
 
   @Test
@@ -386,16 +427,14 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"largest_search_cats",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_ELECTRONICS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"largest_search_cats",
         new FacetBucket("search",2),
         new FacetBucket("software", 2));
   }
 
+  @Test
   public void testFacetWithMultipleSimpleQueryClausesInArbitraryDomain() throws Exception {
     final TermsFacetMap solrCategoriesFacet = new TermsFacetMap("cat")
         .withDomain(new DomainMap()
@@ -407,16 +446,14 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"cats_matching_solr",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_ELECTRONICS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"cats_matching_solr",
         new FacetBucket("search",1),
         new FacetBucket("software", 1));
   }
 
+  @Test
   public void testFacetWithMultipleLocalParamsQueryClausesInArbitraryDomain() throws Exception {
     final TermsFacetMap solrCategoriesFacet = new TermsFacetMap("cat")
         .withDomain(new DomainMap()
@@ -428,12 +465,9 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"cats_matching_solr",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_ELECTRONICS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"cats_matching_solr",
         new FacetBucket("search",1),
         new FacetBucket("software", 1));
   }
@@ -454,19 +488,39 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
 
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_IN_STOCK, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"in_stock_only",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_IN_STOCK, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"in_stock_only",
         new FacetBucket("electronics",8),
         new FacetBucket("currency", 4));
-    assertHasFacetWithBucketValues(rawResponse,"all",
+    assertHasFacetWithBucketValues(topLevelFacetData  ,"all",
         new FacetBucket("electronics",12),
         new FacetBucket("currency", 4));
   }
 
+  @Test
+  public void testRangeFacetWithOtherBucketsRequested() throws Exception {
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("price_range",
+            new RangeFacetMap("price", 0, 100, 20)
+                .setOtherBuckets(RangeFacetMap.OtherBuckets.ALL)
+        );
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+
+    assertHasFacetWithBucketValues(topLevelFacetData, "price_range",
+        new FacetBucket(0.0f, 5),
+        new FacetBucket(20.0f, 0),
+        new FacetBucket(40.0f, 0),
+        new FacetBucket(60.0f, 1),
+        new FacetBucket(80.0f, 1));
+    assertEquals(0, topLevelFacetData.getBucketBasedFacets("price_range").getBefore());
+    assertEquals(9, topLevelFacetData.getBucketBasedFacets("price_range").getAfter());
+    assertEquals(7, topLevelFacetData.getBucketBasedFacets("price_range").getBetween());
+  }
+
   private class FacetBucket {
     private final Object val;
     private final int count;
@@ -479,52 +533,29 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
     public int getCount() { return count; }
   }
 
-  private void assertHasFacetWithBucketValues(NamedList<Object> rawResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
-    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
-    assertFacetResponseHasFacetWithBuckets(facetsTopLevel, expectedFacetName, expectedBuckets);
-  }
-
-  private void assertHasStatFacetWithValue(NamedList<Object> rawResponse, String expectedFacetName, Double expectedStatValue) {
-    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
-    assertFacetResponseHasStatFacetWithValue(facetsTopLevel, expectedFacetName, expectedStatValue);
-  }
-
-  private NamedList<Object> assertHasFacetResponse(NamedList<Object> topLevelResponse) {
-    Object o = topLevelResponse.get("facets");
-    if (o == null) fail("Response has no top-level 'facets' property as expected");
-    if (!(o instanceof NamedList)) fail("Response has a top-level 'facets' property, but it is not a NamedList");
-
-    return (NamedList<Object>) o;
-  }
-
-  private void assertFacetResponseHasFacetWithBuckets(NamedList<Object> facetResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
-    Object o = facetResponse.get(expectedFacetName);
-    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
-    if (!(o instanceof NamedList)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a NamedList");
-
-    final NamedList<Object> expectedFacetTopLevel = (NamedList<Object>) o;
-    o = expectedFacetTopLevel.get("buckets");
-    if (o == null) fail("Response has no 'buckets' property under 'facets'");
-    if (!(o instanceof List)) fail("Response has no 'buckets' property containing actual facet information.");
-
-    final List<NamedList> bucketList = (List<NamedList>) o;
-    assertEquals("Expected " + expectedBuckets.length + " buckets, but found " + bucketList.size(),
-        expectedBuckets.length, bucketList.size());
+  private void assertHasFacetWithBucketValues(NestableJsonFacet response, String expectedFacetName, FacetBucket... expectedBuckets) {
+    assertTrue("Expected response to have facet with name " + expectedFacetName,
+        response.getBucketBasedFacets(expectedFacetName) != null);
+    final List<BucketJsonFacet> buckets = response.getBucketBasedFacets(expectedFacetName).getBuckets();
+    assertEquals(expectedBuckets.length, buckets.size());
     for (int i = 0; i < expectedBuckets.length; i++) {
       final FacetBucket expectedBucket = expectedBuckets[i];
-      final NamedList<Object> actualBucket = bucketList.get(i);
-      assertEquals(expectedBucket.getVal(), actualBucket.get("val"));
-      assertEquals(expectedBucket.getCount(), actualBucket.get("count"));
+      final BucketJsonFacet actualBucket = buckets.get(i);
+      assertEquals(expectedBucket.getVal(), actualBucket.getVal());
+      assertEquals(expectedBucket.getCount(), actualBucket.getCount());
     }
   }
 
-  private void assertFacetResponseHasStatFacetWithValue(NamedList<Object> facetResponse, String expectedFacetName, Double expectedStatValue) {
-    Object o = facetResponse.get(expectedFacetName);
-    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
-    if (!(o instanceof Number)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a Number");
+  private void assertHasStatFacetWithValue(NestableJsonFacet response, String expectedFacetName, Double expectedStatValue) {
+    assertTrue("Expected response to have stat facet named '" + expectedFacetName + "'",
+        response.getStatFacetValue(expectedFacetName) != null);
+    assertEquals(expectedStatValue, response.getStatFacetValue(expectedFacetName));
+  }
 
-    final Number actualStatValueAsNumber = (Number) o;
-    final Double actualStatValueAsDouble = ((Number) o).doubleValue();
-    assertEquals(expectedStatValue, actualStatValueAsDouble, 0.5);
+  private void assertExpectedDocumentsFoundAndReturned(QueryResponse response, int expectedNumFound, int expectedReturned) {
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList documents = response.getResults();
+    assertEquals(expectedNumFound, documents.getNumFound());
+    assertEquals(expectedReturned, documents.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestHeatmapFacetingTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestHeatmapFacetingTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestHeatmapFacetingTest.java
new file mode 100644
index 0000000..1ccd581
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestHeatmapFacetingTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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.client.solrj.request.json;
+
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.json.HeatmapJsonFacet;
+import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class JsonQueryRequestHeatmapFacetingTest extends SolrCloudTestCase {
+  private static final String COLLECTION_NAME = "spatialdata";
+  private static final String CONFIG_NAME = "spatialdata_config";
+  private static final String FIELD = "location_srpt";
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig(CONFIG_NAME, new File(ExternalPaths.SOURCE_HOME, "solrj/src/test-files/solrj/solr/configsets/spatial/conf").toPath())
+        .configure();
+
+    final List<String> solrUrls = new ArrayList<>();
+    solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
+
+    CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
+
+    indexSpatialData();
+  }
+
+  private static void indexSpatialData() throws Exception {
+    final SolrInputDocument doc1 = new SolrInputDocument("id", "0", FIELD, "ENVELOPE(100, 120, 80, 40)");
+    final SolrInputDocument doc2 = new SolrInputDocument("id", "1", FIELD, "ENVELOPE(-120, -110, 80, 20)");
+    final SolrInputDocument doc3 = new SolrInputDocument("id", "3", FIELD, "POINT(70 60)");
+    final SolrInputDocument doc4 = new SolrInputDocument("id", "4", FIELD, "POINT(91 89)");
+    final List<SolrInputDocument> docs = new ArrayList<>();
+    docs.add(doc1);
+    docs.add(doc2);
+    docs.add(doc3);
+    docs.add(doc4);
+
+    cluster.getSolrClient().add(COLLECTION_NAME, docs);
+    cluster.getSolrClient().commit(COLLECTION_NAME);
+  }
+
+
+
+  @Test
+  public void testHeatmapFacet() throws Exception {
+    final List<List<Integer>> expectedHeatmapGrid = Arrays.asList(
+        Arrays.asList(0, 0, 2, 1, 0, 0),
+        Arrays.asList(0, 0, 1, 1, 0, 0),
+        Arrays.asList(0, 1, 1, 1, 0, 0),
+        Arrays.asList(0, 0, 1, 1, 0, 0),
+        Arrays.asList(0, 0, 1, 1, 0, 0),
+        null,
+        null
+    );
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .setLimit(0)
+        .withFacet("heatmapFacet", new HeatmapFacetMap(FIELD)
+            .setHeatmapFormat(HeatmapFacetMap.HeatmapFormat.INTS2D)
+            .setRegionQuery("[\"50 20\" TO \"180 90\"]")
+            .setGridLevel(4)
+        );
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+    final NestableJsonFacet topLevelFacet = response.getJsonFacetingResponse();
+    final HeatmapJsonFacet heatmap = topLevelFacet.getHeatmapFacetByName("heatmapFacet");
+    final List<List<Integer>> actualHeatmapGrid = heatmap.getCountGrid();
+    assertEquals(expectedHeatmapGrid, actualHeatmapGrid);
+  }
+}


[2/2] lucene-solr:master: SOLR-12981: Introduce SolrJ type for JSON facet response parsing

Posted by ge...@apache.org.
SOLR-12981: Introduce SolrJ type for JSON facet response parsing

Recent JIRA's (SOLR-12947, SOLR-12965) have added support making it
easier to compose JSON query/faceting requests using SolrJ.  But neither
made parsing the responses to these queries any easier.

This commit introduces NestableJsonFacet (along with several companion
types) which are Java representations of the JSON faceting response.
They can be accessed via the new QueryResponse method:
`getJsonFacetingResponse()`.


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

Branch: refs/heads/master
Commit: d17c6d8897aad4b06c9c7200d0c445e1170e503f
Parents: dc2c778
Author: Jason Gerlowski <ge...@apache.org>
Authored: Sat Nov 17 08:25:16 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Sat Nov 17 17:14:40 2018 -0500

----------------------------------------------------------------------
 .../solrj/request/json/JsonQueryRequest.java    |   9 +-
 .../solrj/request/json/RangeFacetMap.java       |  38 ++
 .../client/solrj/response/QueryResponse.java    |  18 +
 .../response/json/BucketBasedJsonFacet.java     | 139 +++++++
 .../solrj/response/json/BucketJsonFacet.java    |  63 ++++
 .../solrj/response/json/HeatmapJsonFacet.java   | 125 ++++++
 .../solrj/response/json/NestableJsonFacet.java  | 140 +++++++
 .../solrj/response/json/package-info.java       |  21 ++
 .../solr/configsets/spatial/conf/schema.xml     |  32 ++
 .../solr/configsets/spatial/conf/solrconfig.xml |  57 +++
 .../ref_guide_examples/JsonRequestApiTest.java  |  47 +--
 ...JsonQueryRequestFacetingIntegrationTest.java | 317 +++++++---------
 ...JsonQueryRequestFacetingIntegrationTest.java | 377 ++++++++++---------
 .../JsonQueryRequestHeatmapFacetingTest.java    |  98 +++++
 14 files changed, 1085 insertions(+), 396 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
index 1c7b071..cea678d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
@@ -149,7 +149,8 @@ public class JsonQueryRequest extends QueryRequest {
    *     jsonQueryRequest.withStatFacet("top_cats", catFacetMap);
    * }</pre>
    *
-   * @param facetName the name of the top-level facet you'd like to add.
+   * @param facetName the name of the top-level facet you'd like to add.  Avoid choosing facet names which overload
+   *                  properties already present in the JSON response schema (e.g. "count", "val", "minX", etc.)
    * @param facetJson a Map of values representing the facet you wish to add to the request
    */
   public JsonQueryRequest withFacet(String facetName, Map<String, Object> facetJson) {
@@ -188,7 +189,8 @@ public class JsonQueryRequest extends QueryRequest {
    *     };
    * </pre>
    *
-   * @param facetName the name of the top-level facet you'd like to add.
+   * @param facetName the name of the top-level facet you'd like to add.  Avoid choosing facet names which overload
+   *                  properties already present in the JSON response schema (e.g. "count", "val", "minX", etc.)
    * @param facetWriter a MapWriter representing the facet you wish to add to the request
    */
   public JsonQueryRequest withFacet(String facetName, MapWriter facetWriter) {
@@ -221,7 +223,8 @@ public class JsonQueryRequest extends QueryRequest {
    *     jsonQueryRequest.withStatFacet("avg_price", "avg(price)");
    * }</pre>
    *
-   * @param facetName the name of the top-level stat/agg facet you'd like to add.
+   * @param facetName the name of the top-level stat/agg facet you'd like to add.  Avoid choosing facet names which overload
+   *                  properties already present in the JSON response schema (e.g. "count", "val", "minX", etc.)
    * @param facetValue a String representing the stat/agg facet computation to perform.
    */
   public JsonQueryRequest withStatFacet(String facetName, String facetValue) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java
index 24d5123..ae4d23e 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.client.solrj.request.json;
 
+import java.util.Date;
 import java.util.Map;
 
 /**
@@ -63,6 +64,30 @@ public class RangeFacetMap extends JsonFacetMap<RangeFacetMap> {
     put("gap", gap);
   }
 
+  /**
+   * Creates a "range" facet representation for a date field
+   *
+   * @param gap a DateMathParser compatible time/interval String.  (e.g. "+1MONTH")
+   */
+  public RangeFacetMap(String field, Date start, Date end, String gap) {
+    super("range");
+
+    if (field == null) {
+      throw new IllegalArgumentException("Parameter 'field' must be non-null");
+    }
+    if (start == null) {
+      throw new IllegalArgumentException("Parameter 'start' must be non-null");
+    }
+    if (end == null) {
+      throw new IllegalArgumentException("Parameter 'gap' must be non-null");
+    }
+
+    put("field", field);
+    put("start", start);
+    put("end", end);
+    put("gap", gap);
+  }
+
   @Override
   public RangeFacetMap getThis() { return this; }
 
@@ -102,4 +127,17 @@ public class RangeFacetMap extends JsonFacetMap<RangeFacetMap> {
     put("other", bucketSpecifier.toString());
     return this;
   }
+
+  /**
+   * Indicates that buckets should be returned only if they have a count of at least {@code minOccurrences}
+   *
+   * Defaults to '0' if not specified.
+   */
+  public RangeFacetMap setMinCount(int minOccurrences) {
+    if (minOccurrences < 0) {
+      throw new IllegalArgumentException(" Parameter 'minOccurrences' must be non-negative");
+    }
+    put("mincount", minOccurrences);
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java
index 3124a0b..cdb64e3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java
@@ -26,6 +26,7 @@ import java.util.TreeMap;
 
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.beans.DocumentObjectBinder;
+import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.params.CursorMarkParams;
 import org.apache.solr.common.util.NamedList;
@@ -48,6 +49,7 @@ public class QueryResponse extends SolrResponseBase
   private NamedList<Object> _highlightingInfo = null;
   private NamedList<Object> _spellInfo = null;
   private List<NamedList<Object>> _clusterInfo = null;
+  private NamedList<Object> _jsonFacetingInfo = null;
   private Map<String,NamedList<Object>> _suggestInfo = null;
   private NamedList<Object> _statsInfo = null;
   private NamedList<NamedList<Object>> _termsInfo = null;
@@ -79,6 +81,9 @@ public class QueryResponse extends SolrResponseBase
   // Clustering Response
   private ClusteringResponse _clusterResponse = null;
 
+  // Json Faceting Response
+  private NestableJsonFacet _jsonFacetingResponse = null;
+
   // Suggester Response
   private SuggesterResponse _suggestResponse = null;
 
@@ -157,6 +162,10 @@ public class QueryResponse extends SolrResponseBase
         _clusterInfo = (ArrayList<NamedList<Object>>) res.getVal(i);
         extractClusteringInfo(_clusterInfo);
       }
+      else if ("facets".equals(n)) {
+        _jsonFacetingInfo = (NamedList<Object>) res.getVal(i);
+        // Don't call extractJsonFacetingInfo(_jsonFacetingInfo) here in an effort to do it lazily
+      }
       else if ( "suggest".equals( n ) )  {
         _suggestInfo = (Map<String,NamedList<Object>>) res.getVal( i );
         extractSuggesterInfo(_suggestInfo);
@@ -187,6 +196,10 @@ public class QueryResponse extends SolrResponseBase
     _clusterResponse = new ClusteringResponse(clusterInfo);
   }
 
+  private void extractJsonFacetingInfo(NamedList<Object> facetInfo) {
+    _jsonFacetingResponse = new NestableJsonFacet(facetInfo);
+  }
+
   private void extractSuggesterInfo(Map<String, NamedList<Object>> suggestInfo) {
     _suggestResponse = new SuggesterResponse(suggestInfo);
   }
@@ -554,6 +567,11 @@ public class QueryResponse extends SolrResponseBase
     return _clusterResponse;
   }
 
+  public NestableJsonFacet getJsonFacetingResponse() {
+    if (_jsonFacetingInfo != null && _jsonFacetingResponse == null) extractJsonFacetingInfo(_jsonFacetingInfo);
+    return _jsonFacetingResponse;
+  }
+
   public SuggesterResponse getSuggesterResponse() {
     return _suggestResponse;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/BucketBasedJsonFacet.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/BucketBasedJsonFacet.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/BucketBasedJsonFacet.java
new file mode 100644
index 0000000..29b67a3
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/BucketBasedJsonFacet.java
@@ -0,0 +1,139 @@
+/*
+ * 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.client.solrj.response.json;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Represents the top-level response for a bucket-based JSON facet (i.e. "terms" or "range")
+ *
+ * Allows access to JSON like:
+ * <pre>
+ *   {
+ *     "numBuckets": 2,
+ *     "buckets": [
+ *       {...},
+ *       {...}
+ *     ]
+ *   }
+ * </pre>
+ * <p>
+ * Allows access to all top-level "terms" and "range" response properties (e.g. {@code allBuckets}, {@code numBuckets},
+ * {@code before}, etc.)
+ */
+public class BucketBasedJsonFacet {
+  public static final int UNSET_FLAG = -1;
+  private List<BucketJsonFacet> buckets;
+  private int numBuckets = UNSET_FLAG;
+  private long allBuckets = UNSET_FLAG;
+  private int beforeFirstBucketCount = UNSET_FLAG;
+  private int afterLastBucketCount = UNSET_FLAG;
+  private int betweenAllBucketsCount = UNSET_FLAG;
+
+  public BucketBasedJsonFacet(NamedList<Object> bucketBasedFacet) {
+    for (Map.Entry<String, Object> entry : bucketBasedFacet) {
+      final String key = entry.getKey();
+      final Object value = entry.getValue();
+      if ("buckets".equals(key)) {
+        final List<NamedList> bucketListUnparsed = (List<NamedList>) value;
+        buckets = new ArrayList<>();
+        for (NamedList bucket : bucketListUnparsed) {
+          buckets.add(new BucketJsonFacet(bucket));
+        }
+      } else if ("numBuckets".equals(key)) {
+        numBuckets = (int) value;
+      } else if ("allBuckets".equals(key)) {
+        allBuckets = (long) ((NamedList)value).get("count");
+      } else if ("before".equals(key)) {
+        beforeFirstBucketCount = (int) ((NamedList)value).get("count");
+      } else if ("after".equals(key)) {
+        afterLastBucketCount = (int) ((NamedList)value).get("count");
+      } else if ("between".equals(key)) {
+        betweenAllBucketsCount = (int) ((NamedList)value).get("count");
+      } else {
+        // We don't recognize the key.  Possible JSON faceting schema has changed without updating client.
+        // Silently ignore for now, though we may want to consider throwing an error if this proves problematic.
+      }
+    }
+  }
+
+  /**
+   * Retrieves the facet buckets returned by the server.
+   */
+  public List<BucketJsonFacet> getBuckets() {
+    return buckets;
+  }
+
+  /**
+   * The total number of buckets found in the domain (of which the returned buckets are only a part).
+   *
+   * This value can only be computed on "terms" facets where the user has specifically requested it with the
+   * {@code numBuckets} option.  {@link #UNSET_FLAG} is returned if this is a "range" facet or {@code numBuckets}
+   * computation was not requested in the intiial request.
+   */
+  public int getNumBuckets() {
+    return numBuckets;
+  }
+
+  /**
+   * The sum cardinality of all buckets in the "terms" facet.
+   *
+   * Note that for facets on multi-valued fields, documents may belong to multiple buckets, making {@link #getAllBuckets()}
+   * return a result greater than the number of documents in the domain.
+   * <p>
+   * This value is only present if the user has specifically requested it with the {@code allBuckets} option.
+   * {@link #UNSET_FLAG} is returned if this is not the case.
+   */
+  public long getAllBuckets() {
+    return allBuckets;
+  }
+
+  /**
+   * The count of all records whose field value precedes the {@code start} of this "range" facet
+   *
+   * This value is only present if the user has specifically requested it with the {@code other} option.
+   * {@link #UNSET_FLAG} is returned if this is not the case.
+   */
+  public int getBefore() {
+    return beforeFirstBucketCount;
+  }
+
+  /**
+   * The count of all records whose field value follows the {@code end} of this "range" facet
+   *
+   * This value is only present if the user has specifically requested it with the {@code other} option.
+   * {@link #UNSET_FLAG} is returned if this is not the case.
+   */
+  public int getAfter() {
+    return afterLastBucketCount;
+  }
+
+  /**
+   * The count of all records whose field value falls between {@code start} and {@code end}.
+   *
+   * This value is only present if the user has specifically requested it with the {@code other} option.
+   * {@link #UNSET_FLAG} is returned if this is not the case.
+   */
+  public int getBetween() {
+    return betweenAllBucketsCount;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/BucketJsonFacet.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/BucketJsonFacet.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/BucketJsonFacet.java
new file mode 100644
index 0000000..1786f26
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/BucketJsonFacet.java
@@ -0,0 +1,63 @@
+/*
+ * 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.client.solrj.response.json;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Represents an individual bucket result of a "term" or "range" facet.
+ *
+ * Allows access to JSON like:
+ * <pre>
+ *   {
+ *     "val": "termX",
+ *     "count": 10,
+ *     "subfacetName": ...
+ *   }
+ * </pre>
+ * <p>
+ * Buckets may contain nested facets of any type.
+ */
+public class BucketJsonFacet extends NestableJsonFacet {
+  private Object val;
+
+  public BucketJsonFacet(NamedList<Object> singleBucket) {
+    super(singleBucket); // sets "count", and stats or nested facets
+
+    val = singleBucket.get("val");
+  }
+
+  /**
+   * Retrieves the value (sometimes called the "key") of this bucket.
+   *
+   * The type of this object depends on the type of field being faceted on.  Usually a Date, Double, Integer, or String
+   */
+  public Object getVal() {
+    return val;
+  }
+
+  @Override
+  protected Set<String> getKeysToSkip() {
+    final HashSet<String> keysToSkip = new HashSet<>();
+    keysToSkip.add("val");
+    return keysToSkip;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/HeatmapJsonFacet.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/HeatmapJsonFacet.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/HeatmapJsonFacet.java
new file mode 100644
index 0000000..0a6e680
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/HeatmapJsonFacet.java
@@ -0,0 +1,125 @@
+/*
+ * 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.client.solrj.response.json;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Represents the result of a "heatmap" JSON facet.
+ *
+ * Allows access to all top-level facet properties (e.g. {@code minX}, {@code maxY}, etc.) as well as the heatmap data
+ * itself in one of two forms.
+ */
+public class HeatmapJsonFacet {
+  private int gridLevel;
+  private int columns;
+  private int rows;
+  private double minX;
+  private double maxX;
+  private double minY;
+  private double maxY;
+  private List<List<Integer>> countGrid;
+  private String countEncodedAsBase64PNG;
+
+  public HeatmapJsonFacet(NamedList<Object> heatmapNL) {
+    gridLevel = (int) heatmapNL.get("gridLevel");
+    columns = (int) heatmapNL.get("columns");
+    rows = (int) heatmapNL.get("rows");
+    minX = (double) heatmapNL.get("minX");
+    maxX = (double) heatmapNL.get("maxX");
+    minY = (double) heatmapNL.get("minY");
+    maxY = (double) heatmapNL.get("maxY");
+    System.out.println("Rows is: " + rows);
+    System.out.println("Cols is " + columns);
+    System.out.println("Whole deal is: " + heatmapNL);
+
+    if (heatmapNL.get("counts_ints2D") == null) {
+      countEncodedAsBase64PNG = (String) heatmapNL.get("counts_png");
+    } else {
+      countGrid = (List<List<Integer>>) heatmapNL.get("counts_ints2D");
+    }
+  }
+
+  private int getNumCols(List<List<Integer>> grid) {
+    for (List<Integer> row : grid) {
+      if (row !=null ) return row.size();
+    }
+    throw new IllegalStateException("All rows in heatmap grid were null!");
+  }
+
+  public List<List<Integer>> getCountGrid() {
+    return countGrid;
+  }
+
+  public String getCountPng() {
+    return countEncodedAsBase64PNG;
+  }
+
+  public int getGridLevel() { return gridLevel; }
+  public int getNumColumns() { return columns; }
+  public int getNumRows() { return rows; }
+  public double getMinX() { return minX; }
+  public double getMaxX() { return maxX; }
+  public double getMinY() { return minY; }
+  public double getMaxY() { return maxY; }
+
+  /**
+   * A NamedList is a proper "heatmap" response if it contains <i>all</i> expected properties
+   *
+   * We try to be rather strict in determining whether {@code potentialHeatmapValues} is a "heatmap".  Users can name
+   * subfacets arbitrarily, so having some names match those expected in a "heatmap" response could just be coincidence.
+   * <p>
+   * Heatmap facets do not support subfacets.
+   */
+  public static boolean isHeatmapFacet(NamedList<Object> potentialHeatmapValues) {
+    boolean hasGridLevel = false;
+    boolean hasColumns = false;
+    boolean hasRows = false;
+    boolean hasMinX = false;
+    boolean hasMaxX = false;
+    boolean hasMinY = false;
+    boolean hasMaxY = false;
+    boolean hasCountGrid = false;
+    for (Map.Entry<String, Object> entry : potentialHeatmapValues) {
+      String key = entry.getKey();
+      if ("gridLevel".equals(key)) {
+        hasGridLevel = true;
+      } else if ("columns".equals(key)) {
+        hasColumns = true;
+      } else if ("rows".equals(key)) {
+        hasRows = true;
+      } else if ("minX".equals(key)) {
+        hasMinX = true;
+      } else if ("maxX".equals(key)) {
+        hasMaxX = true;
+      } else if ("minY".equals(key)) {
+        hasMinY = true;
+      } else if ("maxY".equals(key)){
+        hasMaxY = true;
+      } else if (key != null && key.startsWith("counts_")) {
+        hasCountGrid = true;
+      }
+    }
+
+    return potentialHeatmapValues.size() == 8 && hasGridLevel && hasColumns && hasRows && hasMinX && hasMaxX && hasMinY
+        && hasMaxY && hasCountGrid;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/NestableJsonFacet.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/NestableJsonFacet.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/NestableJsonFacet.java
new file mode 100644
index 0000000..b700c2c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/NestableJsonFacet.java
@@ -0,0 +1,140 @@
+/*
+ * 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.client.solrj.response.json;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Represents the response to a "query" JSON facet.
+ *
+ * Relies on several other types to represent the variety of JSON facet responses.  The parsing of these responses
+ * relies partially on the JSON property names.  When naming your facets in your request, avoid choosing names that
+ * match existing values in the JSON faceting response schema, such as "count", "val", "minX", etc.
+ */
+public class NestableJsonFacet {
+  private long domainCount;
+  private final Map<String, NestableJsonFacet> queryFacetsByName;
+  private final Map<String, BucketBasedJsonFacet> bucketBasedFacetByName;
+  private final Map<String, Number> statFacetsByName;
+  private final Map<String, HeatmapJsonFacet> heatmapFacetsByName;
+
+  public NestableJsonFacet(NamedList<Object> facetNL) {
+    queryFacetsByName = new HashMap<>();
+    bucketBasedFacetByName = new HashMap<>();
+    statFacetsByName = new HashMap<>();
+    heatmapFacetsByName = new HashMap<>();
+
+    for (Map.Entry<String, Object> entry : facetNL) {
+      final String key = entry.getKey();
+      if (getKeysToSkip().contains(key)) {
+        continue;
+      } else if ("count".equals(key)) {
+        domainCount = (int) entry.getValue();
+      } else if(entry.getValue() instanceof Number) { // Stat/agg facet value
+        statFacetsByName.put(key, (Number)entry.getValue());
+      } else if(entry.getValue() instanceof NamedList) { // Either heatmap/query/range/terms facet
+        final NamedList<Object> facet = (NamedList<Object>) entry.getValue();
+        final boolean isBucketBased = facet.get("buckets") != null;
+        final boolean isHeatmap = HeatmapJsonFacet.isHeatmapFacet(facet);
+        if (isBucketBased) {
+          bucketBasedFacetByName.put(key, new BucketBasedJsonFacet(facet));
+        } else if (isHeatmap) {
+          heatmapFacetsByName.put(key, new HeatmapJsonFacet(facet));
+        } else { // "query" facet
+          queryFacetsByName.put(key, new NestableJsonFacet(facet));
+        }
+      }
+    }
+  }
+
+  /**
+   * The number of records matching the domain of this facet.
+   */
+  public long getCount() {
+    return domainCount;
+  }
+
+  /**
+   * Retrieve a nested "query" facet by its name
+   */
+  public NestableJsonFacet getQueryFacet(String name) {
+    return queryFacetsByName.get(name);
+  }
+
+  /**
+   * @return the names of any "query" facets that are direct descendants of the current facet
+   */
+  public Set<String> getQueryFacetNames() {
+    return queryFacetsByName.keySet();
+  }
+
+  /**
+   * Retrieve a nested "terms" or "range" facet by its name.
+   */
+  public BucketBasedJsonFacet getBucketBasedFacets(String name) {
+    return bucketBasedFacetByName.get(name);
+  }
+
+  /**
+   * @return the names of any "terms" or "range" facets that are direct descendants of this facet
+   */
+  public Set<String> getBucketBasedFacetNames() {
+    return bucketBasedFacetByName.keySet();
+  }
+
+  /**
+   * Retrieve the value for a stat or agg facet with the provided name
+   */
+  public Number getStatFacetValue(String name) {
+    return statFacetsByName.get(name);
+  }
+
+  /**
+   * @return the names of any stat or agg facets that are direct descendants of this facet
+   */
+  public Set<String> getStatFacetNames() {
+    return statFacetsByName.keySet();
+  }
+
+  /**
+   * Retrieve a "heatmap" facet by its name
+   */
+  public HeatmapJsonFacet getHeatmapFacetByName(String name) {
+    return heatmapFacetsByName.get(name);
+  }
+
+  /**
+   * @return the names of any heatmap facets that are direct descendants of this facet
+   */
+  public Set<String> getHeatmapFacetNames() {
+    return heatmapFacetsByName.keySet();
+  }
+
+  /*
+   * Used by subclasses to control which keys are ignored during parsing.
+   */
+  protected Set<String> getKeysToSkip() {
+    final HashSet<String> keysToSkip = new HashSet<>();
+    return keysToSkip;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/package-info.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/package-info.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/package-info.java
new file mode 100644
index 0000000..914347f
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/json/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Allows parsing of JSON query/faceting API responses
+ */
+package org.apache.solr.client.solrj.response.json;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/test-files/solrj/solr/configsets/spatial/conf/schema.xml
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/spatial/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/configsets/spatial/conf/schema.xml
new file mode 100644
index 0000000..0386579
--- /dev/null
+++ b/solr/solrj/src/test-files/solrj/solr/configsets/spatial/conf/schema.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<!-- A trivial schema containing a s spatial type for SolrJ testing.
+     Feel free to add other spatial types to this as required
+-->
+<schema name="spatial" version="1.1">
+
+  <fieldType name="long" docValues="true" class="${solr.tests.LongFieldType}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="srpt_quad" class="solr.SpatialRecursivePrefixTreeFieldType" prefixTree="quad" distanceUnits="degrees" />
+  <fieldtype name="string" class="solr.StrField" sortMissingLast="true" docValues="true"/>
+
+  <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="false"/>
+  <field name="_version_" type="long" indexed="true" stored="true"/>
+
+  <dynamicField name="*_srpt" type="srpt_quad" indexed="true" stored="true"/>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/test-files/solrj/solr/configsets/spatial/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/spatial/conf/solrconfig.xml b/solr/solrj/src/test-files/solrj/solr/configsets/spatial/conf/solrconfig.xml
new file mode 100644
index 0000000..45c8613
--- /dev/null
+++ b/solr/solrj/src/test-files/solrj/solr/configsets/spatial/conf/solrconfig.xml
@@ -0,0 +1,57 @@
+<?xml version="1.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.
+-->
+
+<!-- This is a "kitchen sink" config file that tests can use.
+     When writting a new test, feel free to add *new* items (plugins,
+     config options, etc...) as long as they don't break any existing
+     tests.  if you need to test something esoteric please add a new
+     "solrconfig-your-esoteric-purpose.xml" config file.
+
+     Note in particular that this test is used by MinimalSchemaTest so
+     Anything added to this file needs to work correctly even if there
+     is now uniqueKey or defaultSearch Field.
+  -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+
+  </updateHandler>
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+</config>
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
index 5fd3876..83ab801 100644
--- a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
@@ -28,12 +28,13 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
 import org.apache.solr.client.solrj.request.json.JsonQueryRequest;
 import org.apache.solr.client.solrj.request.json.TermsFacetMap;
+import org.apache.solr.client.solrj.response.json.BucketJsonFacet;
+import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.ExternalPaths;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -136,7 +137,8 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
     assertEquals(0, queryResponse.getStatus());
     assertEquals(32, queryResponse.getResults().getNumFound());
     assertEquals(10, queryResponse.getResults().size());
-    assertHasFacetWithBucketValues(queryResponse.getResponse(),"categories",
+    final NestableJsonFacet topLevelFacetingData = queryResponse.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetingData,"categories",
         new FacetBucket("electronics",12),
         new FacetBucket("currency", 4),
         new FacetBucket("memory", 3));
@@ -157,7 +159,8 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
     assertEquals(0, queryResponse.getStatus());
     assertEquals(32, queryResponse.getResults().getNumFound());
     assertEquals(10, queryResponse.getResults().size());
-    assertHasFacetWithBucketValues(queryResponse.getResponse(),"categories",
+    final NestableJsonFacet topLevelFacetingData = queryResponse.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetingData,"categories",
         new FacetBucket("electronics",12),
         new FacetBucket("currency", 4),
         new FacetBucket("memory", 3),
@@ -177,38 +180,16 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
     public int getCount() { return count; }
   }
 
-  private void assertHasFacetWithBucketValues(NamedList<Object> rawResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
-    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
-    assertFacetResponseHasFacetWithBuckets(facetsTopLevel, expectedFacetName, expectedBuckets);
-  }
-
-  private NamedList<Object> assertHasFacetResponse(NamedList<Object> topLevelResponse) {
-    Object o = topLevelResponse.get("facets");
-    if (o == null) fail("Response has no top-level 'facets' property as expected");
-    if (!(o instanceof NamedList)) fail("Response has a top-level 'facets' property, but it is not a NamedList");
-
-    return (NamedList<Object>) o;
-  }
-
-  private void assertFacetResponseHasFacetWithBuckets(NamedList<Object> facetResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
-    Object o = facetResponse.get(expectedFacetName);
-    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
-    if (!(o instanceof NamedList)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a NamedList");
-
-    final NamedList<Object> expectedFacetTopLevel = (NamedList<Object>) o;
-    o = expectedFacetTopLevel.get("buckets");
-    if (o == null) fail("Response has no 'buckets' property under 'facets'");
-    if (!(o instanceof List)) fail("Response has no 'buckets' property containing actual facet information.");
-
-    final List<NamedList> bucketList = (List<NamedList>) o;
-    assertEquals("Expected " + expectedBuckets.length + " buckets, but found " + bucketList.size(),
-        expectedBuckets.length, bucketList.size());
+  private void assertHasFacetWithBucketValues(NestableJsonFacet response, String expectedFacetName, FacetBucket... expectedBuckets) {
+    assertTrue("Expected response to have facet with name " + expectedFacetName,
+        response.getBucketBasedFacets(expectedFacetName) != null);
+    final List<BucketJsonFacet> buckets = response.getBucketBasedFacets(expectedFacetName).getBuckets();
+    assertEquals(expectedBuckets.length, buckets.size());
     for (int i = 0; i < expectedBuckets.length; i++) {
       final FacetBucket expectedBucket = expectedBuckets[i];
-      final NamedList<Object> actualBucket = bucketList.get(i);
-      assertEquals(expectedBucket.getVal(), actualBucket.get("val"));
-      assertEquals(expectedBucket.getCount(), actualBucket.get("count"));
+      final BucketJsonFacet actualBucket = buckets.get(i);
+      assertEquals(expectedBucket.getVal(), actualBucket.getVal());
+      assertEquals(expectedBucket.getCount(), actualBucket.getCount());
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d17c6d88/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java
index be4f0cc..d515368 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java
@@ -24,11 +24,12 @@ import java.util.List;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
+import org.apache.solr.client.solrj.response.json.BucketJsonFacet;
+import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.ExternalPaths;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -46,8 +47,6 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
   private static final int NUM_BELKIN = 2;
   private static final int NUM_CANON = 2;
 
-
-
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(1)
@@ -79,15 +78,16 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData, "top_cats",
+        new FacetBucket("electronics", NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
   }
 
   @Test
@@ -108,18 +108,20 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
-    assertHasFacetWithBucketValues(rawResponse,"top_manufacturers", new FacetBucket("corsair",NUM_CORSAIR),
-        new FacetBucket("belkin", NUM_BELKIN), new FacetBucket("canon", NUM_CANON));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_manufacturers",
+        new FacetBucket("corsair",NUM_CORSAIR),
+        new FacetBucket("belkin", NUM_BELKIN),
+        new FacetBucket("canon", NUM_CANON));
   }
 
   @Test
@@ -137,14 +139,13 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"prices",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData,"prices",
         new FacetBucket(0.0f, 5),
         new FacetBucket(20.0f, 0),
         new FacetBucket(40.0f, 0),
@@ -174,20 +175,19 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasFacetWithBucketValues(rawResponse,"prices",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, topLevelFacetData.getCount());
+    assertHasFacetWithBucketValues(topLevelFacetData,"prices",
         new FacetBucket(0.0f, 5),
         new FacetBucket(20.0f, 0),
         new FacetBucket(40.0f, 0),
         new FacetBucket(60.0f, 1),
         new FacetBucket(80.0f, 1));
-    assertHasFacetWithBucketValues(rawResponse, "shipping_weights",
+    assertHasFacetWithBucketValues(topLevelFacetData, "shipping_weights",
         new FacetBucket(0.0f, 6),
         new FacetBucket(50.0f, 0),
         new FacetBucket(100.0f, 0),
@@ -203,14 +203,12 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasStatFacetWithValue(rawResponse,"sum_price", 5251.270030975342);
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasStatFacetWithValue(topLevelFacetData,"sum_price", 5251.270030975342);
   }
 
   @Test
@@ -223,15 +221,13 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasStatFacetWithValue(rawResponse,"sum_price", 5251.270030975342);
-    assertHasStatFacetWithValue(rawResponse,"avg_price", 328.20437693595886);
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasStatFacetWithValue(topLevelFacetData,"sum_price", 5251.270030975342);
+    assertHasStatFacetWithValue(topLevelFacetData,"avg_price", 328.20437693595886);
   }
 
   @Test
@@ -248,20 +244,21 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-    assertHasStatFacetWithValue(rawResponse,"avg_price", 328.20437693595886);
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasStatFacetWithValue(topLevelFacetData,"avg_price", 328.20437693595886);
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
   }
 
   @Test
   public void testNestedTermsFacet() throws Exception {
+    final String subfacetName = "top_manufacturers_for_cat";
     final String jsonBody = String.join("\n","{",
         "  'query': '*:*',",
         "  'facet': {",
@@ -280,26 +277,21 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
-    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
     // Test top level facets
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
-
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
     // Test subfacet values for each top-level facet bucket
-    final List<NamedList<Object>> topLevelFacetResponse = (List<NamedList<Object>>) rawResponse.findRecursive("facets", "top_cats", "buckets");
-    final NamedList<Object> electronicsSubFacet = topLevelFacetResponse.get(0);
-    assertFacetResponseHasFacetWithBuckets(electronicsSubFacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
-    final NamedList<Object> currencySubfacet = topLevelFacetResponse.get(1);
-    assertFacetResponseHasFacetWithBuckets(currencySubfacet, "top_manufacturers_for_cat", new FacetBucket("boa", 1));
-    final NamedList<Object> memorySubfacet = topLevelFacetResponse.get(2);
-    assertFacetResponseHasFacetWithBuckets(memorySubfacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
+    final List<BucketJsonFacet> catBuckets = topLevelFacetData.getBucketBasedFacets("top_cats").getBuckets();
+    assertHasFacetWithBucketValues(catBuckets.get(0), subfacetName, new FacetBucket("corsair", 3));
+    assertHasFacetWithBucketValues(catBuckets.get(1), subfacetName, new FacetBucket("boa", 1));
+    assertHasFacetWithBucketValues(catBuckets.get(2), subfacetName, new FacetBucket("corsair", 3));
   }
 
   @Test
@@ -319,26 +311,21 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
-    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
     // Test top level facets
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
-
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
     // Test subfacet values for each top-level facet bucket
-    final List<NamedList<Object>> topLevelFacetResponse = (List<NamedList<Object>>) rawResponse.findRecursive("facets", "top_cats", "buckets");
-    final NamedList<Object> electronicsSubFacet = topLevelFacetResponse.get(0);
-    assertFacetResponseHasStatFacetWithValue(electronicsSubFacet, subfacetName, 252.02909261530095);
-    final NamedList<Object> currencySubfacet = topLevelFacetResponse.get(1);
-    assertFacetResponseHasStatFacetWithValue(currencySubfacet, subfacetName, 0.0);
-    final NamedList<Object> memorySubfacet = topLevelFacetResponse.get(2);
-    assertFacetResponseHasStatFacetWithValue(memorySubfacet, subfacetName, 129.99499893188477);
+    final List<BucketJsonFacet> catBuckets = topLevelFacetData.getBucketBasedFacets("top_cats").getBuckets();
+    assertHasStatFacetWithValue(catBuckets.get(0), subfacetName, 252.02909261530095); // electronics
+    assertHasStatFacetWithValue(catBuckets.get(1), subfacetName, 0.0); // currency
+    assertHasStatFacetWithValue(catBuckets.get(2), subfacetName, 129.99499893188477); // memory
   }
 
   @Test
@@ -357,17 +344,15 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
-    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    // Test top level facets
-    assertHasFacetWithBucketValues(rawResponse,"top_popular_cats", new FacetBucket("electronics",9),
-        new FacetBucket("graphics card", 2), new FacetBucket("hard drive", 2));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_popular_cats",
+        new FacetBucket("electronics",9),
+        new FacetBucket("graphics card", 2),
+        new FacetBucket("hard drive", 2));
   }
 
   @Test
@@ -388,15 +373,12 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
     QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
-
-    // Test top level facets
-    assertHasFacetWithBucketValues(rawResponse,"top_popular_cats", new FacetBucket("electronics",9),
-        new FacetBucket("graphics card", 2), new FacetBucket("hard drive", 2));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_TECHPRODUCTS_DOCS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_popular_cats",
+        new FacetBucket("electronics",9),
+        new FacetBucket("graphics card", 2),
+        new FacetBucket("hard drive", 2));
   }
 
   @Test
@@ -415,17 +397,15 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
-    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    // Test top level facets
-    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
-        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    assertExpectedDocumentsFoundAndReturned(response, NUM_ELECTRONICS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
   }
 
   @Test
@@ -443,22 +423,17 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
-    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertHasFacetWithBucketValues(rawResponse,"largest_search_cats",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_ELECTRONICS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"largest_search_cats",
         new FacetBucket("search",2),
         new FacetBucket("software", 2));
   }
 
-  /*
-   * Multiple query clauses are effectively AND'd together
-   */
+  @Test
   public void testFacetWithMultipleSimpleQueryClausesInArbitraryDomain() throws Exception {
     final String jsonBody = String.join("\n","{",
         "  'query': 'cat:electronics',",
@@ -473,19 +448,17 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
-    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertHasFacetWithBucketValues(rawResponse,"cats_matching_solr",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_ELECTRONICS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"cats_matching_solr",
         new FacetBucket("search",1),
         new FacetBucket("software", 1));
   }
 
+  @Test
   public void testFacetWithMultipleLocalParamsQueryClausesInArbitraryDomain() throws Exception {
     final String jsonBody = String.join("\n","{",
         "  'query': 'cat:electronics',",
@@ -500,16 +473,12 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
-    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    // Test top level facets
-    assertHasFacetWithBucketValues(rawResponse,"cats_matching_solr",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_ELECTRONICS, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"cats_matching_solr",
         new FacetBucket("search",1),
         new FacetBucket("software", 1));
   }
@@ -536,18 +505,15 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
         "  }",
         "}");
     final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
-    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertEquals(0, response.getStatus());
-    final SolrDocumentList returnedDocs = response.getResults();
-    assertEquals(NUM_IN_STOCK, returnedDocs.getNumFound());
-    assertEquals(10, returnedDocs.size());
-    final NamedList<Object> rawResponse = response.getResponse();
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
 
-    assertHasFacetWithBucketValues(rawResponse,"in_stock_only",
+    assertExpectedDocumentsFoundAndReturned(response, NUM_IN_STOCK, 10);
+    final NestableJsonFacet topLevelFacetData = response.getJsonFacetingResponse();
+    assertHasFacetWithBucketValues(topLevelFacetData,"in_stock_only",
         new FacetBucket("electronics",8),
         new FacetBucket("currency", 4));
-    assertHasFacetWithBucketValues(rawResponse,"all",
+    assertHasFacetWithBucketValues(topLevelFacetData,"all",
         new FacetBucket("electronics",12),
         new FacetBucket("currency", 4));
   }
@@ -564,52 +530,29 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
     public int getCount() { return count; }
   }
 
-  private void assertHasFacetWithBucketValues(NamedList<Object> rawResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
-    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
-    assertFacetResponseHasFacetWithBuckets(facetsTopLevel, expectedFacetName, expectedBuckets);
-  }
-
-  private void assertHasStatFacetWithValue(NamedList<Object> rawResponse, String expectedFacetName, Double expectedStatValue) {
-    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
-    assertFacetResponseHasStatFacetWithValue(facetsTopLevel, expectedFacetName, expectedStatValue);
-  }
-
-  private NamedList<Object> assertHasFacetResponse(NamedList<Object> topLevelResponse) {
-    Object o = topLevelResponse.get("facets");
-    if (o == null) fail("Response has no top-level 'facets' property as expected");
-    if (!(o instanceof NamedList)) fail("Response has a top-level 'facets' property, but it is not a NamedList");
-
-    return (NamedList<Object>) o;
-  }
-
-  private void assertFacetResponseHasFacetWithBuckets(NamedList<Object> facetResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
-    Object o = facetResponse.get(expectedFacetName);
-    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
-    if (!(o instanceof NamedList)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a NamedList");
-
-    final NamedList<Object> expectedFacetTopLevel = (NamedList<Object>) o;
-    o = expectedFacetTopLevel.get("buckets");
-    if (o == null) fail("Response has no 'buckets' property under 'facets'");
-    if (!(o instanceof List)) fail("Response has no 'buckets' property containing actual facet information.");
-
-    final List<NamedList> bucketList = (List<NamedList>) o;
-    assertEquals("Expected " + expectedBuckets.length + " buckets, but found " + bucketList.size(),
-        expectedBuckets.length, bucketList.size());
+  private void assertHasFacetWithBucketValues(NestableJsonFacet response, String expectedFacetName, FacetBucket... expectedBuckets) {
+    assertTrue("Expected response to have facet with name " + expectedFacetName,
+        response.getBucketBasedFacets(expectedFacetName) != null);
+    final List<BucketJsonFacet> buckets = response.getBucketBasedFacets(expectedFacetName).getBuckets();
+    assertEquals(expectedBuckets.length, buckets.size());
     for (int i = 0; i < expectedBuckets.length; i++) {
       final FacetBucket expectedBucket = expectedBuckets[i];
-      final NamedList<Object> actualBucket = bucketList.get(i);
-      assertEquals(expectedBucket.getVal(), actualBucket.get("val"));
-      assertEquals(expectedBucket.getCount(), actualBucket.get("count"));
+      final BucketJsonFacet actualBucket = buckets.get(i);
+      assertEquals(expectedBucket.getVal(), actualBucket.getVal());
+      assertEquals(expectedBucket.getCount(), actualBucket.getCount());
     }
   }
 
-  private void assertFacetResponseHasStatFacetWithValue(NamedList<Object> facetResponse, String expectedFacetName, Double expectedStatValue) {
-    Object o = facetResponse.get(expectedFacetName);
-    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
-    if (!(o instanceof Number)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a Number");
+  private void assertHasStatFacetWithValue(NestableJsonFacet response, String expectedFacetName, Double expectedStatValue) {
+    assertTrue("Expected response to have stat facet named '" + expectedFacetName + "'",
+        response.getStatFacetValue(expectedFacetName) != null);
+    assertEquals(expectedStatValue, response.getStatFacetValue(expectedFacetName));
+  }
 
-    final Number actualStatValueAsNumber = (Number) o;
-    final Double actualStatValueAsDouble = ((Number) o).doubleValue();
-    assertEquals(expectedStatValue, actualStatValueAsDouble, 0.5);
+  private void assertExpectedDocumentsFoundAndReturned(QueryResponse response, int expectedNumFound, int expectedReturned) {
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList documents = response.getResults();
+    assertEquals(expectedNumFound, documents.getNumFound());
+    assertEquals(expectedReturned, documents.size());
   }
 }