diff --git a/TODO b/TODO
index c767fe0ff47..5f146ac08c8 100644
--- a/TODO
+++ b/TODO
@@ -1,6 +1,7 @@
nocommit this!
TODO
+ - allow path.length==0?
- move DocumentBuilder.build -> FacetsConfig.build
- add sugar apis to do sort-by-score, sort-by-field sort AND collect into SimpleFacetsCollector?
- getSpecificValue for a dim isn't reliable
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java b/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java
index 8a0e92057e7..447ebfa4c2a 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java
@@ -45,134 +45,6 @@ public class FacetResult {
return n;
}
- /**
- * A utility for merging multiple {@link FacetResult} of the same
- * (hierarchical) dimension into a single {@link FacetResult}, to reconstruct
- * the hierarchy. The results are merged according to the following rules:
- *
- * If two results share the same dimension (first component in their
- * {@link FacetLabel}), they are merged.
- * If a result is missing ancestors in the other results, e.g. A/B/C but
- * no corresponding A or A/B, these nodes are 'filled' with their label,
- * ordinal and value (obtained from the respective {@link FacetArrays}).
- * If a result does not share a dimension with other results, it is
- * returned as is.
- *
- *
- * NOTE: the returned results are not guaranteed to be in the same
- * order of the input ones.
- *
- * @param results
- * the results to merge
- * @param taxoReader
- * the {@link TaxonomyReader} to use when creating missing ancestor
- * nodes
- * @param dimArrays
- * a mapping from a dimension to the respective {@link FacetArrays}
- * from which to pull the nodes values
- */
- public static List mergeHierarchies(List results, TaxonomyReader taxoReader,
- Map dimArrays) throws IOException {
- final Map> dims = new HashMap>();
- for (FacetResult fr : results) {
- String dim = fr.getFacetRequest().categoryPath.components[0];
- List frs = dims.get(dim);
- if (frs == null) {
- frs = new ArrayList();
- dims.put(dim, frs);
- }
- frs.add(fr);
- }
-
- final List res = new ArrayList();
- for (List frs : dims.values()) {
- FacetResult mergedResult = frs.get(0);
- if (frs.size() > 1) {
- CollectionUtil.introSort(frs, new Comparator() {
- @Override
- public int compare(FacetResult fr1, FacetResult fr2) {
- return fr1.getFacetRequest().categoryPath.compareTo(fr2.getFacetRequest().categoryPath);
- }
- });
- Map mergedNodes = new HashMap();
- FacetArrays arrays = dimArrays != null ? dimArrays.get(frs.get(0).getFacetRequest().categoryPath.components[0]) : null;
- for (FacetResult fr : frs) {
- FacetRequest freq = fr.getFacetRequest();
- OrdinalValueResolver resolver = null;
- if (arrays != null) {
- resolver = freq.createFacetsAggregator(FacetIndexingParams.DEFAULT).createOrdinalValueResolver(freq, arrays);
- }
- FacetResultNode frn = fr.getFacetResultNode();
- FacetResultNode merged = mergedNodes.get(frn.label);
- if (merged == null) {
- FacetLabel parent = frn.label.subpath(frn.label.length - 1);
- FacetResultNode childNode = frn;
- FacetResultNode parentNode = null;
- while (parent.length > 0 && (parentNode = mergedNodes.get(parent)) == null) {
- int parentOrd = taxoReader.getOrdinal(parent);
- double parentValue = -1;
- if (arrays != null) {
- parentValue = resolver.valueOf(parentOrd);
- }
- parentNode = new FacetResultNode(parentOrd, parentValue);
- parentNode.label = parent;
- parentNode.subResults = new ArrayList();
- parentNode.subResults.add(childNode);
- mergedNodes.put(parent, parentNode);
- childNode = parentNode;
- parent = parent.subpath(parent.length - 1);
- }
-
- // at least one parent was added, so link the final (existing)
- // parent with the child
- if (parent.length > 0) {
- if (!(parentNode.subResults instanceof ArrayList)) {
- parentNode.subResults = new ArrayList(parentNode.subResults);
- }
- parentNode.subResults.add(childNode);
- }
-
- // for missing FRNs, add new ones with label and value=-1
- // first time encountered this label, add it and all its children to
- // the map.
- mergedNodes.put(frn.label, frn);
- for (FacetResultNode child : frn.subResults) {
- addIfNotExist(mergedNodes, child);
- }
- } else {
- if (!(merged.subResults instanceof ArrayList)) {
- merged.subResults = new ArrayList(merged.subResults);
- }
- for (FacetResultNode sub : frn.subResults) {
- // make sure sub wasn't already added
- sub = addIfNotExist(mergedNodes, sub);
- if (!merged.subResults.contains(sub)) {
- merged.subResults.add(sub);
- }
- }
- }
- }
-
- // find the 'first' node to put on the FacetResult root
- FacetLabel min = null;
- for (FacetLabel cp : mergedNodes.keySet()) {
- if (min == null || cp.compareTo(min) < 0) {
- min = cp;
- }
- }
- FacetRequest dummy = new FacetRequest(min, frs.get(0).getFacetRequest().numResults) {
- @Override
- public FacetsAggregator createFacetsAggregator(FacetIndexingParams fip) {
- throw new UnsupportedOperationException("not supported by this request");
- }
- };
- mergedResult = new FacetResult(dummy, mergedNodes.get(min), -1);
- }
- res.add(mergedResult);
- }
- return res;
- }
-
private final FacetRequest facetRequest;
private final FacetResultNode rootNode;
private final int numValidDescendants;
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/simple/CachedOrdinalsReader.java b/lucene/facet/src/java/org/apache/lucene/facet/simple/CachedOrdinalsReader.java
index aae66a9b63f..7c8db302d0d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/simple/CachedOrdinalsReader.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/simple/CachedOrdinalsReader.java
@@ -25,6 +25,7 @@ import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
/**
* A per-segment cache of documents' facet ordinals. Every
@@ -45,15 +46,16 @@ import org.apache.lucene.util.IntsRef;
* a {@link DocValuesFormat} that does not cache the data in
* memory, at least for the category lists fields, or
* otherwise you'll be doing double-caching.
+ *
+ *
+ * NOTE: create one instance of this and re-use it
+ * for all facet implementations (the cache is per-instance,
+ * not static).
*/
public class CachedOrdinalsReader extends OrdinalsReader {
private final OrdinalsReader source;
- private CachedOrds current;
- // outer map is a WeakHashMap which uses reader.getCoreCacheKey() as the weak
- // reference. When it's no longer referenced, the entire inner map can be
- // evicted.
private final Map ordsCache = new WeakHashMap();
public CachedOrdinalsReader(OrdinalsReader source) {
@@ -132,4 +134,14 @@ public class CachedOrdinalsReader extends OrdinalsReader {
}
}
}
+
+ /** How many bytes is this cache using? */
+ public synchronized long ramBytesUsed() {
+ long bytes = 0;
+ for(CachedOrds ords : ordsCache.values()) {
+ bytes += RamUsageEstimator.sizeOf(ords);
+ }
+
+ return bytes;
+ }
}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/simple/FacetsConfig.java b/lucene/facet/src/java/org/apache/lucene/facet/simple/FacetsConfig.java
index 2d82ac9492b..5f72a902ac7 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/simple/FacetsConfig.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/simple/FacetsConfig.java
@@ -317,7 +317,7 @@ public class FacetsConfig {
}
// Drill down:
- for(int i=2;i<=cp.length;i++) {
+ for(int i=1;i<=cp.length;i++) {
addedIndexedFields.add(new StringField(indexFieldName, pathToString(cp.components, i), Field.Store.NO));
}
}
@@ -345,6 +345,7 @@ public class FacetsConfig {
// For drill-down:
addedIndexedFields.add(new StringField(indexFieldName, fullPath, Field.Store.NO));
+ addedIndexedFields.add(new StringField(indexFieldName, facetField.dim, Field.Store.NO));
}
}
}
@@ -455,6 +456,14 @@ public class FacetsConfig {
}
public static String pathToString(String[] path, int length) {
+ // nocommit .... too anal? shouldn't we allow drill
+ // down on just dim, to get all docs that have that
+ // dim...?
+ /*
+ if (path.length < 2) {
+ throw new IllegalArgumentException("path length must be > 0 (dim=" + path[0] + ")");
+ }
+ */
if (length == 0) {
return "";
}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java
index 75ac545dfef..6b4e64b5ca6 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java
@@ -52,7 +52,7 @@ import org.apache.lucene.search.TermQuery;
*/
public final class SimpleDrillDownQuery extends Query {
- private static Term term(String field, String dim, String[] path) {
+ public static Term term(String field, String dim, String... path) {
return new Term(field, FacetsConfig.pathToString(dim, path));
}
@@ -153,9 +153,12 @@ public final class SimpleDrillDownQuery extends Query {
String indexedField = config.getDimConfig(dim).indexFieldName;
BooleanQuery bq = new BooleanQuery(true); // disable coord
+ // nocommit too anal?
+ /*
if (path.length == 0) {
throw new IllegalArgumentException("must have at least one facet label under dim");
}
+ */
bq.add(new TermQuery(term(indexedField, dim, path)), Occur.SHOULD);
add(dim, bq);
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java b/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java
deleted file mode 100644
index 7d59a76dfed..00000000000
--- a/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java
+++ /dev/null
@@ -1,68 +0,0 @@
-package org.apache.lucene.facet.util;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.lucene.facet.search.CategoryListIterator;
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.util.IntsRef;
-
-/*
- * 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.
- */
-
-/**
- * Iterates over multiple {@link CategoryListIterator}s, consuming the provided
- * iterators in order.
- *
- * @lucene.experimental
- */
-public class MultiCategoryListIterator implements CategoryListIterator {
-
- private final CategoryListIterator[] iterators;
- private final List validIterators;
-
- /** Receives the iterators to iterate on */
- public MultiCategoryListIterator(CategoryListIterator... iterators) {
- this.iterators = iterators;
- this.validIterators = new ArrayList();
- }
-
- @Override
- public boolean setNextReader(AtomicReaderContext context) throws IOException {
- validIterators.clear();
- for (CategoryListIterator cli : iterators) {
- if (cli.setNextReader(context)) {
- validIterators.add(cli);
- }
- }
- return !validIterators.isEmpty();
- }
-
- @Override
- public void getOrdinals(int docID, IntsRef ints) throws IOException {
- IntsRef tmp = new IntsRef(ints.length);
- for (CategoryListIterator cli : validIterators) {
- cli.getOrdinals(docID, tmp);
- if (ints.ints.length < ints.length + tmp.length) {
- ints.grow(ints.length + tmp.length);
- }
- ints.length += tmp.length;
- }
- }
-
-}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java b/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java
deleted file mode 100644
index 79cbc0f45c0..00000000000
--- a/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java
+++ /dev/null
@@ -1,178 +0,0 @@
-package org.apache.lucene.facet.associations;
-
-import java.util.List;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.FacetTestCase;
-import org.apache.lucene.facet.params.FacetSearchParams;
-import org.apache.lucene.facet.search.FacetResult;
-import org.apache.lucene.facet.search.FacetsCollector;
-import org.apache.lucene.facet.taxonomy.FacetLabel;
-import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.store.Directory;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/*
- * 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.
- */
-
-/** Test for associations */
-public class AssociationsFacetRequestTest extends FacetTestCase {
-
- private static Directory dir;
- private static IndexReader reader;
- private static Directory taxoDir;
-
- private static final FacetLabel aint = new FacetLabel("int", "a");
- private static final FacetLabel bint = new FacetLabel("int", "b");
- private static final FacetLabel afloat = new FacetLabel("float", "a");
- private static final FacetLabel bfloat = new FacetLabel("float", "b");
-
- @BeforeClass
- public static void beforeClassAssociationsFacetRequestTest() throws Exception {
- dir = newDirectory();
- taxoDir = newDirectory();
- // preparations - index, taxonomy, content
- RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig(TEST_VERSION_CURRENT,
- new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)));
-
- TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
-
- AssociationsFacetFields assocFacetFields = new AssociationsFacetFields(taxoWriter);
-
- // index documents, 50% have only 'b' and all have 'a'
- for (int i = 0; i < 110; i++) {
- Document doc = new Document();
- CategoryAssociationsContainer associations = new CategoryAssociationsContainer();
- // every 11th document is added empty, this used to cause the association
- // aggregators to go into an infinite loop
- if (i % 11 != 0) {
- associations.setAssociation(aint, new CategoryIntAssociation(2));
- associations.setAssociation(afloat, new CategoryFloatAssociation(0.5f));
- if (i % 2 == 0) { // 50
- associations.setAssociation(bint, new CategoryIntAssociation(3));
- associations.setAssociation(bfloat, new CategoryFloatAssociation(0.2f));
- }
- }
- assocFacetFields.addFields(doc, associations);
- writer.addDocument(doc);
- }
-
- taxoWriter.close();
- reader = writer.getReader();
- writer.close();
- }
-
- @AfterClass
- public static void afterClassAssociationsFacetRequestTest() throws Exception {
- reader.close();
- reader = null;
- dir.close();
- dir = null;
- taxoDir.close();
- taxoDir = null;
- }
-
- @Test
- public void testIntSumAssociation() throws Exception {
- DirectoryTaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
-
- // facet requests for two facets
- FacetSearchParams fsp = new FacetSearchParams(
- new SumIntAssociationFacetRequest(aint, 10),
- new SumIntAssociationFacetRequest(bint, 10));
-
- Query q = new MatchAllDocsQuery();
-
- FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
-
- IndexSearcher searcher = newSearcher(reader);
- searcher.search(q, fc);
- List res = fc.getFacetResults();
-
- assertNotNull("No results!",res);
- assertEquals("Wrong number of results!",2, res.size());
- assertEquals("Wrong count for category 'a'!", 200, (int) res.get(0).getFacetResultNode().value);
- assertEquals("Wrong count for category 'b'!", 150, (int) res.get(1).getFacetResultNode().value);
-
- taxo.close();
- }
-
- @Test
- public void testFloatSumAssociation() throws Exception {
- DirectoryTaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
-
- // facet requests for two facets
- FacetSearchParams fsp = new FacetSearchParams(
- new SumFloatAssociationFacetRequest(afloat, 10),
- new SumFloatAssociationFacetRequest(bfloat, 10));
-
- Query q = new MatchAllDocsQuery();
-
- FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
-
- IndexSearcher searcher = newSearcher(reader);
- searcher.search(q, fc);
- List res = fc.getFacetResults();
-
- assertNotNull("No results!",res);
- assertEquals("Wrong number of results!", 2, res.size());
- assertEquals("Wrong count for category 'a'!",50f, (float) res.get(0).getFacetResultNode().value, 0.00001);
- assertEquals("Wrong count for category 'b'!",10f, (float) res.get(1).getFacetResultNode().value, 0.00001);
-
- taxo.close();
- }
-
- @Test
- public void testDifferentAggregatorsSameCategoryList() throws Exception {
- DirectoryTaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
-
- // facet requests for two facets
- FacetSearchParams fsp = new FacetSearchParams(
- new SumIntAssociationFacetRequest(aint, 10),
- new SumIntAssociationFacetRequest(bint, 10),
- new SumFloatAssociationFacetRequest(afloat, 10),
- new SumFloatAssociationFacetRequest(bfloat, 10));
-
- Query q = new MatchAllDocsQuery();
-
- FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
-
- IndexSearcher searcher = newSearcher(reader);
- searcher.search(q, fc);
- List res = fc.getFacetResults();
-
- assertEquals("Wrong number of results!", 4, res.size());
- assertEquals("Wrong count for category 'a'!", 200, (int) res.get(0).getFacetResultNode().value);
- assertEquals("Wrong count for category 'b'!", 150, (int) res.get(1).getFacetResultNode().value);
- assertEquals("Wrong count for category 'a'!",50f, (float) res.get(2).getFacetResultNode().value, 0.00001);
- assertEquals("Wrong count for category 'b'!",10f, (float) res.get(3).getFacetResultNode().value, 0.00001);
-
- taxo.close();
- }
-
-}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/params/FacetIndexingParamsTest.java b/lucene/facet/src/test/org/apache/lucene/facet/params/FacetIndexingParamsTest.java
deleted file mode 100644
index d08f1b9de5b..00000000000
--- a/lucene/facet/src/test/org/apache/lucene/facet/params/FacetIndexingParamsTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-package org.apache.lucene.facet.params;
-
-import org.apache.lucene.facet.FacetTestCase;
-import org.apache.lucene.facet.params.CategoryListParams;
-import org.apache.lucene.facet.params.FacetIndexingParams;
-import org.apache.lucene.facet.search.DrillDownQuery;
-import org.apache.lucene.facet.taxonomy.FacetLabel;
-import org.apache.lucene.facet.util.PartitionsUtils;
-import org.apache.lucene.index.Term;
-import org.junit.Test;
-
-/*
- * 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.
- */
-
-public class FacetIndexingParamsTest extends FacetTestCase {
-
- @Test
- public void testDefaultSettings() {
- FacetIndexingParams dfip = FacetIndexingParams.DEFAULT;
- assertNotNull("Missing default category list", dfip.getAllCategoryListParams());
- assertEquals("all categories have the same CategoryListParams by default",
- dfip.getCategoryListParams(null), dfip.getCategoryListParams(new FacetLabel("a")));
- assertEquals("Expected default category list field is $facets", "$facets", dfip.getCategoryListParams(null).field);
- String expectedDDText = "a"
- + dfip.getFacetDelimChar() + "b";
- FacetLabel cp = new FacetLabel("a", "b");
- assertEquals("wrong drill-down term", new Term("$facets",
- expectedDDText), DrillDownQuery.term(dfip,cp));
- char[] buf = new char[20];
- int numchars = dfip.drillDownTermText(cp, buf);
- assertEquals("3 characters should be written", 3, numchars);
- assertEquals("wrong drill-down term text", expectedDDText, new String(
- buf, 0, numchars));
- assertEquals("partition for all ordinals is the first", "",
- PartitionsUtils.partitionNameByOrdinal(dfip, 250));
- assertEquals("for partition 0, the same name should be returned",
- "", PartitionsUtils.partitionName(0));
- assertEquals(
- "for any other, it's the concatenation of name + partition",
- PartitionsUtils.PART_NAME_PREFIX + "1", PartitionsUtils.partitionName(1));
- assertEquals("default partition number is always 0", 0,
- PartitionsUtils.partitionNumber(dfip,100));
- assertEquals("default partition size is unbounded", Integer.MAX_VALUE,
- dfip.getPartitionSize());
- }
-
- @Test
- public void testCategoryListParamsWithDefaultIndexingParams() {
- CategoryListParams clp = new CategoryListParams("clp");
- FacetIndexingParams dfip = new FacetIndexingParams(clp);
- assertEquals("Expected default category list field is " + clp.field, clp.field, dfip.getCategoryListParams(null).field);
- }
-
-}
\ No newline at end of file
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/params/FacetSearchParamsTest.java b/lucene/facet/src/test/org/apache/lucene/facet/params/FacetSearchParamsTest.java
deleted file mode 100644
index d0fba739d35..00000000000
--- a/lucene/facet/src/test/org/apache/lucene/facet/params/FacetSearchParamsTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-package org.apache.lucene.facet.params;
-
-import org.apache.lucene.facet.FacetTestCase;
-import org.apache.lucene.facet.params.FacetSearchParams;
-import org.junit.Test;
-
-/*
- * 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.
- */
-
-public class FacetSearchParamsTest extends FacetTestCase {
-
- @Test
- public void testSearchParamsWithNullRequest() throws Exception {
- try {
- assertNull(new FacetSearchParams());
- fail("FacetSearchParams should throw IllegalArgumentException when not adding requests");
- } catch (IllegalArgumentException e) {
- }
- }
-}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java b/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java
deleted file mode 100644
index e4fbc3eeb38..00000000000
--- a/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java
+++ /dev/null
@@ -1,144 +0,0 @@
-package org.apache.lucene.facet.search;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.document.BinaryDocValuesField;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.FacetTestCase;
-import org.apache.lucene.facet.encoding.DGapIntEncoder;
-import org.apache.lucene.facet.encoding.IntEncoder;
-import org.apache.lucene.facet.encoding.SortingIntEncoder;
-import org.apache.lucene.facet.encoding.UniqueValuesIntEncoder;
-import org.apache.lucene.facet.encoding.VInt8IntEncoder;
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
-import org.junit.Test;
-
-/*
- * 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.
- */
-
-public class CategoryListIteratorTest extends FacetTestCase {
-
- static final IntsRef[] data = new IntsRef[] {
- new IntsRef(new int[] { 1, 2 }, 0, 2),
- new IntsRef(new int[] { 3, 4 }, 0, 2),
- new IntsRef(new int[] { 1, 3 }, 0, 2),
- new IntsRef(new int[] { 1, 2, 3, 4 }, 0, 4)
- };
-
- @Test
- public void test() throws Exception {
- Directory dir = newDirectory();
- final IntEncoder encoder = randomCategoryListParams().createEncoder();
- RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig(TEST_VERSION_CURRENT,
- new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy()));
- BytesRef buf = new BytesRef();
- for (int i = 0; i < data.length; i++) {
- Document doc = new Document();
- encoder.encode(IntsRef.deepCopyOf(data[i]), buf);
- doc.add(new BinaryDocValuesField("f", buf));
- writer.addDocument(doc);
- }
- IndexReader reader = writer.getReader();
- writer.close();
-
- int totalCategories = 0;
- IntsRef ordinals = new IntsRef();
- CategoryListIterator cli = new DocValuesCategoryListIterator("f", encoder.createMatchingDecoder());
- for (AtomicReaderContext context : reader.leaves()) {
- assertTrue("failed to initalize iterator", cli.setNextReader(context));
- int maxDoc = context.reader().maxDoc();
- int dataIdx = context.docBase;
- for (int doc = 0; doc < maxDoc; doc++, dataIdx++) {
- Set values = new HashSet();
- for (int j = 0; j < data[dataIdx].length; j++) {
- values.add(data[dataIdx].ints[j]);
- }
- cli.getOrdinals(doc, ordinals);
- assertTrue("no ordinals for document " + doc, ordinals.length > 0);
- for (int j = 0; j < ordinals.length; j++) {
- assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
- }
- totalCategories += ordinals.length;
- }
- }
- assertEquals("Missing categories!", 10, totalCategories);
- reader.close();
- dir.close();
- }
-
- @Test
- public void testEmptyDocuments() throws Exception {
- Directory dir = newDirectory();
- final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
- // NOTE: test is wired to LogMP... because test relies on certain docids having payloads
- RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
- newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
- for (int i = 0; i < data.length; i++) {
- Document doc = new Document();
- if (i == 0) {
- BytesRef buf = new BytesRef();
- encoder.encode(IntsRef.deepCopyOf(data[i]), buf );
- doc.add(new BinaryDocValuesField("f", buf));
- } else {
- doc.add(new BinaryDocValuesField("f", new BytesRef()));
- }
- writer.addDocument(doc);
- writer.commit();
- }
-
- IndexReader reader = writer.getReader();
- writer.close();
-
- int totalCategories = 0;
- IntsRef ordinals = new IntsRef();
- CategoryListIterator cli = new DocValuesCategoryListIterator("f", encoder.createMatchingDecoder());
- for (AtomicReaderContext context : reader.leaves()) {
- assertTrue("failed to initalize iterator", cli.setNextReader(context));
- int maxDoc = context.reader().maxDoc();
- int dataIdx = context.docBase;
- for (int doc = 0; doc < maxDoc; doc++, dataIdx++) {
- Set values = new HashSet();
- for (int j = 0; j < data[dataIdx].length; j++) {
- values.add(data[dataIdx].ints[j]);
- }
- cli.getOrdinals(doc, ordinals);
- if (dataIdx == 0) {
- assertTrue("document 0 must have ordinals", ordinals.length > 0);
- for (int j = 0; j < ordinals.length; j++) {
- assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
- }
- totalCategories += ordinals.length;
- } else {
- assertTrue("only document 0 should have ordinals", ordinals.length == 0);
- }
- }
- }
- assertEquals("Wrong number of total categories!", 2, totalCategories);
-
- reader.close();
- dir.close();
- }
-
-}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java b/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java
deleted file mode 100644
index ad1cf74c3a7..00000000000
--- a/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java
+++ /dev/null
@@ -1,204 +0,0 @@
-package org.apache.lucene.facet.search;
-
-/*
- * 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.
- */
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.FacetTestCase;
-import org.apache.lucene.facet.FacetTestUtils;
-import org.apache.lucene.facet.index.FacetFields;
-import org.apache.lucene.facet.params.FacetIndexingParams;
-import org.apache.lucene.facet.params.FacetSearchParams;
-import org.apache.lucene.facet.search.DrillSideways.DrillSidewaysResult;
-import org.apache.lucene.facet.taxonomy.FacetLabel;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.RAMDirectory;
-import org.apache.lucene.util.CollectionUtil;
-import org.apache.lucene.util.IOUtils;
-import org.junit.Test;
-
-public class FacetResultTest extends FacetTestCase {
-
- private Document newDocument(FacetFields facetFields, String... categories) throws IOException {
- Document doc = new Document();
- List cats = new ArrayList();
- for (String cat : categories) {
- cats.add(new FacetLabel(cat, '/'));
- }
- facetFields.addFields(doc, cats);
- return doc;
- }
-
- private void initIndex(Directory indexDir, Directory taxoDir) throws IOException {
- IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
- IndexWriter indexWriter = new IndexWriter(indexDir, conf);
- DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
- FacetFields facetFields = new FacetFields(taxoWriter);
- indexWriter.addDocument(newDocument(facetFields, "Date/2010/March/12", "A/1"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2010/March/23", "A/2"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2010/April/17", "A/3"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2010/May/18", "A/1"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2011/January/1", "A/3"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2011/February/12", "A/1"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2011/February/18", "A/4"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2012/August/15", "A/1"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2012/July/5", "A/2"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2013/September/13", "A/1"));
- indexWriter.addDocument(newDocument(facetFields, "Date/2013/September/25", "A/4"));
- IOUtils.close(indexWriter, taxoWriter);
- }
-
- private void searchIndex(TaxonomyReader taxoReader, IndexSearcher searcher, boolean fillMissingCounts, String[] exp,
- String[][] drillDowns, int[] numResults) throws IOException {
- FacetLabel[][] cps = new FacetLabel[drillDowns.length][];
- for (int i = 0; i < cps.length; i++) {
- cps[i] = new FacetLabel[drillDowns[i].length];
- for (int j = 0; j < cps[i].length; j++) {
- cps[i][j] = new FacetLabel(drillDowns[i][j], '/');
- }
- }
- DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT, new MatchAllDocsQuery());
- for (FacetLabel[] cats : cps) {
- ddq.add(cats);
- }
-
- List facetRequests = new ArrayList();
- for (FacetLabel[] cats : cps) {
- for (int i = 0; i < cats.length; i++) {
- FacetLabel cp = cats[i];
- int numres = numResults == null ? 2 : numResults[i];
- // for each drill-down, add itself as well as its parent as requests, so
- // we get the drill-sideways
- facetRequests.add(new CountFacetRequest(cp, numres));
- CountFacetRequest parent = new CountFacetRequest(cp.subpath(cp.length - 1), numres);
- if (!facetRequests.contains(parent) && parent.categoryPath.length > 0) {
- facetRequests.add(parent);
- }
- }
- }
-
- FacetSearchParams fsp = new FacetSearchParams(facetRequests);
- final DrillSideways ds;
- final Map dimArrays;
- if (fillMissingCounts) {
- dimArrays = new HashMap();
- ds = new DrillSideways(searcher, taxoReader) {
- @Override
- protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) throws IOException {
- FacetsAccumulator fa = super.getDrillSidewaysAccumulator(dim, fsp);
- dimArrays.put(dim, ((TaxonomyFacetsAccumulator) fa).facetArrays);
- return fa;
- }
- };
- } else {
- ds = new DrillSideways(searcher, taxoReader);
- dimArrays = null;
- }
-
- final DrillSidewaysResult sidewaysRes = ds.search(null, ddq, 5, fsp);
- List facetResults = FacetResult.mergeHierarchies(sidewaysRes.facetResults, taxoReader, dimArrays);
- CollectionUtil.introSort(facetResults, new Comparator() {
- @Override
- public int compare(FacetResult o1, FacetResult o2) {
- return o1.getFacetRequest().categoryPath.compareTo(o2.getFacetRequest().categoryPath);
- }
- });
- assertEquals(exp.length, facetResults.size()); // A + single one for date
- for (int i = 0; i < facetResults.size(); i++) {
- assertEquals(exp[i], FacetTestUtils.toSimpleString(facetResults.get(i)));
- }
- }
-
- @Test
- public void testMergeHierarchies() throws Exception {
- Directory indexDir = new RAMDirectory(), taxoDir = new RAMDirectory();
- initIndex(indexDir, taxoDir);
-
- DirectoryReader indexReader = DirectoryReader.open(indexDir);
- TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
- IndexSearcher searcher = new IndexSearcher(indexReader);
-
- String[] exp = new String[] { "Date (0)\n 2010 (4)\n 2011 (3)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date" } }, null);
-
- // two dimensions
- exp = new String[] { "A (0)\n 1 (5)\n 4 (2)\n", "Date (0)\n 2010 (4)\n 2011 (3)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date" }, new String[] { "A" } }, null);
-
- // both parent and child are OR'd
- exp = new String[] { "Date (-1)\n 2010 (4)\n March (2)\n 23 (1)\n 12 (1)\n May (1)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010/March", "Date/2010/March/23" }}, null);
-
- // both parent and child are OR'd (fill counts)
- exp = new String[] { "Date (0)\n 2010 (4)\n March (2)\n 23 (1)\n 12 (1)\n May (1)\n" };
- searchIndex(taxoReader, searcher, true, exp, new String[][] { new String[] { "Date/2010/March", "Date/2010/March/23" }}, null);
-
- // same DD twice
- exp = new String[] { "Date (0)\n 2010 (4)\n March (2)\n May (1)\n 2011 (3)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010", "Date/2010" }}, null);
-
- exp = new String[] { "Date (0)\n 2010 (4)\n March (2)\n May (1)\n 2011 (3)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010" }}, null);
-
- exp = new String[] { "Date (0)\n 2010 (4)\n March (2)\n May (1)\n 2011 (3)\n February (2)\n January (1)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010", "Date/2011" }}, null);
-
- exp = new String[] { "Date (0)\n 2010 (4)\n March (2)\n 23 (1)\n 12 (1)\n May (1)\n 2011 (3)\n February (2)\n January (1)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010/March", "Date/2011" }}, null);
-
- // Date/2010/April not in top-2 of Date/2010
- exp = new String[] { "Date (0)\n 2010 (4)\n March (2)\n 23 (1)\n 12 (1)\n May (1)\n April (1)\n 17 (1)\n 2011 (3)\n February (2)\n January (1)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010/March", "Date/2010/April", "Date/2011" }}, null);
-
- // missing ancestors
- exp = new String[] { "Date (-1)\n 2010 (4)\n March (2)\n May (1)\n April (1)\n 17 (1)\n 2011 (-1)\n January (1)\n 1 (1)\n" };
- searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2011/January/1", "Date/2010/April" }}, null);
-
- // missing ancestors (fill counts)
- exp = new String[] { "Date (0)\n 2010 (4)\n March (2)\n May (1)\n April (1)\n 17 (1)\n 2011 (3)\n January (1)\n 1 (1)\n" };
- searchIndex(taxoReader, searcher, true, exp, new String[][] { new String[] { "Date/2011/January/1", "Date/2010/April" }}, null);
-
- // non-hierarchical dimension with both parent and child
- exp = new String[] { "A (0)\n 1 (5)\n 4 (2)\n 3 (2)\n" };
- searchIndex(taxoReader, searcher, INFOSTREAM, exp, new String[][] { new String[] { "A", "A/3" }}, null);
-
- // non-hierarchical dimension with same request but different numResults
- exp = new String[] { "A (0)\n 1 (5)\n 4 (2)\n 3 (2)\n 2 (2)\n" };
- searchIndex(taxoReader, searcher, INFOSTREAM, exp, new String[][] { new String[] { "A", "A" }}, new int[] { 2, 4 });
-
- IOUtils.close(indexReader, taxoReader);
-
- IOUtils.close(indexDir, taxoDir);
- }
-
-}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/MultiCategoryListIteratorTest.java b/lucene/facet/src/test/org/apache/lucene/facet/search/MultiCategoryListIteratorTest.java
deleted file mode 100644
index 268f6bd87f0..00000000000
--- a/lucene/facet/src/test/org/apache/lucene/facet/search/MultiCategoryListIteratorTest.java
+++ /dev/null
@@ -1,118 +0,0 @@
-package org.apache.lucene.facet.search;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Random;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.FacetTestCase;
-import org.apache.lucene.facet.encoding.IntDecoder;
-import org.apache.lucene.facet.index.FacetFields;
-import org.apache.lucene.facet.params.CategoryListParams;
-import org.apache.lucene.facet.params.PerDimensionIndexingParams;
-import org.apache.lucene.facet.search.CategoryListIterator;
-import org.apache.lucene.facet.search.DocValuesCategoryListIterator;
-import org.apache.lucene.facet.taxonomy.FacetLabel;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
-import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
-import org.apache.lucene.facet.util.MultiCategoryListIterator;
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.IntsRef;
-import org.junit.Test;
-
-/*
- * 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.
- */
-
-public class MultiCategoryListIteratorTest extends FacetTestCase {
-
- @Test
- public void testMultipleCategoryLists() throws Exception {
- Random random = random();
- int numDimensions = atLeast(random, 2); // at least 2 dimensions
- String[] dimensions = new String[numDimensions];
- for (int i = 0; i < numDimensions; i++) {
- dimensions[i] = "dim" + i;
- }
-
- // build the PerDimensionIndexingParams
- HashMap clps = new HashMap();
- for (String dim : dimensions) {
- FacetLabel cp = new FacetLabel(dim);
- CategoryListParams clp = randomCategoryListParams("$" + dim);
- clps.put(cp, clp);
- }
- PerDimensionIndexingParams indexingParams = new PerDimensionIndexingParams(clps);
-
- // index some documents
- Directory indexDir = newDirectory();
- Directory taxoDir = newDirectory();
- IndexWriter indexWriter = new IndexWriter(indexDir, newIndexWriterConfig(TEST_VERSION_CURRENT, null).setMaxBufferedDocs(2));
- TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
- FacetFields facetFields = new FacetFields(taxoWriter, indexingParams);
- int ndocs = atLeast(random, 10);
- for (int i = 0; i < ndocs; i++) {
- Document doc = new Document();
- int numCategories = random.nextInt(numDimensions) + 1;
- ArrayList categories = new ArrayList();
- for (int j = 0; j < numCategories; j++) {
- String dimension = dimensions[random.nextInt(dimensions.length)];
- categories.add(new FacetLabel(dimension, Integer.toString(i)));
- }
- facetFields.addFields(doc, categories);
- indexWriter.addDocument(doc);
- }
- IOUtils.close(indexWriter, taxoWriter);
-
- // test the multi iterator
- DirectoryReader indexReader = DirectoryReader.open(indexDir);
- TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
- CategoryListIterator[] iterators = new CategoryListIterator[numDimensions];
- for (int i = 0; i < iterators.length; i++) {
- CategoryListParams clp = indexingParams.getCategoryListParams(new FacetLabel(dimensions[i]));
- IntDecoder decoder = clp.createEncoder().createMatchingDecoder();
- iterators[i] = new DocValuesCategoryListIterator(clp.field, decoder);
- }
- MultiCategoryListIterator cli = new MultiCategoryListIterator(iterators);
- for (AtomicReaderContext context : indexReader.leaves()) {
- assertTrue("failed to init multi-iterator", cli.setNextReader(context));
- IntsRef ordinals = new IntsRef();
- final int maxDoc = context.reader().maxDoc();
- for (int i = 0; i < maxDoc; i++) {
- cli.getOrdinals(i, ordinals);
- assertTrue("document " + i + " does not have categories", ordinals.length > 0);
- for (int j = 0; j < ordinals.length; j++) {
- FacetLabel cp = taxoReader.getPath(ordinals.ints[j]);
- assertNotNull("ordinal " + ordinals.ints[j] + " not found in taxonomy", cp);
- if (cp.length == 2) {
- int globalDoc = i + context.docBase;
- assertEquals("invalid category for document " + globalDoc, globalDoc, Integer.parseInt(cp.components[1]));
- }
- }
- }
- }
-
- IOUtils.close(indexReader, taxoReader);
- IOUtils.close(indexDir, taxoDir);
- }
-
-}
\ No newline at end of file
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java b/lucene/facet/src/test/org/apache/lucene/facet/simple/TestCachedOrdinalsReader.java
similarity index 74%
rename from lucene/facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java
rename to lucene/facet/src/test/org/apache/lucene/facet/simple/TestCachedOrdinalsReader.java
index ef1e9acb7e8..e368f5d5b41 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/simple/TestCachedOrdinalsReader.java
@@ -1,4 +1,21 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
+
+/*
+ * 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.
+ */
import java.io.IOException;
import java.util.Arrays;
@@ -18,45 +35,27 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.junit.Test;
-/*
- * 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.
- */
-
-public class OrdinalsCacheTest extends FacetTestCase {
+public class TestCachedOrdinalsReader extends FacetTestCase {
@Test
- public void testOrdinalsCacheWithThreads() throws Exception {
+ public void testWithThreads() throws Exception {
// LUCENE-5303: OrdinalsCache used the ThreadLocal BinaryDV instead of reader.getCoreCacheKey().
Directory indexDir = newDirectory();
Directory taxoDir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
IndexWriter writer = new IndexWriter(indexDir, conf);
DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
- FacetFields facetFields = new FacetFields(taxoWriter);
+ FacetsConfig config = new FacetsConfig(taxoWriter);
Document doc = new Document();
- facetFields.addFields(doc, Arrays.asList(new FacetLabel("A", "1")));
- writer.addDocument(doc);
+ doc.add(new FacetField("A", "1"));
+ writer.addDocument(config.build(doc));
doc = new Document();
- facetFields.addFields(doc, Arrays.asList(new FacetLabel("A", "2")));
- writer.addDocument(doc);
- writer.close();
- taxoWriter.close();
+ doc.add(new FacetField("A", "2"));
+ writer.addDocument(config.build(doc));
- final DirectoryReader reader = DirectoryReader.open(indexDir);
+ final DirectoryReader reader = DirectoryReader.open(writer, true);
+ final CachedOrdinalsReader ordsReader = new CachedOrdinalsReader(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME));
Thread[] threads = new Thread[3];
for (int i = 0; i < threads.length; i++) {
threads[i] = new Thread("CachedOrdsThread-" + i) {
@@ -64,7 +63,7 @@ public class OrdinalsCacheTest extends FacetTestCase {
public void run() {
for (AtomicReaderContext context : reader.leaves()) {
try {
- OrdinalsCache.getCachedOrds(context, FacetIndexingParams.DEFAULT.getCategoryListParams(new FacetLabel("A")));
+ ordsReader.getReader(context);
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -73,22 +72,17 @@ public class OrdinalsCacheTest extends FacetTestCase {
};
}
- OrdinalsCache.clear();
-
long ramBytesUsed = 0;
for (Thread t : threads) {
t.start();
t.join();
if (ramBytesUsed == 0) {
- ramBytesUsed = OrdinalsCache.ramBytesUsed();
+ ramBytesUsed = ordsReader.ramBytesUsed();
} else {
- assertEquals(ramBytesUsed, OrdinalsCache.ramBytesUsed());
+ assertEquals(ramBytesUsed, ordsReader.ramBytesUsed());
}
}
- reader.close();
-
- IOUtils.close(indexDir, taxoDir);
+ IOUtils.close(writer, taxoWriter, reader, indexDir, taxoDir);
}
-
}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownQueryTest.java b/lucene/facet/src/test/org/apache/lucene/facet/simple/TestDrillDownQuery.java
similarity index 67%
rename from lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownQueryTest.java
rename to lucene/facet/src/test/org/apache/lucene/facet/simple/TestDrillDownQuery.java
index c47dbf86931..afeca0a658f 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownQueryTest.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/simple/TestDrillDownQuery.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -53,15 +53,13 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
-public class DrillDownQueryTest extends FacetTestCase {
+public class TestDrillDownQuery extends FacetTestCase {
private static IndexReader reader;
private static DirectoryTaxonomyReader taxo;
private static Directory dir;
private static Directory taxoDir;
-
- private FacetIndexingParams defaultParams;
- private PerDimensionIndexingParams nonDefaultParams;
+ private static FacetsConfig config;
@AfterClass
public static void afterClassDrillDownQueryTest() throws Exception {
@@ -70,6 +68,7 @@ public class DrillDownQueryTest extends FacetTestCase {
taxo = null;
dir = null;
taxoDir = null;
+ config = null;
}
@BeforeClass
@@ -81,9 +80,24 @@ public class DrillDownQueryTest extends FacetTestCase {
taxoDir = newDirectory();
TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
-
+ config = new FacetsConfig(taxoWriter);
+
+ // Randomize the per-dim config:
+ config.setHierarchical("a", random().nextBoolean());
+ config.setMultiValued("a", random().nextBoolean());
+ if (random().nextBoolean()) {
+ config.setIndexFieldName("a", "$a");
+ }
+ config.setRequireDimCount("a", true);
+
+ config.setHierarchical("b", random().nextBoolean());
+ config.setMultiValued("b", random().nextBoolean());
+ if (random().nextBoolean()) {
+ config.setIndexFieldName("b", "$b");
+ }
+ config.setRequireDimCount("b", true);
+
for (int i = 0; i < 100; i++) {
- ArrayList paths = new ArrayList();
Document doc = new Document();
if (i % 2 == 0) { // 50
doc.add(new TextField("content", "foo", Field.Store.NO));
@@ -93,19 +107,15 @@ public class DrillDownQueryTest extends FacetTestCase {
}
if (i % 4 == 0) { // 25
if (r.nextBoolean()) {
- paths.add(new FacetLabel("a/1", '/'));
+ doc.add(new FacetField("a", "1"));
} else {
- paths.add(new FacetLabel("a/2", '/'));
+ doc.add(new FacetField("a", "2"));
}
}
if (i % 5 == 0) { // 20
- paths.add(new FacetLabel("b"));
+ doc.add(new FacetField("b", "1"));
}
- FacetFields facetFields = new FacetFields(taxoWriter);
- if (paths.size() > 0) {
- facetFields.addFields(doc, paths);
- }
- writer.addDocument(doc);
+ writer.addDocument(config.build(doc));
}
taxoWriter.close();
@@ -115,96 +125,75 @@ public class DrillDownQueryTest extends FacetTestCase {
taxo = new DirectoryTaxonomyReader(taxoDir);
}
- public DrillDownQueryTest() {
- Map paramsMap = new HashMap();
- paramsMap.put(new FacetLabel("a"), randomCategoryListParams("testing_facets_a"));
- paramsMap.put(new FacetLabel("b"), randomCategoryListParams("testing_facets_b"));
- nonDefaultParams = new PerDimensionIndexingParams(paramsMap);
- defaultParams = new FacetIndexingParams(randomCategoryListParams(CategoryListParams.DEFAULT_FIELD));
- }
-
- @Test
- public void testDefaultField() {
- String defaultField = CategoryListParams.DEFAULT_FIELD;
-
- Term termA = DrillDownQuery.term(defaultParams, new FacetLabel("a"));
- assertEquals(new Term(defaultField, "a"), termA);
-
- Term termB = DrillDownQuery.term(defaultParams, new FacetLabel("b"));
- assertEquals(new Term(defaultField, "b"), termB);
- }
-
- @Test
public void testAndOrs() throws Exception {
IndexSearcher searcher = newSearcher(reader);
- // test (a/1 OR a/2) AND b
- DrillDownQuery q = new DrillDownQuery(defaultParams);
- q.add(new FacetLabel("a/1", '/'), new FacetLabel("a/2", '/'));
- q.add(new FacetLabel("b"));
+ // test (a/1 OR a/2) AND b/1
+ SimpleDrillDownQuery q = new SimpleDrillDownQuery(config);
+ q.add("a", "1");
+ q.add("a", "2");
+ q.add("b", "1");
TopDocs docs = searcher.search(q, 100);
assertEquals(5, docs.totalHits);
}
- @Test
public void testQuery() throws IOException {
IndexSearcher searcher = newSearcher(reader);
// Making sure the query yields 25 documents with the facet "a"
- DrillDownQuery q = new DrillDownQuery(defaultParams);
- q.add(new FacetLabel("a"));
+ SimpleDrillDownQuery q = new SimpleDrillDownQuery(config);
+ q.add("a");
+ System.out.println("q=" + q);
QueryUtils.check(q);
TopDocs docs = searcher.search(q, 100);
assertEquals(25, docs.totalHits);
// Making sure the query yields 5 documents with the facet "b" and the
// previous (facet "a") query as a base query
- DrillDownQuery q2 = new DrillDownQuery(defaultParams, q);
- q2.add(new FacetLabel("b"));
+ SimpleDrillDownQuery q2 = new SimpleDrillDownQuery(config, q);
+ q2.add("b");
docs = searcher.search(q2, 100);
assertEquals(5, docs.totalHits);
// Making sure that a query of both facet "a" and facet "b" yields 5 results
- DrillDownQuery q3 = new DrillDownQuery(defaultParams);
- q3.add(new FacetLabel("a"));
- q3.add(new FacetLabel("b"));
+ SimpleDrillDownQuery q3 = new SimpleDrillDownQuery(config);
+ q3.add("a");
+ q3.add("b");
docs = searcher.search(q3, 100);
assertEquals(5, docs.totalHits);
// Check that content:foo (which yields 50% results) and facet/b (which yields 20%)
// would gather together 10 results (10%..)
Query fooQuery = new TermQuery(new Term("content", "foo"));
- DrillDownQuery q4 = new DrillDownQuery(defaultParams, fooQuery);
- q4.add(new FacetLabel("b"));
+ SimpleDrillDownQuery q4 = new SimpleDrillDownQuery(config, fooQuery);
+ q4.add("b");
docs = searcher.search(q4, 100);
assertEquals(10, docs.totalHits);
}
- @Test
public void testQueryImplicitDefaultParams() throws IOException {
IndexSearcher searcher = newSearcher(reader);
// Create the base query to start with
- DrillDownQuery q = new DrillDownQuery(defaultParams);
- q.add(new FacetLabel("a"));
+ SimpleDrillDownQuery q = new SimpleDrillDownQuery(config);
+ q.add("a");
// Making sure the query yields 5 documents with the facet "b" and the
// previous (facet "a") query as a base query
- DrillDownQuery q2 = new DrillDownQuery(defaultParams, q);
- q2.add(new FacetLabel("b"));
+ SimpleDrillDownQuery q2 = new SimpleDrillDownQuery(config, q);
+ q2.add("b");
TopDocs docs = searcher.search(q2, 100);
assertEquals(5, docs.totalHits);
// Check that content:foo (which yields 50% results) and facet/b (which yields 20%)
// would gather together 10 results (10%..)
Query fooQuery = new TermQuery(new Term("content", "foo"));
- DrillDownQuery q4 = new DrillDownQuery(defaultParams, fooQuery);
- q4.add(new FacetLabel("b"));
+ SimpleDrillDownQuery q4 = new SimpleDrillDownQuery(config, fooQuery);
+ q4.add("b");
docs = searcher.search(q4, 100);
assertEquals(10, docs.totalHits);
}
- @Test
public void testScoring() throws IOException {
// verify that drill-down queries do not modify scores
IndexSearcher searcher = newSearcher(reader);
@@ -218,58 +207,50 @@ public class DrillDownQueryTest extends FacetTestCase {
}
// create a drill-down query with category "a", scores should not change
- DrillDownQuery q2 = new DrillDownQuery(defaultParams, q);
- q2.add(new FacetLabel("a"));
+ SimpleDrillDownQuery q2 = new SimpleDrillDownQuery(config, q);
+ q2.add("a");
docs = searcher.search(q2, reader.maxDoc()); // fetch all available docs to this query
for (ScoreDoc sd : docs.scoreDocs) {
assertEquals("score of doc=" + sd.doc + " modified", scores[sd.doc], sd.score, 0f);
}
}
- @Test
public void testScoringNoBaseQuery() throws IOException {
// verify that drill-down queries (with no base query) returns 0.0 score
IndexSearcher searcher = newSearcher(reader);
- DrillDownQuery q = new DrillDownQuery(defaultParams);
- q.add(new FacetLabel("a"));
+ SimpleDrillDownQuery q = new SimpleDrillDownQuery(config);
+ q.add("a");
TopDocs docs = searcher.search(q, reader.maxDoc()); // fetch all available docs to this query
for (ScoreDoc sd : docs.scoreDocs) {
assertEquals(0f, sd.score, 0f);
}
}
- @Test
public void testTermNonDefault() {
- Term termA = DrillDownQuery.term(nonDefaultParams, new FacetLabel("a"));
- assertEquals(new Term("testing_facets_a", "a"), termA);
+ String aField = config.getDimConfig("a").indexFieldName;
+ Term termA = SimpleDrillDownQuery.term(aField, "a");
+ assertEquals(new Term(aField, "a"), termA);
- Term termB = DrillDownQuery.term(nonDefaultParams, new FacetLabel("b"));
- assertEquals(new Term("testing_facets_b", "b"), termB);
+ String bField = config.getDimConfig("b").indexFieldName;
+ Term termB = SimpleDrillDownQuery.term(bField, "b");
+ assertEquals(new Term(bField, "b"), termB);
}
- @Test
public void testClone() throws Exception {
- DrillDownQuery q = new DrillDownQuery(defaultParams, new MatchAllDocsQuery());
- q.add(new FacetLabel("a"));
+ SimpleDrillDownQuery q = new SimpleDrillDownQuery(config, new MatchAllDocsQuery());
+ q.add("a");
- DrillDownQuery clone = q.clone();
- clone.add(new FacetLabel("b"));
+ SimpleDrillDownQuery clone = q.clone();
+ clone.add("b");
assertFalse("query wasn't cloned: source=" + q + " clone=" + clone, q.toString().equals(clone.toString()));
}
- @Test(expected=IllegalStateException.class)
- public void testNoBaseNorDrillDown() throws Exception {
- DrillDownQuery q = new DrillDownQuery(defaultParams);
- q.rewrite(reader);
- }
-
public void testNoDrillDown() throws Exception {
Query base = new MatchAllDocsQuery();
- DrillDownQuery q = new DrillDownQuery(defaultParams, base);
+ SimpleDrillDownQuery q = new SimpleDrillDownQuery(config, base);
Query rewrite = q.rewrite(reader).rewrite(reader);
assertSame(base, rewrite);
}
-
}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/TestSearcherTaxonomyManager.java b/lucene/facet/src/test/org/apache/lucene/facet/simple/TestSearcherTaxonomyManager.java
similarity index 87%
rename from lucene/facet/src/test/org/apache/lucene/facet/search/TestSearcherTaxonomyManager.java
rename to lucene/facet/src/test/org/apache/lucene/facet/simple/TestSearcherTaxonomyManager.java
index 653d7efa79e..76e8856350a 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/search/TestSearcherTaxonomyManager.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/simple/TestSearcherTaxonomyManager.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -31,6 +31,7 @@ import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.params.FacetIndexingParams;
import org.apache.lucene.facet.params.FacetSearchParams;
import org.apache.lucene.facet.search.SearcherTaxonomyManager.SearcherAndTaxonomy;
+import org.apache.lucene.facet.search.SearcherTaxonomyManager;
import org.apache.lucene.facet.taxonomy.FacetLabel;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
import org.apache.lucene.index.IndexWriter;
@@ -45,7 +46,8 @@ public class TestSearcherTaxonomyManager extends FacetTestCase {
Directory taxoDir = newDirectory();
final IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
final DirectoryTaxonomyWriter tw = new DirectoryTaxonomyWriter(taxoDir);
- final FacetFields facetFields = new FacetFields(tw);
+ final FacetsConfig config = new FacetsConfig(tw);
+ config.setMultiValued("field", true);
final AtomicBoolean stop = new AtomicBoolean();
// How many unique facets to index before stopping:
@@ -78,11 +80,10 @@ public class TestSearcherTaxonomyManager extends FacetTestCase {
}
}
}
- docPaths.add(new FacetLabel("field", path));
+ doc.add(new FacetField("field", path));
}
try {
- facetFields.addFields(doc, docPaths);
- w.addDocument(doc);
+ w.addDocument(config.build(doc));
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
@@ -132,19 +133,15 @@ public class TestSearcherTaxonomyManager extends FacetTestCase {
try {
//System.out.println("search maxOrd=" + pair.taxonomyReader.getSize());
int topN = _TestUtil.nextInt(random(), 1, 20);
- CountFacetRequest cfr = new CountFacetRequest(new FacetLabel("field"), topN);
- FacetSearchParams fsp = new FacetSearchParams(cfr);
- FacetsCollector fc = FacetsCollector.create(fsp, pair.searcher.getIndexReader(), pair.taxonomyReader);
- pair.searcher.search(new MatchAllDocsQuery(), fc);
- List results = fc.getFacetResults();
- FacetResult fr = results.get(0);
- FacetResultNode root = results.get(0).getFacetResultNode();
- assertTrue(root.ordinal != 0);
-
+
+ SimpleFacetsCollector sfc = new SimpleFacetsCollector();
+ pair.searcher.search(new MatchAllDocsQuery(), sfc);
+ Facets facets = getTaxonomyFacetCounts(pair.taxonomyReader, config, sfc);
+ SimpleFacetResult result = facets.getTopChildren(10, "field");
if (pair.searcher.getIndexReader().numDocs() > 0) {
//System.out.println(pair.taxonomyReader.getSize());
- assertTrue(fr.getNumValidDescendants() > 0);
- assertFalse(root.subResults.isEmpty());
+ assertTrue(result.childCount > 0);
+ assertTrue(result.labelValues.length > 0);
}
//if (VERBOSE) {
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsAggregatorTest.java b/lucene/facet/src/test/org/apache/lucene/facet/simple/TestTaxonomyFacetCounts2.java
similarity index 54%
rename from lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsAggregatorTest.java
rename to lucene/facet/src/test/org/apache/lucene/facet/simple/TestTaxonomyFacetCounts2.java
index ed62055569b..d06166e592f 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsAggregatorTest.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/simple/TestTaxonomyFacetCounts2.java
@@ -1,4 +1,21 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
+
+/*
+ * 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.
+ */
import java.io.IOException;
import java.util.ArrayList;
@@ -41,78 +58,60 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
-/*
- * 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.
- */
-
-public class CountingFacetsAggregatorTest extends FacetTestCase {
+public class TestTaxonomyFacetCounts2 extends FacetTestCase {
private static final Term A = new Term("f", "a");
- private static final FacetLabel CP_A = new FacetLabel("A"), CP_B = new FacetLabel("B");
- private static final FacetLabel CP_C = new FacetLabel("C"), CP_D = new FacetLabel("D"); // indexed w/ NO_PARENTS
+ private static final String CP_A = "A", CP_B = "B";
+ private static final String CP_C = "C", CP_D = "D"; // indexed w/ NO_PARENTS
private static final int NUM_CHILDREN_CP_A = 5, NUM_CHILDREN_CP_B = 3;
private static final int NUM_CHILDREN_CP_C = 5, NUM_CHILDREN_CP_D = 5;
- private static final FacetLabel[] CATEGORIES_A, CATEGORIES_B;
- private static final FacetLabel[] CATEGORIES_C, CATEGORIES_D;
+ private static final FacetField[] CATEGORIES_A, CATEGORIES_B;
+ private static final FacetField[] CATEGORIES_C, CATEGORIES_D;
static {
- CATEGORIES_A = new FacetLabel[NUM_CHILDREN_CP_A];
+ CATEGORIES_A = new FacetField[NUM_CHILDREN_CP_A];
for (int i = 0; i < NUM_CHILDREN_CP_A; i++) {
- CATEGORIES_A[i] = new FacetLabel(CP_A.components[0], Integer.toString(i));
+ CATEGORIES_A[i] = new FacetField(CP_A, Integer.toString(i));
}
- CATEGORIES_B = new FacetLabel[NUM_CHILDREN_CP_B];
+ CATEGORIES_B = new FacetField[NUM_CHILDREN_CP_B];
for (int i = 0; i < NUM_CHILDREN_CP_B; i++) {
- CATEGORIES_B[i] = new FacetLabel(CP_B.components[0], Integer.toString(i));
+ CATEGORIES_B[i] = new FacetField(CP_B, Integer.toString(i));
}
// NO_PARENTS categories
- CATEGORIES_C = new FacetLabel[NUM_CHILDREN_CP_C];
+ CATEGORIES_C = new FacetField[NUM_CHILDREN_CP_C];
for (int i = 0; i < NUM_CHILDREN_CP_C; i++) {
- CATEGORIES_C[i] = new FacetLabel(CP_C.components[0], Integer.toString(i));
+ CATEGORIES_C[i] = new FacetField(CP_C, Integer.toString(i));
}
// Multi-level categories
- CATEGORIES_D = new FacetLabel[NUM_CHILDREN_CP_D];
+ CATEGORIES_D = new FacetField[NUM_CHILDREN_CP_D];
for (int i = 0; i < NUM_CHILDREN_CP_D; i++) {
String val = Integer.toString(i);
- CATEGORIES_D[i] = new FacetLabel(CP_D.components[0], val, val + val); // e.g. D/1/11, D/2/22...
+ CATEGORIES_D[i] = new FacetField(CP_D, val, val + val); // e.g. D/1/11, D/2/22...
}
}
private static Directory indexDir, taxoDir;
- private static ObjectToIntMap allExpectedCounts, termExpectedCounts;
- private static FacetIndexingParams fip;
+ private static ObjectToIntMap allExpectedCounts, termExpectedCounts;
@AfterClass
public static void afterClassCountingFacetsAggregatorTest() throws Exception {
IOUtils.close(indexDir, taxoDir);
}
- private static List randomCategories(Random random) {
+ private static List randomCategories(Random random) {
// add random categories from the two dimensions, ensuring that the same
// category is not added twice.
int numFacetsA = random.nextInt(3) + 1; // 1-3
int numFacetsB = random.nextInt(2) + 1; // 1-2
- ArrayList categories_a = new ArrayList();
+ ArrayList categories_a = new ArrayList();
categories_a.addAll(Arrays.asList(CATEGORIES_A));
- ArrayList categories_b = new ArrayList();
+ ArrayList categories_b = new ArrayList();
categories_b.addAll(Arrays.asList(CATEGORIES_B));
Collections.shuffle(categories_a, random);
Collections.shuffle(categories_b, random);
- ArrayList categories = new ArrayList();
+ ArrayList categories = new ArrayList();
categories.addAll(categories_a.subList(0, numFacetsA));
categories.addAll(categories_b.subList(0, numFacetsB));
@@ -126,14 +125,13 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
private static void addField(Document doc) {
doc.add(new StringField(A.field(), A.text(), Store.NO));
}
-
- private static void addFacets(Document doc, FacetFields facetFields, boolean updateTermExpectedCounts)
+
+ private static void addFacets(Document doc, FacetsConfig config, boolean updateTermExpectedCounts)
throws IOException {
- List docCategories = randomCategories(random());
- for (FacetLabel cp : docCategories) {
- if (cp.components[0].equals(CP_D.components[0])) {
- cp = cp.subpath(2); // we'll get counts for the 2nd level only
- }
+ List docCategories = randomCategories(random());
+ for (FacetField ff : docCategories) {
+ doc.add(ff);
+ String cp = ff.dim + "/" + ff.path[0];
allExpectedCounts.put(cp, allExpectedCounts.get(cp) + 1);
if (updateTermExpectedCounts) {
termExpectedCounts.put(cp, termExpectedCounts.get(cp) + 1);
@@ -148,8 +146,19 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
termExpectedCounts.put(CP_C, termExpectedCounts.get(CP_C) + 1);
termExpectedCounts.put(CP_D, termExpectedCounts.get(CP_D) + 1);
}
-
- facetFields.addFields(doc, docCategories);
+ }
+
+ private static FacetsConfig getConfig(TaxonomyWriter taxoWriter) {
+ FacetsConfig config = new FacetsConfig(taxoWriter);
+ config.setMultiValued("A", true);
+ config.setMultiValued("B", true);
+ config.setRequireDimCount("B", true);
+ config.setHierarchical("D", true);
+ return config;
+ }
+
+ private static FacetsConfig getConfig() {
+ return getConfig(null);
}
private static void indexDocsNoFacets(IndexWriter indexWriter) throws IOException {
@@ -163,67 +172,67 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
}
private static void indexDocsWithFacetsNoTerms(IndexWriter indexWriter, TaxonomyWriter taxoWriter,
- ObjectToIntMap expectedCounts) throws IOException {
+ ObjectToIntMap expectedCounts) throws IOException {
Random random = random();
int numDocs = atLeast(random, 2);
- FacetFields facetFields = new FacetFields(taxoWriter, fip);
+ FacetsConfig config = getConfig(taxoWriter);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
- addFacets(doc, facetFields, false);
- indexWriter.addDocument(doc);
+ addFacets(doc, config, false);
+ indexWriter.addDocument(config.build(doc));
}
indexWriter.commit(); // flush a segment
}
private static void indexDocsWithFacetsAndTerms(IndexWriter indexWriter, TaxonomyWriter taxoWriter,
- ObjectToIntMap expectedCounts) throws IOException {
+ ObjectToIntMap expectedCounts) throws IOException {
Random random = random();
int numDocs = atLeast(random, 2);
- FacetFields facetFields = new FacetFields(taxoWriter, fip);
+ FacetsConfig config = getConfig(taxoWriter);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
- addFacets(doc, facetFields, true);
+ addFacets(doc, config, true);
addField(doc);
- indexWriter.addDocument(doc);
+ indexWriter.addDocument(config.build(doc));
}
indexWriter.commit(); // flush a segment
}
private static void indexDocsWithFacetsAndSomeTerms(IndexWriter indexWriter, TaxonomyWriter taxoWriter,
- ObjectToIntMap expectedCounts) throws IOException {
+ ObjectToIntMap expectedCounts) throws IOException {
Random random = random();
int numDocs = atLeast(random, 2);
- FacetFields facetFields = new FacetFields(taxoWriter, fip);
+ FacetsConfig config = getConfig(taxoWriter);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
boolean hasContent = random.nextBoolean();
if (hasContent) {
addField(doc);
}
- addFacets(doc, facetFields, hasContent);
- indexWriter.addDocument(doc);
+ addFacets(doc, config, hasContent);
+ indexWriter.addDocument(config.build(doc));
}
indexWriter.commit(); // flush a segment
}
// initialize expectedCounts w/ 0 for all categories
- private static ObjectToIntMap newCounts() {
- ObjectToIntMap counts = new ObjectToIntMap();
+ private static ObjectToIntMap newCounts() {
+ ObjectToIntMap counts = new ObjectToIntMap();
counts.put(CP_A, 0);
counts.put(CP_B, 0);
counts.put(CP_C, 0);
counts.put(CP_D, 0);
- for (FacetLabel cp : CATEGORIES_A) {
- counts.put(cp, 0);
+ for (FacetField ff : CATEGORIES_A) {
+ counts.put(ff.dim + "/" + ff.path[0], 0);
}
- for (FacetLabel cp : CATEGORIES_B) {
- counts.put(cp, 0);
+ for (FacetField ff : CATEGORIES_B) {
+ counts.put(ff.dim + "/" + ff.path[0], 0);
}
- for (FacetLabel cp : CATEGORIES_C) {
- counts.put(cp, 0);
+ for (FacetField ff : CATEGORIES_C) {
+ counts.put(ff.dim + "/" + ff.path[0], 0);
}
- for (FacetLabel cp : CATEGORIES_D) {
- counts.put(cp.subpath(2), 0);
+ for (FacetField ff : CATEGORIES_D) {
+ counts.put(ff.dim + "/" + ff.path[0], 0);
}
return counts;
}
@@ -244,13 +253,6 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
IndexWriter indexWriter = new IndexWriter(indexDir, conf);
TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
- Map policies = new HashMap();
- policies.put(CP_B.components[0], OrdinalPolicy.ALL_PARENTS);
- policies.put(CP_C.components[0], OrdinalPolicy.NO_PARENTS);
- policies.put(CP_D.components[0], OrdinalPolicy.NO_PARENTS);
- CategoryListParams clp = new PerDimensionOrdinalPolicy(policies);
- fip = new FacetIndexingParams(clp);
-
allExpectedCounts = newCounts();
termExpectedCounts = newCounts();
@@ -269,24 +271,6 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
IOUtils.close(indexWriter, taxoWriter);
}
- private TaxonomyFacetsAccumulator randomAccumulator(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
- final FacetsAggregator aggregator;
- double val = random().nextDouble();
- if (val < 0.6) {
- aggregator = new FastCountingFacetsAggregator(); // it's the default, so give it the highest chance
- } else if (val < 0.8) {
- aggregator = new CountingFacetsAggregator();
- } else {
- aggregator = new CachedOrdsCountingFacetsAggregator();
- }
- return new TaxonomyFacetsAccumulator(fsp, indexReader, taxoReader) {
- @Override
- public FacetsAggregator getAggregator() {
- return aggregator;
- }
- };
- }
-
@Test
public void testDifferentNumResults() throws Exception {
// test the collector w/ FacetRequests and different numResults
@@ -294,20 +278,19 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = newSearcher(indexReader);
- FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, NUM_CHILDREN_CP_A),
- new CountFacetRequest(CP_B, NUM_CHILDREN_CP_B));
- FacetsCollector fc = FacetsCollector.create(randomAccumulator(fsp, indexReader, taxoReader));
+ SimpleFacetsCollector sfc = new SimpleFacetsCollector();
TermQuery q = new TermQuery(A);
- searcher.search(q, fc);
-
- List facetResults = fc.getFacetResults();
- assertEquals("invalid number of facet results", 2, facetResults.size());
- for (FacetResult res : facetResults) {
- FacetResultNode root = res.getFacetResultNode();
- assertEquals("wrong count for " + root.label, termExpectedCounts.get(root.label), (int) root.value);
- for (FacetResultNode child : root.subResults) {
- assertEquals("wrong count for " + child.label, termExpectedCounts.get(child.label), (int) child.value);
- }
+ searcher.search(q, sfc);
+ Facets facets = getTaxonomyFacetCounts(taxoReader, getConfig(), sfc);
+ SimpleFacetResult result = facets.getTopChildren(NUM_CHILDREN_CP_A, CP_A);
+ assertEquals(-1, result.value.intValue());
+ for(LabelAndValue labelValue : result.labelValues) {
+ assertEquals(termExpectedCounts.get(CP_A + "/" + labelValue.label), labelValue.value.intValue());
+ }
+ result = facets.getTopChildren(NUM_CHILDREN_CP_B, CP_B);
+ assertEquals(termExpectedCounts.get(CP_B), result.value.intValue());
+ for(LabelAndValue labelValue : result.labelValues) {
+ assertEquals(termExpectedCounts.get(CP_B + "/" + labelValue.label), labelValue.value.intValue());
}
IOUtils.close(indexReader, taxoReader);
@@ -319,29 +302,29 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = newSearcher(indexReader);
- FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, NUM_CHILDREN_CP_A),
- new CountFacetRequest(CP_B, NUM_CHILDREN_CP_B));
- FacetsCollector fc = FacetsCollector.create(randomAccumulator(fsp, indexReader, taxoReader));
- searcher.search(new MatchAllDocsQuery(), fc);
+ SimpleFacetsCollector sfc = new SimpleFacetsCollector();
+ searcher.search(new MatchAllDocsQuery(), sfc);
+
+ Facets facets = getTaxonomyFacetCounts(taxoReader, getConfig(), sfc);
- List facetResults = fc.getFacetResults();
- assertEquals("invalid number of facet results", 2, facetResults.size());
- for (FacetResult res : facetResults) {
- FacetResultNode root = res.getFacetResultNode();
- assertEquals("wrong count for " + root.label, allExpectedCounts.get(root.label), (int) root.value);
- int prevValue = Integer.MAX_VALUE;
- int prevOrdinal = Integer.MAX_VALUE;
- for (FacetResultNode child : root.subResults) {
- assertEquals("wrong count for " + child.label, allExpectedCounts.get(child.label), (int) child.value);
- assertTrue("wrong sort order of sub results: child.value=" + child.value + " prevValue=" + prevValue, child.value <= prevValue);
- if (child.value == prevValue) {
- assertTrue("wrong sort order of sub results", child.ordinal < prevOrdinal);
- }
- prevValue = (int) child.value;
- prevOrdinal = child.ordinal;
- }
+ SimpleFacetResult result = facets.getTopChildren(NUM_CHILDREN_CP_A, CP_A);
+ assertEquals(-1, result.value.intValue());
+ int prevValue = Integer.MAX_VALUE;
+ for(LabelAndValue labelValue : result.labelValues) {
+ assertEquals(allExpectedCounts.get(CP_A + "/" + labelValue.label), labelValue.value.intValue());
+ assertTrue("wrong sort order of sub results: labelValue.value=" + labelValue.value + " prevValue=" + prevValue, labelValue.value.intValue() <= prevValue);
+ prevValue = labelValue.value.intValue();
}
-
+
+ result = facets.getTopChildren(NUM_CHILDREN_CP_B, CP_B);
+ assertEquals(allExpectedCounts.get(CP_B), result.value.intValue());
+ prevValue = Integer.MAX_VALUE;
+ for(LabelAndValue labelValue : result.labelValues) {
+ assertEquals(allExpectedCounts.get(CP_B + "/" + labelValue.label), labelValue.value.intValue());
+ assertTrue("wrong sort order of sub results: labelValue.value=" + labelValue.value + " prevValue=" + prevValue, labelValue.value.intValue() <= prevValue);
+ prevValue = labelValue.value.intValue();
+ }
+
IOUtils.close(indexReader, taxoReader);
}
@@ -351,19 +334,20 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = newSearcher(indexReader);
- FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, Integer.MAX_VALUE),
- new CountFacetRequest(CP_B, Integer.MAX_VALUE));
- FacetsCollector fc = FacetsCollector.create(randomAccumulator(fsp, indexReader, taxoReader));
- searcher.search(new MatchAllDocsQuery(), fc);
-
- List facetResults = fc.getFacetResults();
- assertEquals("invalid number of facet results", 2, facetResults.size());
- for (FacetResult res : facetResults) {
- FacetResultNode root = res.getFacetResultNode();
- assertEquals("wrong count for " + root.label, allExpectedCounts.get(root.label), (int) root.value);
- for (FacetResultNode child : root.subResults) {
- assertEquals("wrong count for " + child.label, allExpectedCounts.get(child.label), (int) child.value);
- }
+ SimpleFacetsCollector sfc = new SimpleFacetsCollector();
+ searcher.search(new MatchAllDocsQuery(), sfc);
+
+ Facets facets = getTaxonomyFacetCounts(taxoReader, getConfig(), sfc);
+
+ SimpleFacetResult result = facets.getTopChildren(Integer.MAX_VALUE, CP_A);
+ assertEquals(-1, result.value.intValue());
+ for(LabelAndValue labelValue : result.labelValues) {
+ assertEquals(allExpectedCounts.get(CP_A + "/" + labelValue.label), labelValue.value.intValue());
+ }
+ result = facets.getTopChildren(Integer.MAX_VALUE, CP_B);
+ assertEquals(allExpectedCounts.get(CP_B), result.value.intValue());
+ for(LabelAndValue labelValue : result.labelValues) {
+ assertEquals(allExpectedCounts.get(CP_B + "/" + labelValue.label), labelValue.value.intValue());
}
IOUtils.close(indexReader, taxoReader);
@@ -374,22 +358,23 @@ public class CountingFacetsAggregatorTest extends FacetTestCase {
DirectoryReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = newSearcher(indexReader);
- FacetSearchParams fsp = new FacetSearchParams(fip, new CountFacetRequest(CP_C, NUM_CHILDREN_CP_C),
- new CountFacetRequest(CP_D, NUM_CHILDREN_CP_D));
- FacetsCollector fc = FacetsCollector.create(randomAccumulator(fsp, indexReader, taxoReader));
- searcher.search(new MatchAllDocsQuery(), fc);
- List facetResults = fc.getFacetResults();
- assertEquals("invalid number of facet results", fsp.facetRequests.size(), facetResults.size());
- for (FacetResult res : facetResults) {
- FacetResultNode root = res.getFacetResultNode();
- assertEquals("wrong count for " + root.label, allExpectedCounts.get(root.label), (int) root.value);
- for (FacetResultNode child : root.subResults) {
- assertEquals("wrong count for " + child.label, allExpectedCounts.get(child.label), (int) child.value);
- }
+ SimpleFacetsCollector sfc = new SimpleFacetsCollector();
+ searcher.search(new MatchAllDocsQuery(), sfc);
+
+ Facets facets = getTaxonomyFacetCounts(taxoReader, getConfig(), sfc);
+
+ SimpleFacetResult result = facets.getTopChildren(NUM_CHILDREN_CP_C, CP_C);
+ assertEquals(allExpectedCounts.get(CP_C), result.value.intValue());
+ for(LabelAndValue labelValue : result.labelValues) {
+ assertEquals(allExpectedCounts.get(CP_C + "/" + labelValue.label), labelValue.value.intValue());
+ }
+ result = facets.getTopChildren(NUM_CHILDREN_CP_D, CP_D);
+ assertEquals(allExpectedCounts.get(CP_C), result.value.intValue());
+ for(LabelAndValue labelValue : result.labelValues) {
+ assertEquals(allExpectedCounts.get(CP_D + "/" + labelValue.label), labelValue.value.intValue());
}
IOUtils.close(indexReader, taxoReader);
}
-
}