You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/02/12 12:39:23 UTC

svn commit: r1445108 - in /lucene/dev/trunk/lucene: ./ demo/src/java/org/apache/lucene/demo/facet/ demo/src/test/org/apache/lucene/demo/facet/ facet/src/java/org/apache/lucene/facet/associations/ facet/src/java/org/apache/lucene/facet/search/ facet/src...

Author: shaie
Date: Tue Feb 12 11:39:23 2013
New Revision: 1445108

URL: http://svn.apache.org/r1445108
Log:
LUCENE-4772: Move Facet associations to new FacetsAggregator API

Added:
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumFloatAssociationFacetsAggregator.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumIntAssociationFacetsAggregator.java
      - copied, changed from r1444520, lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetsAggregator.java
Removed:
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationFloatSumAggregator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumAggregator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetsAggregator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsIterator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsIterator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsIterator.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java
    lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/facet/TestAssociationsFacetsExample.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationFloatSumFacetRequest.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetRequest.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1445108&r1=1445107&r2=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Feb 12 11:39:23 2013
@@ -72,6 +72,10 @@ Changes in backwards compatibility polic
 * LUCENE-4759: remove FacetRequest.SortBy; result categories are always 
   sorted by value, while ties are broken by category ordinal. (Shai Erera)
 
+* LUCENE-4772: Facet associations moved to new FacetsAggregator API. You
+  should override FacetsAccumualtor and return the relevant aggregator,
+  for aggregating the association values. (Shai Erera)
+  
 Optimizations
 
 * LUCENE-4687: BloomFilterPostingsFormat now lazily initializes delegate

Modified: lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java?rev=1445108&r1=1445107&r2=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java (original)
+++ lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java Tue Feb 12 11:39:23 2013
@@ -1,7 +1,9 @@
 package org.apache.lucene.demo.facet;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
 import org.apache.lucene.document.Document;
@@ -12,12 +14,15 @@ import org.apache.lucene.facet.associati
 import org.apache.lucene.facet.associations.CategoryAssociationsContainer;
 import org.apache.lucene.facet.associations.CategoryFloatAssociation;
 import org.apache.lucene.facet.associations.CategoryIntAssociation;
+import org.apache.lucene.facet.associations.MultiAssociationsFacetsAggregator;
+import org.apache.lucene.facet.associations.SumFloatAssociationFacetsAggregator;
+import org.apache.lucene.facet.associations.SumIntAssociationFacetsAggregator;
 import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetsAccumulator;
+import org.apache.lucene.facet.search.FacetsAggregator;
 import org.apache.lucene.facet.search.FacetsCollector;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
@@ -122,43 +127,29 @@ public class AssociationsFacetsExample {
     taxoWriter.close();
   }
 
-  /** User runs a query and aggregates facets by summing their int associations. */
-  private List<FacetResult> sumIntAssociations() throws IOException {
+  /** User runs a query and aggregates facets by summing their association values. */
+  private List<FacetResult> sumAssociations() throws IOException {
     DirectoryReader indexReader = DirectoryReader.open(indexDir);
     IndexSearcher searcher = new IndexSearcher(indexReader);
     TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
-
-    // sum the 'tags' dimension
-    FacetSearchParams fsp = new FacetSearchParams(new AssociationIntSumFacetRequest(new CategoryPath("tags"), 10));
-
-    FacetsAccumulator fa = new StandardFacetsAccumulator(fsp, indexReader, taxoReader);
-    FacetsCollector fc = FacetsCollector.create(fa);
-
-    // MatchAllDocsQuery is for "browsing" (counts facets
-    // for all non-deleted docs in the index); normally
-    // you'd use a "normal" query, and use MultiCollector to
-    // wrap collecting the "normal" hits and also facets:
-    searcher.search(new MatchAllDocsQuery(), fc);
-
-    // Retrieve results
-    List<FacetResult> facetResults = fc.getFacetResults();
-    
-    indexReader.close();
-    taxoReader.close();
     
-    return facetResults;
-  }
+    CategoryPath tags = new CategoryPath("tags");
+    CategoryPath genre = new CategoryPath("genre");
+    FacetSearchParams fsp = new FacetSearchParams(
+        new AssociationIntSumFacetRequest(tags, 10), 
+        new AssociationFloatSumFacetRequest(genre, 10));
   
-  /** User runs a query and aggregates facets by summing their float associations. */
-  private List<FacetResult> sumFloatAssociations() throws IOException {
-    DirectoryReader indexReader = DirectoryReader.open(indexDir);
-    IndexSearcher searcher = new IndexSearcher(indexReader);
-    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
-    
-    // sum the 'tags' dimension
-    FacetSearchParams fsp = new FacetSearchParams(new AssociationFloatSumFacetRequest(new CategoryPath("genre"), 10));
-    
-    FacetsAccumulator fa = new StandardFacetsAccumulator(fsp, indexReader, taxoReader);
+    // every category has a different type of association, so use chain their
+    // respective aggregators.
+    final Map<CategoryPath,FacetsAggregator> aggregators = new HashMap<CategoryPath,FacetsAggregator>();
+    aggregators.put(tags, new SumIntAssociationFacetsAggregator());
+    aggregators.put(genre, new SumFloatAssociationFacetsAggregator());
+    FacetsAccumulator fa = new FacetsAccumulator(fsp, indexReader, taxoReader) {
+      @Override
+      public FacetsAggregator getAggregator() {
+        return new MultiAssociationsFacetsAggregator(aggregators);
+      }
+    };
     FacetsCollector fc = FacetsCollector.create(fa);
     
     // MatchAllDocsQuery is for "browsing" (counts facets
@@ -176,31 +167,17 @@ public class AssociationsFacetsExample {
     return facetResults;
   }
   
-  /** Runs summing int association example. */
-  public List<FacetResult> runSumIntAssociations() throws IOException {
+  /** Runs summing association example. */
+  public List<FacetResult> runSumAssociations() throws IOException {
     index();
-    return sumIntAssociations();
+    return sumAssociations();
   }
   
-  /** Runs summing float association example. */
-  public List<FacetResult> runSumFloatAssociations() throws IOException {
-    index();
-    return sumFloatAssociations();
-  }
-
   /** Runs the sum int/float associations examples and prints the results. */
   public static void main(String[] args) throws Exception {
-    System.out.println("Sum int-associations example:");
-    System.out.println("-----------------------------");
-    List<FacetResult> results = new AssociationsFacetsExample().runSumIntAssociations();
-    for (FacetResult res : results) {
-      System.out.println(res);
-    }
-
-    System.out.println("\n");
-    System.out.println("Sum float-associations example:");
-    System.out.println("-------------------------------");
-    results = new AssociationsFacetsExample().runSumFloatAssociations();
+    System.out.println("Sum associations example:");
+    System.out.println("-------------------------");
+    List<FacetResult> results = new AssociationsFacetsExample().runSumAssociations();
     for (FacetResult res : results) {
       System.out.println(res);
     }

Modified: lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/facet/TestAssociationsFacetsExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/facet/TestAssociationsFacetsExample.java?rev=1445108&r1=1445107&r2=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/facet/TestAssociationsFacetsExample.java (original)
+++ lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/facet/TestAssociationsFacetsExample.java Tue Feb 12 11:39:23 2013
@@ -31,19 +31,24 @@ public class TestAssociationsFacetsExamp
 
   @Test
   public void testExamples() throws Exception {
-    assertExampleResult(new AssociationsFacetsExample().runSumIntAssociations(), EXPECTED_INT_SUM_RESULTS);
-    assertExampleResult(new AssociationsFacetsExample().runSumFloatAssociations(), EXPECTED_FLOAT_SUM_RESULTS);
-  }
-
-  private void assertExampleResult(List<FacetResult> res, double[] expectedResults) {
-    assertNotNull("Null result!", res);
-    assertEquals("Wrong number of results!", 1, res.size());
-    assertEquals("Wrong number of facets!", 2, res.get(0).getNumValidDescendants());
+    List<FacetResult> res = new AssociationsFacetsExample().runSumAssociations();
+   
+    assertEquals("Wrong number of results", 2, res.size());
+    
+    for (FacetResult fres : res) {
+      assertEquals("Wrong number of facets", 2, fres.getNumValidDescendants());
+    }
     
     Iterable<? extends FacetResultNode> it = res.get(0).getFacetResultNode().subResults;
     int i = 0;
     for (FacetResultNode fResNode : it) {
-      assertEquals("Wrong result for facet "+fResNode.label, expectedResults[i++], fResNode.value, 1E-5);
+      assertEquals("Wrong result for facet " + fResNode.label, EXPECTED_INT_SUM_RESULTS[i++], fResNode.value, 1E-5);
+    }
+    
+    it = res.get(1).getFacetResultNode().subResults;
+    i = 0;
+    for (FacetResultNode fResNode : it) {
+      assertEquals("Wrong result for facet " + fResNode.label, EXPECTED_FLOAT_SUM_RESULTS[i++], fResNode.value, 1E-5);
     }
   }
   

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationFloatSumFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationFloatSumFacetRequest.java?rev=1445108&r1=1445107&r2=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationFloatSumFacetRequest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationFloatSumFacetRequest.java Tue Feb 12 11:39:23 2013
@@ -1,12 +1,8 @@
 package org.apache.lucene.facet.associations;
 
-import java.io.IOException;
-
-import org.apache.lucene.facet.search.Aggregator;
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.FacetRequest;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -27,10 +23,7 @@ import org.apache.lucene.facet.taxonomy.
 
 /**
  * A {@link FacetRequest} for weighting facets according to their float
- * association by summing the association values. Note that this class caches
- * the associations data in-memory by default. You can override
- * {@link #createAggregator(boolean, FacetArrays, TaxonomyReader)} to return an
- * {@link AssociationFloatSumAggregator} which does otherwise.
+ * association by summing the association values.
  * 
  * @lucene.experimental
  */
@@ -45,13 +38,6 @@ public class AssociationFloatSumFacetReq
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
-      throws IOException {
-    assert !useComplements : "complements are not supported by this FacetRequest";
-    return new AssociationFloatSumAggregator(arrays.getFloatArray());
-  }
-
-  @Override
   public double getValueOf(FacetArrays arrays, int ordinal) {
     return arrays.getFloatArray()[ordinal];
   }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetRequest.java?rev=1445108&r1=1445107&r2=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetRequest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetRequest.java Tue Feb 12 11:39:23 2013
@@ -1,12 +1,8 @@
 package org.apache.lucene.facet.associations;
 
-import java.io.IOException;
-
-import org.apache.lucene.facet.search.Aggregator;
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.FacetRequest;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -27,10 +23,7 @@ import org.apache.lucene.facet.taxonomy.
 
 /**
  * A {@link FacetRequest} for weighting facets according to their integer
- * association by summing the association values. Note that this class caches
- * the associations data in-memory by default. You can override
- * {@link #createAggregator(boolean, FacetArrays, TaxonomyReader)} to return an
- * {@link AssociationFloatSumAggregator} which does otherwise.
+ * association by summing the association values.
  * 
  * @lucene.experimental
  */
@@ -45,13 +38,6 @@ public class AssociationIntSumFacetReque
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
-      throws IOException {
-    assert !useComplements : "complements are not supported by this FacetRequest";
-    return new AssociationIntSumAggregator(arrays.getIntArray());
-  }
-
-  @Override
   public FacetArraysSource getFacetArraysSource() {
     return FacetArraysSource.INT;
   }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java?rev=1445108&r1=1445107&r2=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java Tue Feb 12 11:39:23 2013
@@ -7,7 +7,6 @@ import java.util.Map;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.facet.index.CountingListBuilder;
 import org.apache.lucene.facet.index.DrillDownStream;
 import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.CategoryListParams;
@@ -100,12 +99,7 @@ public class AssociationsFacetFields ext
   protected Map<String,BytesRef> getCategoryListData(CategoryListParams categoryListParams, IntsRef ordinals,
       Iterable<CategoryPath> categories) throws IOException {
     AssociationsListBuilder associations = new AssociationsListBuilder((CategoryAssociationsContainer) categories);
-    CountingListBuilder counting = new CountingListBuilder(categoryListParams, indexingParams, taxonomyWriter);
-    // CountingListBuilder modifies the ordinals array, by e.g. adding parent ordinals, sorting etc.
-    // Therefore first build the associations list and only afterwards the counting list.
-    final Map<String,BytesRef> res = associations.build(ordinals, categories);
-    res.putAll(counting.build(ordinals, categories));
-    return res;
+    return associations.build(ordinals, categories);
   }
   
   @Override

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java?rev=1445108&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java Tue Feb 12 11:39:23 2013
@@ -0,0 +1,92 @@
+package org.apache.lucene.facet.associations;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.facet.params.CategoryListParams;
+import org.apache.lucene.facet.search.FacetArrays;
+import org.apache.lucene.facet.search.FacetRequest;
+import org.apache.lucene.facet.search.FacetsAggregator;
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+
+/*
+ * 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 {@link FacetsAggregator} which chains multiple aggregators for aggregating
+ * the association values of categories that belong to the same category list.
+ * While nothing prevents you from chaining general purpose aggregators, it is
+ * only useful for aggregating association values, as each association type is
+ * written in its own list.
+ * 
+ * @lucene.experimental
+ */
+public class MultiAssociationsFacetsAggregator implements FacetsAggregator {
+  
+  private final Map<CategoryPath,FacetsAggregator> categoryAggregators;
+  private final List<FacetsAggregator> aggregators;
+  
+  /**
+   * Creates a new {@link MultiAssociationsFacetsAggregator} over the given
+   * aggregators. The mapping is used by
+   * {@link #rollupValues(FacetRequest, int, int[], int[], FacetArrays)} to
+   * rollup the values of the speicfic category by the corresponding
+   * {@link FacetsAggregator}. However, since each {@link FacetsAggregator}
+   * handles the associations of a specific type, which could cover multiple
+   * categories, the aggregation is done on the unique set of aggregators, which
+   * are identified by their class.
+   */
+  public MultiAssociationsFacetsAggregator(Map<CategoryPath,FacetsAggregator> aggregators) {
+    this.categoryAggregators = aggregators;
+    
+    // make sure that each FacetsAggregator class is invoked only once, or
+    // otherwise categories may be aggregated multiple times.
+    Map<Class<? extends FacetsAggregator>, FacetsAggregator> aggsClasses = 
+        new HashMap<Class<? extends FacetsAggregator>,FacetsAggregator>();
+    for (FacetsAggregator fa : aggregators.values()) {
+      aggsClasses.put(fa.getClass(), fa);
+    }
+    this.aggregators = new ArrayList<FacetsAggregator>(aggsClasses.values());
+  }
+  
+  @Override
+  public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp, FacetArrays facetArrays) throws IOException {
+    for (FacetsAggregator fa : aggregators) {
+      fa.aggregate(matchingDocs, clp, facetArrays);
+    }
+  }
+  
+  @Override
+  public void rollupValues(FacetRequest fr, int ordinal, int[] children, int[] siblings, FacetArrays facetArrays) {
+    categoryAggregators.get(fr.categoryPath).rollupValues(fr, ordinal, children, siblings, facetArrays);
+  }
+  
+  @Override
+  public boolean requiresDocScores() {
+    for (FacetsAggregator fa : aggregators) {
+      if (fa.requiresDocScores()) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+}

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumFloatAssociationFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumFloatAssociationFacetsAggregator.java?rev=1445108&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumFloatAssociationFacetsAggregator.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumFloatAssociationFacetsAggregator.java Tue Feb 12 11:39:23 2013
@@ -0,0 +1,98 @@
+package org.apache.lucene.facet.associations;
+
+import java.io.IOException;
+
+import org.apache.lucene.facet.params.CategoryListParams;
+import org.apache.lucene.facet.search.FacetArrays;
+import org.apache.lucene.facet.search.FacetRequest;
+import org.apache.lucene.facet.search.FacetsAggregator;
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.util.BytesRef;
+
+/*
+ * 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 {@link FacetsAggregator} which computes the weight of a category as the sum
+ * of the float values associated with it in the result documents. Assumes that
+ * the association encoded for each ordinal is {@link CategoryFloatAssociation}.
+ * 
+ * @lucene.experimental
+ */
+public class SumFloatAssociationFacetsAggregator implements FacetsAggregator {
+
+  private final BytesRef bytes = new BytesRef(32);
+  
+  @Override
+  public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp, FacetArrays facetArrays) throws IOException {
+    BinaryDocValues dv = matchingDocs.context.reader().getBinaryDocValues(clp.field + CategoryFloatAssociation.ASSOCIATION_LIST_ID);
+    if (dv == null) {
+      return; // no float associations in this reader
+    }
+    
+    final int length = matchingDocs.bits.length();
+    final float[] values = facetArrays.getFloatArray();
+    int doc = 0;
+    while (doc < length && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
+      dv.get(doc, bytes);
+      if (bytes.length == 0) {
+        continue; // no associations for this document
+      }
+
+      // aggreate float association values for ordinals
+      int bytesUpto = bytes.offset + bytes.length;
+      int pos = bytes.offset;
+      while (pos < bytesUpto) {
+        int ordinal = ((bytes.bytes[pos++] & 0xFF) << 24) | ((bytes.bytes[pos++] & 0xFF) << 16)
+            | ((bytes.bytes[pos++] & 0xFF) <<  8) | (bytes.bytes[pos++] & 0xFF);
+        
+        int value = ((bytes.bytes[pos++] & 0xFF) << 24) | ((bytes.bytes[pos++] & 0xFF) << 16)
+            | ((bytes.bytes[pos++] & 0xFF) <<  8) | (bytes.bytes[pos++] & 0xFF);
+
+        values[ordinal] += Float.intBitsToFloat(value);
+      }
+      
+      ++doc;
+    }
+  }
+
+  @Override
+  public boolean requiresDocScores() {
+    return false;
+  }
+
+  private float rollupValues(int ordinal, int[] children, int[] siblings, float[] scores) {
+    float Value = 0f;
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      float childValue = scores[ordinal];
+      childValue += rollupValues(children[ordinal], children, siblings, scores);
+      scores[ordinal] = childValue;
+      Value += childValue;
+      ordinal = siblings[ordinal];
+    }
+    return Value;
+  }
+
+  @Override
+  public void rollupValues(FacetRequest fr, int ordinal, int[] children, int[] siblings, FacetArrays facetArrays) {
+    float[] values = facetArrays.getFloatArray();
+    values[ordinal] += rollupValues(children[ordinal], children, siblings, values);
+  }
+  
+}

Copied: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumIntAssociationFacetsAggregator.java (from r1444520, lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetsAggregator.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumIntAssociationFacetsAggregator.java?p2=lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumIntAssociationFacetsAggregator.java&p1=lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetsAggregator.java&r1=1444520&r2=1445108&rev=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationIntSumFacetsAggregator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/SumIntAssociationFacetsAggregator.java Tue Feb 12 11:39:23 2013
@@ -7,6 +7,9 @@ import org.apache.lucene.facet.search.Fa
 import org.apache.lucene.facet.search.FacetRequest;
 import org.apache.lucene.facet.search.FacetsAggregator;
 import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.util.BytesRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -27,21 +30,67 @@ import org.apache.lucene.facet.search.Fa
 
 /**
  * A {@link FacetsAggregator} which computes the weight of a category as the sum
- * of the integer values associated with it in the result documents.
+ * of the integer values associated with it in the result documents. Assumes that
+ * the association encoded for each ordinal is {@link CategoryIntAssociation}.
  */
-public class AssociationIntSumFacetsAggregator implements FacetsAggregator {
-  
-  @Override
-  public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp,
-      FacetArrays facetArrays) throws IOException {}
+public class SumIntAssociationFacetsAggregator implements FacetsAggregator {
+
+  private final BytesRef bytes = new BytesRef(32);
   
   @Override
-  public void rollupValues(FacetRequest fr, int ordinal, int[] children,
-      int[] siblings, FacetArrays facetArrays) {}
-  
+  public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp, FacetArrays facetArrays) throws IOException {
+    BinaryDocValues dv = matchingDocs.context.reader().getBinaryDocValues(clp.field + CategoryIntAssociation.ASSOCIATION_LIST_ID);
+    if (dv == null) {
+      return; // no int associations in this reader
+    }
+    
+    final int length = matchingDocs.bits.length();
+    final int[] values = facetArrays.getIntArray();
+    int doc = 0;
+    while (doc < length && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
+      dv.get(doc, bytes);
+      if (bytes.length == 0) {
+        continue; // no associations for this document
+      }
+
+      // aggreate association values for ordinals
+      int bytesUpto = bytes.offset + bytes.length;
+      int pos = bytes.offset;
+      while (pos < bytesUpto) {
+        int ordinal = ((bytes.bytes[pos++] & 0xFF) << 24) | ((bytes.bytes[pos++] & 0xFF) << 16)
+            | ((bytes.bytes[pos++] & 0xFF) <<  8) | (bytes.bytes[pos++] & 0xFF);
+        
+        int value = ((bytes.bytes[pos++] & 0xFF) << 24) | ((bytes.bytes[pos++] & 0xFF) << 16)
+            | ((bytes.bytes[pos++] & 0xFF) <<  8) | (bytes.bytes[pos++] & 0xFF);
+
+        values[ordinal] += value;
+      }
+      
+      ++doc;
+    }
+  }
+
   @Override
   public boolean requiresDocScores() {
     return false;
   }
-  
+
+  private float rollupValues(int ordinal, int[] children, int[] siblings, float[] scores) {
+    float Value = 0f;
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      float childValue = scores[ordinal];
+      childValue += rollupValues(children[ordinal], children, siblings, scores);
+      scores[ordinal] = childValue;
+      Value += childValue;
+      ordinal = siblings[ordinal];
+    }
+    return Value;
+  }
+
+  @Override
+  public void rollupValues(FacetRequest fr, int ordinal, int[] children, int[] siblings, FacetArrays facetArrays) {
+    float[] values = facetArrays.getFloatArray();
+    values[ordinal] += rollupValues(children[ordinal], children, siblings, values);
+  }
+
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java?rev=1445108&r1=1445107&r2=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java Tue Feb 12 11:39:23 2013
@@ -36,7 +36,7 @@ import org.apache.lucene.facet.taxonomy.
  * @lucene.experimental
  */
 public abstract class FacetRequest {
-
+  
   /**
    * Result structure manner of applying request's limits such as
    * {@link FacetRequest#getNumLabel()} and {@link FacetRequest#numResults}.
@@ -45,7 +45,7 @@ public abstract class FacetRequest {
   public enum ResultMode { 
     /** Limits are applied per node, and the result has a full tree structure. */
     PER_NODE_IN_TREE, 
-
+    
     /** Limits are applied globally, on total number of results, and the result has a flat structure. */
     GLOBAL_FLAT
   }
@@ -62,10 +62,10 @@ public abstract class FacetRequest {
    * {@link FacetResultsHandler}.
    */
   public enum FacetArraysSource { INT, FLOAT, BOTH }
-
+  
   /** Requested sort order for the results. */
   public enum SortOrder { ASCENDING, DESCENDING }
-
+  
   /**
    * Default depth for facets accumulation.
    * @see #getDepth()
@@ -84,15 +84,15 @@ public abstract class FacetRequest {
   private int numLabel;
   private int depth;
   private SortOrder sortOrder;
-
+  
   /**
    * Computed at construction, this hashCode is based on two final members
    * {@link CategoryPath} and <code>numResults</code>
    */
   private final int hashCode;
-
+  
   private ResultMode resultMode = DEFAULT_RESULT_MODE;
-
+  
   /**
    * Initialize the request with a given path, and a requested number of facets
    * results. By default, all returned results would be labeled - to alter this
@@ -123,7 +123,7 @@ public abstract class FacetRequest {
     
     hashCode = categoryPath.hashCode() ^ this.numResults;
   }
-
+  
   /**
    * Create an aggregator for this facet request. Aggregator action depends on
    * request definition. For a count request, it will usually increment the
@@ -138,9 +138,12 @@ public abstract class FacetRequest {
    *          reader of taxonomy in effect.
    * @throws IOException If there is a low-level I/O error.
    */
-  public abstract Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
-      throws IOException;
-
+  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
+      throws IOException {
+    throw new UnsupportedOperationException("this FacetRequest does not support this type of Aggregator anymore; " +
+        "you should override FacetsAccumulator to return the proper FacetsAggregator");
+  }
+  
   @Override
   public boolean equals(Object o) {
     if (o instanceof FacetRequest) {
@@ -154,7 +157,7 @@ public abstract class FacetRequest {
     }
     return false;
   }
-
+  
   /**
    * How deeply to look under the given category. If the depth is 0,
    * only the category itself is counted. If the depth is 1, its immediate
@@ -198,17 +201,17 @@ public abstract class FacetRequest {
   public final int getNumLabel() {
     return numLabel;
   }
-
+  
   /** Return the requested result mode. */
   public final ResultMode getResultMode() {
     return resultMode;
   }
-
+  
   /** Return the requested order of results. */
   public final SortOrder getSortOrder() {
     return sortOrder;
   }
-
+  
   /**
    * Return the value of a category used for facets computations for this
    * request. For a count request this would be the count for that facet, i.e.
@@ -232,16 +235,16 @@ public abstract class FacetRequest {
   // That, together with getFacetArraysSource should allow ResultHandlers to
   // efficiently obtain the values from the arrays directly
   public abstract double getValueOf(FacetArrays arrays, int idx);
-
+  
   @Override
   public int hashCode() {
     return hashCode; 
   }
-
+  
   public void setDepth(int depth) {
     this.depth = depth;
   }
-
+  
   public void setNumLabel(int numLabel) {
     this.numLabel = numLabel;
   }
@@ -253,7 +256,7 @@ public abstract class FacetRequest {
   public void setResultMode(ResultMode resultMode) {
     this.resultMode = resultMode;
   }
-
+  
   public void setSortOrder(SortOrder sortOrder) {
     this.sortOrder = sortOrder;
   }
@@ -262,6 +265,5 @@ public abstract class FacetRequest {
   public String toString() {
     return categoryPath.toString()+" nRes="+numResults+" nLbl="+numLabel;
   }
-
+  
 }
-  
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java?rev=1445108&r1=1445107&r2=1445108&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java Tue Feb 12 11:39:23 2013
@@ -1,19 +1,17 @@
 package org.apache.lucene.facet.associations;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 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.associations.AssociationFloatSumFacetRequest;
-import org.apache.lucene.facet.associations.AssociationIntSumFacetRequest;
-import org.apache.lucene.facet.associations.AssociationsFacetFields;
-import org.apache.lucene.facet.associations.CategoryAssociationsContainer;
-import org.apache.lucene.facet.associations.CategoryFloatAssociation;
-import org.apache.lucene.facet.associations.CategoryIntAssociation;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.FacetResult;
+import org.apache.lucene.facet.search.FacetsAccumulator;
+import org.apache.lucene.facet.search.FacetsAggregator;
 import org.apache.lucene.facet.search.FacetsCollector;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
@@ -48,7 +46,7 @@ import org.junit.Test;
 
 /** Test for associations */
 public class AssociationsFacetRequestTest extends FacetTestCase {
-
+  
   private static Directory dir;
   private static IndexReader reader;
   private static Directory taxoDir;
@@ -102,15 +100,22 @@ public class AssociationsFacetRequestTes
   @Test
   public void testIntSumAssociation() throws Exception {
     DirectoryTaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
-
+    
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
         new AssociationIntSumFacetRequest(aint, 10),
         new AssociationIntSumFacetRequest(bint, 10));
     
     Query q = new MatchAllDocsQuery();
-
-    FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
+    
+    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
+      @Override
+      public FacetsAggregator getAggregator() {
+        return new SumIntAssociationFacetsAggregator();
+      }
+    };
+    
+    FacetsCollector fc = FacetsCollector.create(fa);
     
     IndexSearcher searcher = newSearcher(reader);
     searcher.search(q, fc);
@@ -127,35 +132,39 @@ public class AssociationsFacetRequestTes
   @Test
   public void testFloatSumAssociation() throws Exception {
     DirectoryTaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
-
+    
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
         new AssociationFloatSumFacetRequest(afloat, 10),
         new AssociationFloatSumFacetRequest(bfloat, 10));
     
     Query q = new MatchAllDocsQuery();
-
-    FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
+    
+    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
+      @Override
+      public FacetsAggregator getAggregator() {
+        return new SumFloatAssociationFacetsAggregator();
+      }
+    };
+    
+    FacetsCollector fc = FacetsCollector.create(fa);
     
     IndexSearcher searcher = newSearcher(reader);
     searcher.search(q, fc);
     List<FacetResult> res = fc.getFacetResults();
     
     assertNotNull("No results!",res);
-    assertEquals("Wrong number of results!",2, res.size());
+    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 {
-    // Same category list cannot be aggregated by two different aggregators. If
-    // you want to do that, you need to separate the categories into two
-    // category list (you'll still have one association list).
     DirectoryTaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
-
+    
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
         new AssociationIntSumFacetRequest(aint, 10),
@@ -164,18 +173,33 @@ public class AssociationsFacetRequestTes
         new AssociationFloatSumFacetRequest(bfloat, 10));
     
     Query q = new MatchAllDocsQuery();
-
-    FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
+    
+    final SumIntAssociationFacetsAggregator sumInt = new SumIntAssociationFacetsAggregator();
+    final SumFloatAssociationFacetsAggregator sumFloat = new SumFloatAssociationFacetsAggregator();
+    final Map<CategoryPath,FacetsAggregator> aggregators = new HashMap<CategoryPath,FacetsAggregator>();
+    aggregators.put(aint, sumInt);
+    aggregators.put(bint, sumInt);
+    aggregators.put(afloat, sumFloat);
+    aggregators.put(bfloat, sumFloat);
+    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
+      @Override
+      public FacetsAggregator getAggregator() {
+        return new MultiAssociationsFacetsAggregator(aggregators);
+      }
+    };
+    FacetsCollector fc = FacetsCollector.create(fa);
     
     IndexSearcher searcher = newSearcher(reader);
     searcher.search(q, fc);
-    try {
-      fc.getFacetResults();
-      fail("different aggregators for same category list should not be supported");
-    } catch (RuntimeException e) {
-      // ok - expected
-    }
+    List<FacetResult> 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();
   }  
-
+  
 }