You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gs...@apache.org on 2022/04/07 14:48:14 UTC

[lucene] branch branch_9x updated: LUCENE-10444: Support alternate aggregation functions in association facets (#719)

This is an automated email from the ASF dual-hosted git repository.

gsmiller pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 9e10ba02ec3 LUCENE-10444: Support alternate aggregation functions in association facets (#719)
9e10ba02ec3 is described below

commit 9e10ba02ec350f267458926035bb172ea82291b9
Author: Greg Miller <gs...@gmail.com>
AuthorDate: Thu Apr 7 07:48:08 2022 -0700

    LUCENE-10444: Support alternate aggregation functions in association facets (#719)
---
 lucene/CHANGES.txt                                 |   4 +-
 .../demo/facet/AssociationsFacetsExample.java      |  17 +-
 .../facet/ExpressionAggregationFacetsExample.java  |  11 +-
 .../java/org/apache/lucene/facet/package-info.java |   5 +-
 .../taxonomy/AssociationAggregationFunction.java   |  66 ++++
 .../facet/taxonomy/FastTaxonomyFacetCounts.java    |   4 +-
 .../facet/taxonomy/FloatAssociationFacetField.java |   2 +-
 .../lucene/facet/taxonomy/FloatTaxonomyFacets.java |  45 ++-
 .../facet/taxonomy/IntAssociationFacetField.java   |   2 +-
 .../lucene/facet/taxonomy/IntTaxonomyFacets.java   |  80 +++--
 .../lucene/facet/taxonomy/TaxonomyFacetCounts.java |   7 +-
 .../taxonomy/TaxonomyFacetFloatAssociations.java   | 244 ++++++++++++++
 .../taxonomy/TaxonomyFacetIntAssociations.java     |  87 +++++
 .../TaxonomyFacetSumFloatAssociations.java         |  49 +--
 .../taxonomy/TaxonomyFacetSumIntAssociations.java  |  49 +--
 .../taxonomy/TaxonomyFacetSumValueSource.java      |  98 +-----
 .../taxonomy/TestTaxonomyFacetAssociations.java    | 240 ++++++++++++-
 ...urce.java => TestTaxonomyFacetValueSource.java} | 372 ++++++++++++++++-----
 18 files changed, 1069 insertions(+), 313 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b38e03bbe4b..832d7a5dc80 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -17,12 +17,14 @@ New Features
   to speed up computing the number of hits when possible. (Lu Xugang, Luca Cavanna, Adrien Grand)
 
 * LUCENE-10422: Monitor Improvements: `Monitor` can use a custom `Directory` 
-  implementation. `Monitor` can be created with a readonly `QueryIndex` in order to 
+  implementation. `Monitor` can be created with a readonly `QueryIndex` in order to
   have readonly `Monitor` instances. (Niko Usai)
 
 * LUCENE-10456: Implement rewrite and Weight#count for MultiRangeQuery
   by merging overlapping ranges . (Jianping Weng)
 
+* LUCENE-10444: Support alternate aggregation functions in association facets. (Greg Miller)
+
 Improvements
 ---------------------
 
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java
index f1fcc5dd3b4..9d7bfb646fc 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/AssociationsFacetsExample.java
@@ -26,10 +26,11 @@ import org.apache.lucene.facet.FacetResult;
 import org.apache.lucene.facet.Facets;
 import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.taxonomy.AssociationAggregationFunction;
 import org.apache.lucene.facet.taxonomy.FloatAssociationFacetField;
 import org.apache.lucene.facet.taxonomy.IntAssociationFacetField;
-import org.apache.lucene.facet.taxonomy.TaxonomyFacetSumFloatAssociations;
-import org.apache.lucene.facet.taxonomy.TaxonomyFacetSumIntAssociations;
+import org.apache.lucene.facet.taxonomy.TaxonomyFacetFloatAssociations;
+import org.apache.lucene.facet.taxonomy.TaxonomyFacetIntAssociations;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
@@ -102,8 +103,12 @@ public class AssociationsFacetsExample {
     // you'd use a "normal" query:
     FacetsCollector.search(searcher, new MatchAllDocsQuery(), 10, fc);
 
-    Facets tags = new TaxonomyFacetSumIntAssociations("$tags", taxoReader, config, fc);
-    Facets genre = new TaxonomyFacetSumFloatAssociations("$genre", taxoReader, config, fc);
+    Facets tags =
+        new TaxonomyFacetIntAssociations(
+            "$tags", taxoReader, config, fc, AssociationAggregationFunction.SUM);
+    Facets genre =
+        new TaxonomyFacetFloatAssociations(
+            "$genre", taxoReader, config, fc, AssociationAggregationFunction.SUM);
 
     // Retrieve results
     List<FacetResult> results = new ArrayList<>();
@@ -132,7 +137,9 @@ public class AssociationsFacetsExample {
     FacetsCollector.search(searcher, q, 10, fc);
 
     // Retrieve results
-    Facets facets = new TaxonomyFacetSumFloatAssociations("$genre", taxoReader, config, fc);
+    Facets facets =
+        new TaxonomyFacetFloatAssociations(
+            "$genre", taxoReader, config, fc, AssociationAggregationFunction.SUM);
     FacetResult result = facets.getTopChildren(10, "genre");
 
     indexReader.close();
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
index b86c3357129..40a16b36bff 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
@@ -31,7 +31,8 @@ import org.apache.lucene.facet.FacetResult;
 import org.apache.lucene.facet.Facets;
 import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.facet.taxonomy.TaxonomyFacetSumValueSource;
+import org.apache.lucene.facet.taxonomy.AssociationAggregationFunction;
+import org.apache.lucene.facet.taxonomy.TaxonomyFacetFloatAssociations;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
@@ -105,8 +106,12 @@ public class ExpressionAggregationFacetsExample {
 
     // Retrieve results
     Facets facets =
-        new TaxonomyFacetSumValueSource(
-            taxoReader, config, fc, expr.getDoubleValuesSource(bindings));
+        new TaxonomyFacetFloatAssociations(
+            taxoReader,
+            config,
+            fc,
+            AssociationAggregationFunction.SUM,
+            expr.getDoubleValuesSource(bindings));
     FacetResult result = facets.getTopChildren(10, "A");
 
     indexReader.close();
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
index a0c64aecc5c..dd0029dab74 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
@@ -25,9 +25,8 @@
  *       global int ordinals for fast counting at search time; these methods can compute counts
  *       (({@link org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts}, {@link
  *       org.apache.lucene.facet.taxonomy.TaxonomyFacetCounts}) aggregate long or double values
- *       {@link org.apache.lucene.facet.taxonomy.TaxonomyFacetSumIntAssociations}, {@link
- *       org.apache.lucene.facet.taxonomy.TaxonomyFacetSumFloatAssociations}, {@link
- *       org.apache.lucene.facet.taxonomy.TaxonomyFacetSumValueSource}. Add {@link
+ *       {@link org.apache.lucene.facet.taxonomy.TaxonomyFacetIntAssociations}, {@link
+ *       org.apache.lucene.facet.taxonomy.TaxonomyFacetFloatAssociations}. Add {@link
  *       org.apache.lucene.facet.FacetField} or {@link
  *       org.apache.lucene.facet.taxonomy.AssociationFacetField} to your documents at index time to
  *       use taxonomy-based methods.
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/AssociationAggregationFunction.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/AssociationAggregationFunction.java
new file mode 100644
index 00000000000..8b0c8c75d6b
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/AssociationAggregationFunction.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.facet.taxonomy;
+
+/**
+ * Specify aggregation logic used in {@link TaxonomyFacetIntAssociations} and {@link
+ * TaxonomyFacetFloatAssociations}.
+ */
+public abstract class AssociationAggregationFunction {
+
+  // TODO: Would be nice to add support for MIN as well here, but there are a number of places
+  // in our facet implementations where we attribute special meaning to 0 and assume that valid
+  // values are always positive. I think we'd want to break that assumption for MIN to really
+  // make sense.
+
+  /** Sole constructor. */
+  protected AssociationAggregationFunction() {}
+
+  /** Implement aggregation logic for integers */
+  public abstract int aggregate(int existingVal, int newVal);
+
+  /** Implement aggregation logic for floats */
+  public abstract float aggregate(float existingVal, float newVal);
+
+  /** Aggregation that computes the maximum value */
+  public static final AssociationAggregationFunction MAX =
+      new AssociationAggregationFunction() {
+        @Override
+        public int aggregate(int existingVal, int newVal) {
+          return Math.max(existingVal, newVal);
+        }
+
+        @Override
+        public float aggregate(float existingVal, float newVal) {
+          return Math.max(existingVal, newVal);
+        }
+      };
+
+  /** Aggregation that computes the sum */
+  public static final AssociationAggregationFunction SUM =
+      new AssociationAggregationFunction() {
+        @Override
+        public int aggregate(int existingVal, int newVal) {
+          return existingVal + newVal;
+        }
+
+        @Override
+        public float aggregate(float existingVal, float newVal) {
+          return existingVal + newVal;
+        }
+      };
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FastTaxonomyFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FastTaxonomyFacetCounts.java
index 2e496921da7..1adb3993333 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FastTaxonomyFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FastTaxonomyFacetCounts.java
@@ -54,7 +54,7 @@ public class FastTaxonomyFacetCounts extends IntTaxonomyFacets {
   public FastTaxonomyFacetCounts(
       String indexFieldName, TaxonomyReader taxoReader, FacetsConfig config, FacetsCollector fc)
       throws IOException {
-    super(indexFieldName, taxoReader, config, fc);
+    super(indexFieldName, taxoReader, config, AssociationAggregationFunction.SUM, fc);
     count(fc.getMatchingDocs());
   }
 
@@ -66,7 +66,7 @@ public class FastTaxonomyFacetCounts extends IntTaxonomyFacets {
   public FastTaxonomyFacetCounts(
       String indexFieldName, IndexReader reader, TaxonomyReader taxoReader, FacetsConfig config)
       throws IOException {
-    super(indexFieldName, taxoReader, config, null);
+    super(indexFieldName, taxoReader, config, AssociationAggregationFunction.SUM, null);
     countAll(reader);
   }
 
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FloatAssociationFacetField.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FloatAssociationFacetField.java
index 9c6ce6e9dfc..5947747b744 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FloatAssociationFacetField.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FloatAssociationFacetField.java
@@ -22,7 +22,7 @@ import org.apache.lucene.util.BytesRef;
 
 /**
  * Add an instance of this to your {@link Document} to add a facet label associated with a float.
- * Use {@link TaxonomyFacetSumFloatAssociations} to aggregate float values per facet label at search
+ * Use {@link TaxonomyFacetFloatAssociations} to aggregate float values per facet label at search
  * time.
  *
  * @lucene.experimental
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FloatTaxonomyFacets.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FloatTaxonomyFacets.java
index 8d874c684bd..6d481710927 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FloatTaxonomyFacets.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FloatTaxonomyFacets.java
@@ -38,13 +38,32 @@ public abstract class FloatTaxonomyFacets extends TaxonomyFacets {
 
   // TODO: also use native hash map for sparse collection, like IntTaxonomyFacets
 
+  /** Aggregation function used for combining values. */
+  protected final AssociationAggregationFunction aggregationFunction;
+
   /** Per-ordinal value. */
   protected final float[] values;
 
-  /** Sole constructor. */
+  /**
+   * Constructor that defaults the aggregation function to {@link
+   * AssociationAggregationFunction#SUM}.
+   */
   protected FloatTaxonomyFacets(
       String indexFieldName, TaxonomyReader taxoReader, FacetsConfig config) throws IOException {
     super(indexFieldName, taxoReader, config);
+    this.aggregationFunction = AssociationAggregationFunction.SUM;
+    values = new float[taxoReader.getSize()];
+  }
+
+  /** Constructor that uses the provided aggregation function. */
+  protected FloatTaxonomyFacets(
+      String indexFieldName,
+      TaxonomyReader taxoReader,
+      AssociationAggregationFunction aggregationFunction,
+      FacetsConfig config)
+      throws IOException {
+    super(indexFieldName, taxoReader, config);
+    this.aggregationFunction = aggregationFunction;
     values = new float[taxoReader.getSize()];
   }
 
@@ -58,7 +77,9 @@ public abstract class FloatTaxonomyFacets extends TaxonomyFacets {
       if (ft.hierarchical && ft.multiValued == false) {
         int dimRootOrd = taxoReader.getOrdinal(new FacetLabel(dim));
         assert dimRootOrd > 0;
-        values[dimRootOrd] += rollup(children[dimRootOrd]);
+        float newValue =
+            aggregationFunction.aggregate(values[dimRootOrd], rollup(children[dimRootOrd]));
+        values[dimRootOrd] = newValue;
       }
     }
   }
@@ -66,14 +87,14 @@ public abstract class FloatTaxonomyFacets extends TaxonomyFacets {
   private float rollup(int ord) throws IOException {
     int[] children = getChildren();
     int[] siblings = getSiblings();
-    float sum = 0;
+    float aggregationValue = 0f;
     while (ord != TaxonomyReader.INVALID_ORDINAL) {
-      float childValue = values[ord] + rollup(children[ord]);
+      float childValue = aggregationFunction.aggregate(values[ord], rollup(children[ord]));
       values[ord] = childValue;
-      sum += childValue;
+      aggregationValue = aggregationFunction.aggregate(aggregationValue, childValue);
       ord = siblings[ord];
     }
-    return sum;
+    return aggregationValue;
   }
 
   @Override
@@ -113,13 +134,13 @@ public abstract class FloatTaxonomyFacets extends TaxonomyFacets {
     int[] siblings = getSiblings();
 
     int ord = children[dimOrd];
-    float sumValues = 0;
+    float aggregatedValue = 0;
     int childCount = 0;
 
     TopOrdAndFloatQueue.OrdAndValue reuse = null;
     while (ord != TaxonomyReader.INVALID_ORDINAL) {
       if (values[ord] > 0) {
-        sumValues += values[ord];
+        aggregatedValue = aggregationFunction.aggregate(aggregatedValue, values[ord]);
         childCount++;
         if (values[ord] > bottomValue) {
           if (reuse == null) {
@@ -137,16 +158,16 @@ public abstract class FloatTaxonomyFacets extends TaxonomyFacets {
       ord = siblings[ord];
     }
 
-    if (sumValues == 0) {
+    if (aggregatedValue == 0) {
       return null;
     }
 
     if (dimConfig.multiValued) {
       if (dimConfig.requireDimCount) {
-        sumValues = values[dimOrd];
+        aggregatedValue = values[dimOrd];
       } else {
         // Our sum'd count is not correct, in general:
-        sumValues = -1;
+        aggregatedValue = -1;
       }
     } else {
       // Our sum'd dim count is accurate, so we keep it
@@ -167,6 +188,6 @@ public abstract class FloatTaxonomyFacets extends TaxonomyFacets {
       labelValues[i] = new LabelAndValue(bulkPath[i].components[cp.length], values[i]);
     }
 
-    return new FacetResult(dim, path, sumValues, labelValues, childCount);
+    return new FacetResult(dim, path, aggregatedValue, labelValues, childCount);
   }
 }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/IntAssociationFacetField.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/IntAssociationFacetField.java
index cfeaf97367e..698e3810c9f 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/IntAssociationFacetField.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/IntAssociationFacetField.java
@@ -23,7 +23,7 @@ import org.apache.lucene.util.BytesRef;
 
 /**
  * Add an instance of this to your {@link Document} to add a facet label associated with an int. Use
- * {@link TaxonomyFacetSumIntAssociations} to aggregate int values per facet label at search time.
+ * {@link TaxonomyFacetIntAssociations} to aggregate int values per facet label at search time.
  *
  * @lucene.experimental
  */
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/IntTaxonomyFacets.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/IntTaxonomyFacets.java
index 99757d76eb0..354d9c3aac3 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/IntTaxonomyFacets.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/IntTaxonomyFacets.java
@@ -33,13 +33,16 @@ import org.apache.lucene.facet.TopOrdAndIntQueue;
  *
  * @deprecated Visibility of this class will be reduced to pkg-private in a future version. This
  *     class is meant to host common code as an internal implementation detail to {@link
- *     FastTaxonomyFacetCounts} and {@link TaxonomyFacetSumIntAssociations},and is not intended as
- *     an extension point for user-created {@code Facets} implementations. If your code is relying
- *     on this, please migrate necessary functionality down into your own class.
+ *     FastTaxonomyFacetCounts} and {@link TaxonomyFacetIntAssociations},and is not intended as an
+ *     extension point for user-created {@code Facets} implementations. If your code is relying on
+ *     this, please migrate necessary functionality down into your own class.
  */
 @Deprecated
 public abstract class IntTaxonomyFacets extends TaxonomyFacets {
 
+  /** Aggregation function used for combining values. */
+  protected final AssociationAggregationFunction aggregationFunction;
+
   /**
    * Dense ordinal values.
    *
@@ -56,11 +59,35 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
    */
   protected final IntIntHashMap sparseValues;
 
-  /** Sole constructor. */
+  /**
+   * Constructor that defaults the aggregation function to {@link
+   * AssociationAggregationFunction#SUM}.
+   */
   protected IntTaxonomyFacets(
       String indexFieldName, TaxonomyReader taxoReader, FacetsConfig config, FacetsCollector fc)
       throws IOException {
     super(indexFieldName, taxoReader, config);
+    this.aggregationFunction = AssociationAggregationFunction.SUM;
+
+    if (useHashTable(fc, taxoReader)) {
+      sparseValues = new IntIntHashMap();
+      values = null;
+    } else {
+      sparseValues = null;
+      values = new int[taxoReader.getSize()];
+    }
+  }
+
+  /** Constructor that uses the provided aggregation function. */
+  protected IntTaxonomyFacets(
+      String indexFieldName,
+      TaxonomyReader taxoReader,
+      FacetsConfig config,
+      AssociationAggregationFunction aggregationFunction,
+      FacetsCollector fc)
+      throws IOException {
+    super(indexFieldName, taxoReader, config);
+    this.aggregationFunction = aggregationFunction;
 
     if (useHashTable(fc, taxoReader)) {
       sparseValues = new IntIntHashMap();
@@ -108,6 +135,15 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
     }
   }
 
+  /** Set the count for this ordinal to {@code newValue}. */
+  void setValue(int ordinal, int newValue) {
+    if (sparseValues != null) {
+      sparseValues.put(ordinal, newValue);
+    } else {
+      values[ordinal] = newValue;
+    }
+  }
+
   /** Get the count for this ordinal. */
   protected int getValue(int ordinal) {
     if (sparseValues != null) {
@@ -133,7 +169,9 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
             // lazy init
             children = getChildren();
           }
-          increment(dimRootOrd, rollup(children[dimRootOrd]));
+          int currentValue = getValue(dimRootOrd);
+          int newValue = aggregationFunction.aggregate(currentValue, rollup(children[dimRootOrd]));
+          setValue(dimRootOrd, newValue);
         }
       }
     }
@@ -142,13 +180,15 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
   private int rollup(int ord) throws IOException {
     int[] children = getChildren();
     int[] siblings = getSiblings();
-    int sum = 0;
+    int aggregatedValue = 0;
     while (ord != TaxonomyReader.INVALID_ORDINAL) {
-      increment(ord, rollup(children[ord]));
-      sum += getValue(ord);
+      int currentValue = getValue(ord);
+      int newValue = aggregationFunction.aggregate(currentValue, rollup(children[ord]));
+      setValue(ord, newValue);
+      aggregatedValue = aggregationFunction.aggregate(aggregatedValue, getValue(ord));
       ord = siblings[ord];
     }
-    return sum;
+    return aggregatedValue;
   }
 
   @Override
@@ -185,7 +225,7 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
 
     int bottomValue = 0;
 
-    int totValue = 0;
+    int aggregatedValue = 0;
     int childCount = 0;
 
     TopOrdAndIntQueue.OrdAndValue reuse = null;
@@ -195,17 +235,17 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
 
     if (sparseValues != null) {
       for (IntIntCursor c : sparseValues) {
-        int count = c.value;
+        int value = c.value;
         int ord = c.key;
-        if (parents[ord] == dimOrd && count > 0) {
-          totValue += count;
+        if (parents[ord] == dimOrd && value > 0) {
+          aggregatedValue = aggregationFunction.aggregate(aggregatedValue, value);
           childCount++;
-          if (count > bottomValue) {
+          if (value > bottomValue) {
             if (reuse == null) {
               reuse = new TopOrdAndIntQueue.OrdAndValue();
             }
             reuse.ord = ord;
-            reuse.value = count;
+            reuse.value = value;
             reuse = q.insertWithOverflow(reuse);
             if (q.size() == topN) {
               bottomValue = q.top().value;
@@ -220,7 +260,7 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
       while (ord != TaxonomyReader.INVALID_ORDINAL) {
         int value = values[ord];
         if (value > 0) {
-          totValue += value;
+          aggregatedValue = aggregationFunction.aggregate(aggregatedValue, value);
           childCount++;
           if (value > bottomValue) {
             if (reuse == null) {
@@ -239,16 +279,16 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
       }
     }
 
-    if (totValue == 0) {
+    if (aggregatedValue == 0) {
       return null;
     }
 
     if (dimConfig.multiValued) {
       if (dimConfig.requireDimCount) {
-        totValue = getValue(dimOrd);
+        aggregatedValue = getValue(dimOrd);
       } else {
         // Our sum'd value is not correct, in general:
-        totValue = -1;
+        aggregatedValue = -1;
       }
     } else {
       // Our sum'd dim value is accurate, so we keep it
@@ -269,6 +309,6 @@ public abstract class IntTaxonomyFacets extends TaxonomyFacets {
       labelValues[i] = new LabelAndValue(bulkPath[i].components[cp.length], values[i]);
     }
 
-    return new FacetResult(dim, path, totValue, labelValues, childCount);
+    return new FacetResult(dim, path, aggregatedValue, labelValues, childCount);
   }
 }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetCounts.java
index 601870e5524..8dd55e96ea2 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetCounts.java
@@ -47,7 +47,12 @@ public class TaxonomyFacetCounts extends IntTaxonomyFacets {
       FacetsConfig config,
       FacetsCollector fc)
       throws IOException {
-    super(ordinalsReader.getIndexFieldName(), taxoReader, config, fc);
+    super(
+        ordinalsReader.getIndexFieldName(),
+        taxoReader,
+        config,
+        AssociationAggregationFunction.SUM,
+        fc);
     this.ordinalsReader = ordinalsReader;
     count(fc.getMatchingDocs());
   }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetFloatAssociations.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetFloatAssociations.java
new file mode 100644
index 00000000000..b85ffc90471
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetFloatAssociations.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.facet.taxonomy;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.lucene.facet.FacetField;
+import org.apache.lucene.facet.FacetUtils;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.taxonomy.OrdinalsReader.OrdinalsSegmentReader;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
+import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+
+/**
+ * Aggregates float values associated with facet fields. Supports two different approaches:
+ *
+ * <ol>
+ *   <li>Fields can be indexed with {@link FloatAssociationFacetField}, associating weights with
+ *       facet values at indexing time.
+ *   <li>Fields can be indexed with {@link FacetField} and a {@link DoubleValuesSource} can
+ *       dynamically supply a weight from each doc. With this approach, the document's weight gets
+ *       contributed to each facet value associated with the doc.
+ * </ol>
+ *
+ * Aggregation logic is supplied by the provided {@link FloatAssociationFacetField}.
+ *
+ * @lucene.experimental
+ */
+public class TaxonomyFacetFloatAssociations extends FloatTaxonomyFacets {
+
+  private final OrdinalsReader ordinalsReader;
+
+  /** Create {@code TaxonomyFacetFloatAssociations} against the default index field. */
+  public TaxonomyFacetFloatAssociations(
+      TaxonomyReader taxoReader,
+      FacetsConfig config,
+      FacetsCollector fc,
+      AssociationAggregationFunction aggregationFunction)
+      throws IOException {
+    this(FacetsConfig.DEFAULT_INDEX_FIELD_NAME, taxoReader, config, fc, aggregationFunction);
+  }
+
+  /**
+   * Create {@code TaxonomyFacetFloatAssociations} against the default index field. Sources values
+   * from the provided {@code valuesSource}.
+   */
+  public TaxonomyFacetFloatAssociations(
+      TaxonomyReader taxoReader,
+      FacetsConfig config,
+      FacetsCollector fc,
+      AssociationAggregationFunction aggregationFunction,
+      DoubleValuesSource valuesSource)
+      throws IOException {
+    this(
+        FacetsConfig.DEFAULT_INDEX_FIELD_NAME,
+        taxoReader,
+        config,
+        fc,
+        aggregationFunction,
+        valuesSource);
+  }
+
+  /** Create {@code TaxonomyFacetFloatAssociations} against the specified index field. */
+  public TaxonomyFacetFloatAssociations(
+      String indexFieldName,
+      TaxonomyReader taxoReader,
+      FacetsConfig config,
+      FacetsCollector fc,
+      AssociationAggregationFunction aggregationFunction)
+      throws IOException {
+    super(indexFieldName, taxoReader, aggregationFunction, config);
+    ordinalsReader = null;
+    aggregateValues(aggregationFunction, fc.getMatchingDocs());
+  }
+
+  /**
+   * Create {@code TaxonomyFacetFloatAssociations} against the specified index field. Sources values
+   * from the provided {@code valuesSource}.
+   */
+  public TaxonomyFacetFloatAssociations(
+      String indexFieldName,
+      TaxonomyReader taxoReader,
+      FacetsConfig config,
+      FacetsCollector fc,
+      AssociationAggregationFunction aggregationFunction,
+      DoubleValuesSource valuesSource)
+      throws IOException {
+    super(indexFieldName, taxoReader, aggregationFunction, config);
+    ordinalsReader = null;
+    aggregateValues(aggregationFunction, fc.getMatchingDocs(), fc.getKeepScores(), valuesSource);
+  }
+
+  /**
+   * Create {@code TaxonomyFacetFloatAssociations} against the specified index field. Sources values
+   * from the provided {@code valuesSource}.
+   *
+   * @deprecated Custom binary encodings for taxonomy ordinals are no longer supported starting with
+   *     Lucene 9
+   */
+  @Deprecated
+  public TaxonomyFacetFloatAssociations(
+      OrdinalsReader ordinalsReader,
+      TaxonomyReader taxoReader,
+      FacetsConfig config,
+      FacetsCollector fc,
+      AssociationAggregationFunction aggregationFunction,
+      DoubleValuesSource valuesSource)
+      throws IOException {
+    super(ordinalsReader.getIndexFieldName(), taxoReader, aggregationFunction, config);
+    this.ordinalsReader = ordinalsReader;
+    aggregateValues(aggregationFunction, fc.getMatchingDocs(), fc.getKeepScores(), valuesSource);
+  }
+
+  private static DoubleValues scores(MatchingDocs hits) {
+    return new DoubleValues() {
+
+      int index = -1;
+
+      @Override
+      public double doubleValue() throws IOException {
+        return hits.scores[index];
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        index = doc;
+        return true;
+      }
+    };
+  }
+
+  /** Aggregate using the provided {@code DoubleValuesSource}. */
+  private void aggregateValues(
+      AssociationAggregationFunction aggregationFunction,
+      List<MatchingDocs> matchingDocs,
+      boolean keepScores,
+      DoubleValuesSource valueSource)
+      throws IOException {
+
+    if (ordinalsReader != null) {
+      // If the user provided a custom ordinals reader, use it to retrieve the document ordinals:
+      IntsRef scratch = new IntsRef();
+      for (MatchingDocs hits : matchingDocs) {
+        OrdinalsSegmentReader ords = ordinalsReader.getReader(hits.context);
+        DoubleValues scores = keepScores ? scores(hits) : null;
+        DoubleValues functionValues = valueSource.getValues(hits.context, scores);
+        DocIdSetIterator docs = hits.bits.iterator();
+
+        int doc;
+        while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+          ords.get(doc, scratch);
+          if (functionValues.advanceExact(doc)) {
+            float value = (float) functionValues.doubleValue();
+            for (int i = 0; i < scratch.length; i++) {
+              int ord = scratch.ints[i];
+              float newValue = aggregationFunction.aggregate(values[ord], value);
+              values[ord] = newValue;
+            }
+          }
+        }
+      }
+    } else {
+      for (MatchingDocs hits : matchingDocs) {
+        SortedNumericDocValues ordinalValues =
+            FacetUtils.loadOrdinalValues(hits.context.reader(), indexFieldName);
+        if (ordinalValues == null) {
+          continue;
+        }
+
+        DoubleValues scores = keepScores ? scores(hits) : null;
+        DoubleValues functionValues = valueSource.getValues(hits.context, scores);
+        DocIdSetIterator it =
+            ConjunctionUtils.intersectIterators(List.of(hits.bits.iterator(), ordinalValues));
+
+        for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+          if (functionValues.advanceExact(doc)) {
+            float value = (float) functionValues.doubleValue();
+            int ordinalCount = ordinalValues.docValueCount();
+            for (int i = 0; i < ordinalCount; i++) {
+              int ord = (int) ordinalValues.nextValue();
+              float newValue = aggregationFunction.aggregate(values[ord], value);
+              values[ord] = newValue;
+            }
+          }
+        }
+      }
+    }
+
+    // Hierarchical dimensions are supported when using a value source, so we need to rollup:
+    rollup();
+  }
+
+  /** Aggregate from indexed association values. */
+  private void aggregateValues(
+      AssociationAggregationFunction aggregationFunction, List<MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (MatchingDocs hits : matchingDocs) {
+      BinaryDocValues dv = DocValues.getBinary(hits.context.reader(), indexFieldName);
+      DocIdSetIterator it =
+          ConjunctionUtils.intersectIterators(Arrays.asList(hits.bits.iterator(), dv));
+
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        final BytesRef bytesRef = dv.binaryValue();
+        byte[] bytes = bytesRef.bytes;
+        int end = bytesRef.offset + bytesRef.length;
+        int offset = bytesRef.offset;
+        while (offset < end) {
+          int ord = (int) BitUtil.VH_BE_INT.get(bytes, offset);
+          offset += 4;
+          float value = (float) BitUtil.VH_BE_FLOAT.get(bytes, offset);
+          offset += 4;
+          float newValue = aggregationFunction.aggregate(values[ord], value);
+          values[ord] = newValue;
+        }
+      }
+    }
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetIntAssociations.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetIntAssociations.java
new file mode 100644
index 00000000000..09638bc6f30
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetIntAssociations.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.facet.taxonomy;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Aggregates int values previously indexed with {@link IntAssociationFacetField}, assuming the
+ * default encoding. The aggregation function is defined by a provided {@link
+ * AssociationAggregationFunction}.
+ *
+ * @lucene.experimental
+ */
+public class TaxonomyFacetIntAssociations extends IntTaxonomyFacets {
+
+  /** Create {@code TaxonomyFacetIntAssociations} against the default index field. */
+  public TaxonomyFacetIntAssociations(
+      TaxonomyReader taxoReader,
+      FacetsConfig config,
+      FacetsCollector fc,
+      AssociationAggregationFunction aggregationFunction)
+      throws IOException {
+    this(FacetsConfig.DEFAULT_INDEX_FIELD_NAME, taxoReader, config, fc, aggregationFunction);
+  }
+
+  /** Create {@code TaxonomyFacetIntAssociations} against the specified index field. */
+  public TaxonomyFacetIntAssociations(
+      String indexFieldName,
+      TaxonomyReader taxoReader,
+      FacetsConfig config,
+      FacetsCollector fc,
+      AssociationAggregationFunction aggregationFunction)
+      throws IOException {
+    super(indexFieldName, taxoReader, config, aggregationFunction, fc);
+    aggregateValues(aggregationFunction, fc.getMatchingDocs());
+  }
+
+  private void aggregateValues(
+      AssociationAggregationFunction aggregationFunction, List<MatchingDocs> matchingDocs)
+      throws IOException {
+    for (MatchingDocs hits : matchingDocs) {
+      BinaryDocValues dv = DocValues.getBinary(hits.context.reader(), indexFieldName);
+      DocIdSetIterator it = ConjunctionUtils.intersectIterators(List.of(hits.bits.iterator(), dv));
+
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        final BytesRef bytesRef = dv.binaryValue();
+        byte[] bytes = bytesRef.bytes;
+        int end = bytesRef.offset + bytesRef.length;
+        int offset = bytesRef.offset;
+        while (offset < end) {
+          int ord = (int) BitUtil.VH_BE_INT.get(bytes, offset);
+          offset += 4;
+          int value = (int) BitUtil.VH_BE_INT.get(bytes, offset);
+          offset += 4;
+          // TODO: Can we optimize the null check in setValue? See LUCENE-10373.
+          int currentValue = getValue(ord);
+          int newValue = aggregationFunction.aggregate(currentValue, value);
+          setValue(ord, newValue);
+        }
+      }
+    }
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumFloatAssociations.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumFloatAssociations.java
index 5773d2e49d4..9325863d467 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumFloatAssociations.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumFloatAssociations.java
@@ -17,22 +17,19 @@
 package org.apache.lucene.facet.taxonomy;
 
 import java.io.IOException;
-import java.util.List;
 import org.apache.lucene.facet.FacetsCollector;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.BitUtil;
-import org.apache.lucene.util.BytesRef;
 
 /**
  * Aggregates sum of int values previously indexed with {@link FloatAssociationFacetField}, assuming
  * the default encoding.
  *
  * @lucene.experimental
+ * @deprecated This class is being deprecated in favor of {@link TaxonomyFacetFloatAssociations},
+ *     which provides more flexible aggregation functionality beyond just "sum"
  */
-public class TaxonomyFacetSumFloatAssociations extends FloatTaxonomyFacets {
+@Deprecated
+public class TaxonomyFacetSumFloatAssociations extends TaxonomyFacetFloatAssociations {
 
   /** Create {@code TaxonomyFacetSumFloatAssociations} against the default index field. */
   public TaxonomyFacetSumFloatAssociations(
@@ -44,42 +41,6 @@ public class TaxonomyFacetSumFloatAssociations extends FloatTaxonomyFacets {
   public TaxonomyFacetSumFloatAssociations(
       String indexFieldName, TaxonomyReader taxoReader, FacetsConfig config, FacetsCollector fc)
       throws IOException {
-    super(indexFieldName, taxoReader, config);
-    sumValues(fc.getMatchingDocs());
-  }
-
-  private final void sumValues(List<MatchingDocs> matchingDocs) throws IOException {
-    // System.out.println("count matchingDocs=" + matchingDocs + " facetsField=" + facetsFieldName);
-    for (MatchingDocs hits : matchingDocs) {
-      BinaryDocValues dv = hits.context.reader().getBinaryDocValues(indexFieldName);
-      if (dv == null) { // this reader does not have DocValues for the requested category list
-        continue;
-      }
-
-      DocIdSetIterator docs = hits.bits.iterator();
-
-      int doc;
-      while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        // System.out.println("  doc=" + doc);
-        // TODO: use OrdinalsReader?  we'd need to add a
-        // BytesRef getAssociation()?
-        if (dv.docID() < doc) {
-          dv.advance(doc);
-        }
-        if (dv.docID() == doc) {
-          final BytesRef bytesRef = dv.binaryValue();
-          byte[] bytes = bytesRef.bytes;
-          int end = bytesRef.offset + bytesRef.length;
-          int offset = bytesRef.offset;
-          while (offset < end) {
-            int ord = (int) BitUtil.VH_BE_INT.get(bytes, offset);
-            offset += 4;
-            float value = (float) BitUtil.VH_BE_FLOAT.get(bytes, offset);
-            offset += 4;
-            values[ord] += value;
-          }
-        }
-      }
-    }
+    super(indexFieldName, taxoReader, config, fc, AssociationAggregationFunction.SUM);
   }
 }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumIntAssociations.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumIntAssociations.java
index 96c23a9a3ee..93715205e25 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumIntAssociations.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumIntAssociations.java
@@ -17,22 +17,19 @@
 package org.apache.lucene.facet.taxonomy;
 
 import java.io.IOException;
-import java.util.List;
 import org.apache.lucene.facet.FacetsCollector;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.BitUtil;
-import org.apache.lucene.util.BytesRef;
 
 /**
  * Aggregates sum of int values previously indexed with {@link IntAssociationFacetField}, assuming
  * the default encoding.
  *
  * @lucene.experimental
+ * @deprecated This class is being deprecated in favor of {@link TaxonomyFacetIntAssociations},
+ *     which provides more flexible aggregation functionality beyond just "sum"
  */
-public class TaxonomyFacetSumIntAssociations extends IntTaxonomyFacets {
+@Deprecated
+public class TaxonomyFacetSumIntAssociations extends TaxonomyFacetIntAssociations {
 
   /** Create {@code TaxonomyFacetSumIntAssociations} against the default index field. */
   public TaxonomyFacetSumIntAssociations(
@@ -44,42 +41,6 @@ public class TaxonomyFacetSumIntAssociations extends IntTaxonomyFacets {
   public TaxonomyFacetSumIntAssociations(
       String indexFieldName, TaxonomyReader taxoReader, FacetsConfig config, FacetsCollector fc)
       throws IOException {
-    super(indexFieldName, taxoReader, config, fc);
-    sumValues(fc.getMatchingDocs());
-  }
-
-  private final void sumValues(List<MatchingDocs> matchingDocs) throws IOException {
-    // System.out.println("count matchingDocs=" + matchingDocs + " facetsField=" + facetsFieldName);
-    for (MatchingDocs hits : matchingDocs) {
-      BinaryDocValues dv = hits.context.reader().getBinaryDocValues(indexFieldName);
-      if (dv == null) { // this reader does not have DocValues for the requested category list
-        continue;
-      }
-
-      DocIdSetIterator docs = hits.bits.iterator();
-
-      int doc;
-      while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        // System.out.println("  doc=" + doc);
-        // TODO: use OrdinalsReader?  we'd need to add a
-        // BytesRef getAssociation()?
-        if (dv.docID() < doc) {
-          dv.advance(doc);
-        }
-        if (dv.docID() == doc) {
-          final BytesRef bytesRef = dv.binaryValue();
-          byte[] bytes = bytesRef.bytes;
-          int end = bytesRef.offset + bytesRef.length;
-          int offset = bytesRef.offset;
-          while (offset < end) {
-            int ord = (int) BitUtil.VH_BE_INT.get(bytes, offset);
-            offset += 4;
-            int value = (int) BitUtil.VH_BE_INT.get(bytes, offset);
-            offset += 4;
-            increment(ord, value);
-          }
-        }
-      }
-    }
+    super(indexFieldName, taxoReader, config, fc, AssociationAggregationFunction.SUM);
   }
 }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
index a5126f069b5..4c6e3b5c669 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
@@ -17,28 +17,23 @@
 package org.apache.lucene.facet.taxonomy;
 
 import java.io.IOException;
-import java.util.List;
-import org.apache.lucene.facet.FacetUtils;
 import org.apache.lucene.facet.FacetsCollector;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.search.ConjunctionUtils;
-import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
-import org.apache.lucene.util.IntsRef;
 
 /**
  * Aggregates sum of values from {@link DoubleValues#doubleValue()}, for each facet label.
  *
  * @lucene.experimental
+ * @deprecated This class is being deprecated in favor of {@link TaxonomyFacetFloatAssociations},
+ *     which provides more flexible aggregation functionality beyond just "sum"
  */
-public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
-  private final OrdinalsReader ordinalsReader;
+@Deprecated
+public class TaxonomyFacetSumValueSource extends TaxonomyFacetFloatAssociations {
 
   /**
-   * Aggreggates double facet values from the provided {@link DoubleValuesSource}, pulling ordinals
+   * Aggregates double facet values from the provided {@link DoubleValuesSource}, pulling ordinals
    * from the default indexed facet field {@link FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
    */
   public TaxonomyFacetSumValueSource(
@@ -51,7 +46,7 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
   }
 
   /**
-   * Aggreggates double facet values from the provided {@link DoubleValuesSource}, pulling ordinals
+   * Aggregates double facet values from the provided {@link DoubleValuesSource}, pulling ordinals
    * from the specified indexed facet field.
    */
   public TaxonomyFacetSumValueSource(
@@ -61,13 +56,11 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
       FacetsCollector fc,
       DoubleValuesSource valueSource)
       throws IOException {
-    super(indexField, taxoReader, config);
-    ordinalsReader = null;
-    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource);
+    super(indexField, taxoReader, config, fc, AssociationAggregationFunction.SUM, valueSource);
   }
 
   /**
-   * Aggreggates float facet values from the provided {@link DoubleValuesSource}, and pulls ordinals
+   * Aggregates float facet values from the provided {@link DoubleValuesSource}, and pulls ordinals
    * from the provided {@link OrdinalsReader}.
    *
    * @deprecated Custom binary encodings for taxonomy ordinals are no longer supported starting with
@@ -81,79 +74,6 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
       FacetsCollector fc,
       DoubleValuesSource vs)
       throws IOException {
-    super(ordinalsReader.getIndexFieldName(), taxoReader, config);
-    this.ordinalsReader = ordinalsReader;
-    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), vs);
-  }
-
-  private static DoubleValues scores(MatchingDocs hits) {
-    return new DoubleValues() {
-
-      int index = -1;
-
-      @Override
-      public double doubleValue() throws IOException {
-        return hits.scores[index];
-      }
-
-      @Override
-      public boolean advanceExact(int doc) throws IOException {
-        index = doc;
-        return true;
-      }
-    };
-  }
-
-  private void sumValues(
-      List<MatchingDocs> matchingDocs, boolean keepScores, DoubleValuesSource valueSource)
-      throws IOException {
-
-    if (ordinalsReader != null) {
-      // If the user provided a custom ordinals reader, use it to retrieve the document ordinals:
-      IntsRef scratch = new IntsRef();
-      for (MatchingDocs hits : matchingDocs) {
-        OrdinalsReader.OrdinalsSegmentReader ords = ordinalsReader.getReader(hits.context);
-        DoubleValues scores = keepScores ? scores(hits) : null;
-        DoubleValues functionValues = valueSource.getValues(hits.context, scores);
-        DocIdSetIterator docs = hits.bits.iterator();
-
-        int doc;
-        while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-          ords.get(doc, scratch);
-          if (functionValues.advanceExact(doc)) {
-            float value = (float) functionValues.doubleValue();
-            for (int i = 0; i < scratch.length; i++) {
-              values[scratch.ints[i]] += value;
-            }
-          }
-        }
-      }
-    } else {
-      // If no custom ordinals reader is provided, expect the default encoding:
-      for (MatchingDocs hits : matchingDocs) {
-        SortedNumericDocValues ordinalValues =
-            FacetUtils.loadOrdinalValues(hits.context.reader(), indexFieldName);
-        if (ordinalValues == null) {
-          continue;
-        }
-
-        DoubleValues scores = keepScores ? scores(hits) : null;
-        DoubleValues functionValues = valueSource.getValues(hits.context, scores);
-        DocIdSetIterator it =
-            ConjunctionUtils.intersectIterators(List.of(hits.bits.iterator(), ordinalValues));
-
-        for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
-          if (functionValues.advanceExact(doc)) {
-            float value = (float) functionValues.doubleValue();
-            int ordinalCount = ordinalValues.docValueCount();
-            for (int i = 0; i < ordinalCount; i++) {
-              values[(int) ordinalValues.nextValue()] += value;
-            }
-          }
-        }
-      }
-    }
-
-    rollup();
+    super(ordinalsReader, taxoReader, config, fc, AssociationAggregationFunction.SUM, vs);
   }
 }
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetAssociations.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetAssociations.java
index 43de8880ade..1cf52396848 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetAssociations.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetAssociations.java
@@ -16,8 +16,14 @@
  */
 package org.apache.lucene.facet.taxonomy;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.facet.DrillDownQuery;
+import org.apache.lucene.facet.FacetResult;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.Facets;
 import org.apache.lucene.facet.FacetsCollector;
@@ -26,6 +32,7 @@ import org.apache.lucene.facet.FacetsConfig;
 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.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.store.Directory;
@@ -44,6 +51,11 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
 
   private static FacetsConfig config;
 
+  private static Map<String, List<Integer>> randomIntValues;
+  private static Map<String, List<Float>> randomFloatValues;
+  private static Map<String, List<Integer>> randomIntSingleValued;
+  private static Map<String, List<Float>> randomFloatSingleValued;
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     dir = newDirectory();
@@ -56,8 +68,14 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
     config = new FacetsConfig();
     config.setIndexFieldName("int", "$facets.int");
     config.setMultiValued("int", true);
+    config.setIndexFieldName("int_random", "$facets.int");
+    config.setMultiValued("int_random", true);
+    config.setIndexFieldName("int_single_valued", "$facets.int");
     config.setIndexFieldName("float", "$facets.float");
     config.setMultiValued("float", true);
+    config.setIndexFieldName("float_random", "$facets.float");
+    config.setMultiValued("float_random", true);
+    config.setIndexFieldName("float_single_valued", "$facets.float");
 
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
 
@@ -77,6 +95,49 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
       writer.addDocument(config.build(taxoWriter, doc));
     }
 
+    // Also index random content for more random testing:
+    String[] paths = new String[] {"a", "b", "c"};
+    int count = random().nextInt(1000);
+    randomIntValues = new HashMap<>();
+    randomFloatValues = new HashMap<>();
+    randomIntSingleValued = new HashMap<>();
+    randomFloatSingleValued = new HashMap<>();
+    for (int i = 0; i < count; i++) {
+      Document doc = new Document();
+
+      if (random().nextInt(10) >= 2) { // occasionally don't add any fields
+        // Add up to five ordinals + values for each doc. Note that duplicates are totally fine:
+        for (int j = 0; j < 5; j++) {
+          String path = paths[random().nextInt(3)];
+          if (random().nextBoolean()) { // maybe index an int association with the dim
+            int nextInt = atLeast(1);
+            randomIntValues.computeIfAbsent(path, k -> new ArrayList<>()).add(nextInt);
+            doc.add(new IntAssociationFacetField(nextInt, "int_random", path));
+          }
+          if (random().nextBoolean()) { // maybe index a float association with the dim
+            float nextFloat = random().nextFloat() * 10000f;
+            randomFloatValues.computeIfAbsent(path, k -> new ArrayList<>()).add(nextFloat);
+            doc.add(new FloatAssociationFacetField(nextFloat, "float_random", path));
+          }
+        }
+
+        // Also, (maybe) add to the single-valued association fields:
+        String path = paths[random().nextInt(3)];
+        if (random().nextBoolean()) {
+          int nextInt = atLeast(1);
+          randomIntSingleValued.computeIfAbsent(path, k -> new ArrayList<>()).add(nextInt);
+          doc.add(new IntAssociationFacetField(nextInt, "int_single_valued", path));
+        }
+        if (random().nextBoolean()) {
+          float nextFloat = random().nextFloat() * 10000f;
+          randomFloatSingleValued.computeIfAbsent(path, k -> new ArrayList<>()).add(nextFloat);
+          doc.add(new FloatAssociationFacetField(nextFloat, "float_single_valued", path));
+        }
+      }
+
+      writer.addDocument(config.build(taxoWriter, doc));
+    }
+
     taxoWriter.close();
     reader = writer.getReader();
     writer.close();
@@ -100,7 +161,7 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
     IndexSearcher searcher = newSearcher(reader);
     FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
 
-    Facets facets = new TaxonomyFacetSumIntAssociations("$facets.int", taxoReader, config, fc);
+    Facets facets = getIntSumFacets("$facets.int", taxoReader, config, fc);
     assertEquals(
         "dim=int path=[] value=-1 childCount=2\n  a (200)\n  b (150)\n",
         facets.getTopChildren(10, "int").toString());
@@ -110,11 +171,52 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
         "Wrong count for category 'b'!", 150, facets.getSpecificValue("int", "b").intValue());
   }
 
+  public void testIntAssociationRandom() throws Exception {
+
+    FacetsCollector fc = new FacetsCollector();
+
+    IndexSearcher searcher = newSearcher(reader);
+    searcher.search(new MatchAllDocsQuery(), fc);
+
+    Map<String, Integer> expected;
+    Facets facets;
+
+    // SUM:
+    facets =
+        new TaxonomyFacetIntAssociations(
+            "$facets.int", taxoReader, config, fc, AssociationAggregationFunction.SUM);
+    expected = new HashMap<>();
+    for (Map.Entry<String, List<Integer>> e : randomIntValues.entrySet()) {
+      expected.put(e.getKey(), e.getValue().stream().reduce(Integer::sum).orElse(0));
+    }
+    validateInts("int_random", expected, AssociationAggregationFunction.SUM, true, facets);
+    expected = new HashMap<>();
+    for (Map.Entry<String, List<Integer>> e : randomIntSingleValued.entrySet()) {
+      expected.put(e.getKey(), e.getValue().stream().reduce(Integer::sum).orElse(0));
+    }
+    validateInts("int_single_valued", expected, AssociationAggregationFunction.SUM, false, facets);
+
+    // MAX:
+    facets =
+        new TaxonomyFacetIntAssociations(
+            "$facets.int", taxoReader, config, fc, AssociationAggregationFunction.MAX);
+    expected = new HashMap<>();
+    for (Map.Entry<String, List<Integer>> e : randomIntValues.entrySet()) {
+      expected.put(e.getKey(), e.getValue().stream().max(Integer::compareTo).orElse(0));
+    }
+    validateInts("int_random", expected, AssociationAggregationFunction.MAX, true, facets);
+    expected = new HashMap<>();
+    for (Map.Entry<String, List<Integer>> e : randomIntSingleValued.entrySet()) {
+      expected.put(e.getKey(), e.getValue().stream().max(Integer::compareTo).orElse(0));
+    }
+    validateInts("int_single_valued", expected, AssociationAggregationFunction.MAX, false, facets);
+  }
+
   public void testFloatSumAssociation() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
 
-    Facets facets = new TaxonomyFacetSumFloatAssociations("$facets.float", taxoReader, config, fc);
+    Facets facets = getFloatSumFacets("$facets.float", taxoReader, config, fc, null);
     assertEquals(
         "dim=float path=[] value=-1.0 childCount=2\n  a (50.0)\n  b (9.999995)\n",
         facets.getTopChildren(10, "float").toString());
@@ -130,6 +232,49 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
         0.00001);
   }
 
+  public void testFloatAssociationRandom() throws Exception {
+
+    FacetsCollector fc = new FacetsCollector();
+
+    IndexSearcher searcher = newSearcher(reader);
+    searcher.search(new MatchAllDocsQuery(), fc);
+
+    Map<String, Float> expected;
+    Facets facets;
+
+    // SUM:
+    facets =
+        new TaxonomyFacetFloatAssociations(
+            "$facets.float", taxoReader, config, fc, AssociationAggregationFunction.SUM);
+    expected = new HashMap<>();
+    for (Map.Entry<String, List<Float>> e : randomFloatValues.entrySet()) {
+      expected.put(e.getKey(), e.getValue().stream().reduce(Float::sum).orElse(0f));
+    }
+    validateFloats("float_random", expected, AssociationAggregationFunction.SUM, true, facets);
+    expected = new HashMap<>();
+    for (Map.Entry<String, List<Float>> e : randomFloatSingleValued.entrySet()) {
+      expected.put(e.getKey(), e.getValue().stream().reduce(Float::sum).orElse(0f));
+    }
+    validateFloats(
+        "float_single_valued", expected, AssociationAggregationFunction.SUM, false, facets);
+
+    // MAX:
+    facets =
+        new TaxonomyFacetFloatAssociations(
+            "$facets.float", taxoReader, config, fc, AssociationAggregationFunction.MAX);
+    expected = new HashMap<>();
+    for (Map.Entry<String, List<Float>> e : randomFloatValues.entrySet()) {
+      expected.put(e.getKey(), e.getValue().stream().max(Float::compareTo).orElse(0f));
+    }
+    validateFloats("float_random", expected, AssociationAggregationFunction.MAX, true, facets);
+    expected = new HashMap<>();
+    for (Map.Entry<String, List<Float>> e : randomFloatSingleValued.entrySet()) {
+      expected.put(e.getKey(), e.getValue().stream().max(Float::compareTo).orElse(0f));
+    }
+    validateFloats(
+        "float_single_valued", expected, AssociationAggregationFunction.MAX, false, facets);
+  }
+
   /**
    * Make sure we can test both int and float assocs in one index, as long as we send each to a
    * different field.
@@ -138,7 +283,7 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
     IndexSearcher searcher = newSearcher(reader);
     FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
 
-    Facets facets = new TaxonomyFacetSumFloatAssociations("$facets.float", taxoReader, config, fc);
+    Facets facets = getFloatSumFacets("$facets.float", taxoReader, config, fc, null);
     assertEquals(
         "Wrong count for category 'a'!",
         50f,
@@ -150,7 +295,7 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
         facets.getSpecificValue("float", "b").floatValue(),
         0.00001);
 
-    facets = new TaxonomyFacetSumIntAssociations("$facets.int", taxoReader, config, fc);
+    facets = getIntSumFacets("$facets.int", taxoReader, config, fc);
     assertEquals(
         "Wrong count for category 'a'!", 200, facets.getSpecificValue("int", "a").intValue());
     assertEquals(
@@ -160,7 +305,7 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
   public void testWrongIndexFieldName() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
-    Facets facets = new TaxonomyFacetSumFloatAssociations(taxoReader, config, fc);
+    Facets facets = getFloatSumFacets("wrong_field", taxoReader, config, fc, null);
     expectThrows(
         IllegalArgumentException.class,
         () -> {
@@ -242,7 +387,7 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
     q.add("int", "b");
     FacetsCollector fc = searcher.search(q, new FacetsCollectorManager());
 
-    Facets facets = new TaxonomyFacetSumIntAssociations("$facets.int", taxoReader, config, fc);
+    Facets facets = getIntSumFacets("$facets.int", taxoReader, config, fc);
     assertEquals(
         "dim=int path=[] value=-1 childCount=2\n  b (150)\n  a (100)\n",
         facets.getTopChildren(10, "int").toString());
@@ -251,4 +396,87 @@ public class TestTaxonomyFacetAssociations extends FacetTestCase {
     assertEquals(
         "Wrong count for category 'b'!", 150, facets.getSpecificValue("int", "b").intValue());
   }
+
+  private Facets getIntSumFacets(
+      String fieldName, TaxonomyReader taxoReader, FacetsConfig facetsConfig, FacetsCollector fc)
+      throws IOException {
+    if (random().nextBoolean()) {
+      return new TaxonomyFacetSumIntAssociations(fieldName, taxoReader, facetsConfig, fc);
+    } else {
+      return new TaxonomyFacetIntAssociations(
+          fieldName, taxoReader, facetsConfig, fc, AssociationAggregationFunction.SUM);
+    }
+  }
+
+  private Facets getFloatSumFacets(
+      String fieldName,
+      TaxonomyReader taxoReader,
+      FacetsConfig facetsConfig,
+      FacetsCollector fc,
+      DoubleValuesSource dvs)
+      throws IOException {
+    if (random().nextBoolean()) {
+      if (dvs == null) {
+        return new TaxonomyFacetSumFloatAssociations(fieldName, taxoReader, facetsConfig, fc);
+      } else {
+        return new TaxonomyFacetSumValueSource(fieldName, taxoReader, facetsConfig, fc, dvs);
+      }
+    } else {
+      if (dvs == null) {
+        return new TaxonomyFacetFloatAssociations(
+            fieldName, taxoReader, facetsConfig, fc, AssociationAggregationFunction.SUM);
+      } else {
+        return new TaxonomyFacetFloatAssociations(
+            fieldName, taxoReader, facetsConfig, fc, AssociationAggregationFunction.SUM, dvs);
+      }
+    }
+  }
+
+  private void validateInts(
+      String dim,
+      Map<String, Integer> expected,
+      AssociationAggregationFunction aggregationFunction,
+      boolean isMultiValued,
+      Facets facets)
+      throws IOException {
+    int aggregatedValue = 0;
+    for (Map.Entry<String, Integer> e : expected.entrySet()) {
+      int value = e.getValue();
+      assertEquals(value, facets.getSpecificValue(dim, e.getKey()).intValue());
+      aggregatedValue = aggregationFunction.aggregate(aggregatedValue, value);
+    }
+
+    if (isMultiValued) {
+      aggregatedValue = -1;
+    }
+
+    FacetResult facetResult = facets.getTopChildren(10, dim);
+    assertEquals(dim, facetResult.dim);
+    assertEquals(aggregatedValue, facetResult.value.intValue());
+    assertEquals(expected.size(), facetResult.childCount);
+  }
+
+  private void validateFloats(
+      String dim,
+      Map<String, Float> expected,
+      AssociationAggregationFunction aggregationFunction,
+      boolean isMultiValued,
+      Facets facets)
+      throws IOException {
+    float aggregatedValue = 0f;
+    for (Map.Entry<String, Float> e : expected.entrySet()) {
+      float value = e.getValue();
+      assertEquals(value, facets.getSpecificValue(dim, e.getKey()).floatValue(), 1);
+      aggregatedValue = aggregationFunction.aggregate(aggregatedValue, value);
+    }
+
+    if (isMultiValued) {
+      aggregatedValue = -1;
+    }
+
+    FacetResult facetResult = facets.getTopChildren(10, dim);
+    assertEquals(dim, facetResult.dim);
+    assertEquals(aggregatedValue, facetResult.value.floatValue(), 1);
+    assertEquals(expected.size(), facetResult.childCount);
+  }
 }
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetValueSource.java
similarity index 67%
rename from lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
rename to lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetValueSource.java
index bd23587f237..4c513dabf5e 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetValueSource.java
@@ -16,6 +16,7 @@
  */
 package org.apache.lucene.facet.taxonomy;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -55,7 +56,7 @@ import org.apache.lucene.tests.index.RandomIndexWriter;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.IOUtils;
 
-public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
+public class TestTaxonomyFacetValueSource extends FacetTestCase {
 
   public void testBasic() throws Exception {
 
@@ -111,20 +112,44 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     // Facets.search utility methods:
     FacetsCollector c = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
 
-    TaxonomyFacetSumValueSource facets =
-        new TaxonomyFacetSumValueSource(
-            taxoReader, new FacetsConfig(), c, DoubleValuesSource.fromIntField("num"));
+    FacetsConfig facetsConfig = new FacetsConfig();
+    DoubleValuesSource valuesSource = DoubleValuesSource.fromIntField("num");
 
+    // Test SUM:
+    Facets facets =
+        getFacets(
+            null,
+            taxoReader,
+            facetsConfig,
+            c,
+            AssociationAggregationFunction.SUM,
+            valuesSource,
+            null);
     // Retrieve & verify results:
     assertEquals(
         "dim=Author path=[] value=145.0 childCount=4\n  Lisa (50.0)\n  Frank (45.0)\n  Susan (40.0)\n  Bob (10.0)\n",
         facets.getTopChildren(10, "Author").toString());
 
+    // Test MAX:
+    facets =
+        getFacets(
+            null,
+            taxoReader,
+            facetsConfig,
+            c,
+            AssociationAggregationFunction.MAX,
+            valuesSource,
+            null);
+    assertEquals(
+        "dim=Author path=[] value=45.0 childCount=4\n  Frank (45.0)\n  Susan (40.0)\n  Lisa (30.0)\n  Bob (10.0)\n",
+        facets.getTopChildren(10, "Author").toString());
+
     // test getTopChildren(0, dim)
+    final Facets f = facets;
     expectThrows(
         IllegalArgumentException.class,
         () -> {
-          facets.getTopChildren(0, "Author");
+          f.getTopChildren(0, "Author");
         });
 
     taxoReader.close();
@@ -182,9 +207,15 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
 
     FacetsCollector c = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
 
-    TaxonomyFacetSumValueSource facets =
-        new TaxonomyFacetSumValueSource(
-            taxoReader, new FacetsConfig(), c, DoubleValuesSource.fromIntField("num"));
+    Facets facets =
+        getFacets(
+            null,
+            taxoReader,
+            new FacetsConfig(),
+            c,
+            AssociationAggregationFunction.SUM,
+            DoubleValuesSource.fromIntField("num"),
+            null);
 
     // Ask for top 10 labels for any dims that have counts:
     List<FacetResult> results = facets.getAllDims(10);
@@ -262,9 +293,15 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
 
     FacetsCollector c = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
 
-    TaxonomyFacetSumValueSource facets =
-        new TaxonomyFacetSumValueSource(
-            taxoReader, config, c, DoubleValuesSource.fromIntField("num"));
+    Facets facets =
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            c,
+            AssociationAggregationFunction.SUM,
+            DoubleValuesSource.fromIntField("num"),
+            null);
 
     // Ask for top 10 labels for any dims that have counts:
     List<FacetResult> results = facets.getAllDims(10);
@@ -288,7 +325,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     IOUtils.close(searcher.getIndexReader(), taxoReader, dir, taxoDir);
   }
 
-  public void testSumScoreAggregator() throws Exception {
+  public void testScoreAggregator() throws Exception {
     Directory indexDir = newDirectory();
     Directory taxoDir = newDirectory();
 
@@ -314,12 +351,32 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
 
     TopDocs td = FacetsCollector.search(newSearcher(r), csq, 10, fc);
 
+    // Test SUM:
     Facets facets =
-        new TaxonomyFacetSumValueSource(taxoReader, config, fc, DoubleValuesSource.SCORES);
-
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            fc,
+            AssociationAggregationFunction.SUM,
+            DoubleValuesSource.SCORES,
+            null);
     int expected = (int) (csq.getBoost() * td.totalHits.value);
     assertEquals(expected, facets.getSpecificValue("dim", "a").intValue());
 
+    // Test MAX:
+    facets =
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            fc,
+            AssociationAggregationFunction.MAX,
+            DoubleValuesSource.SCORES,
+            null);
+    expected = (int) csq.getBoost();
+    assertEquals(expected, facets.getSpecificValue("dim", "a").intValue());
+
     iw.close();
     IOUtils.close(taxoWriter, taxoReader, taxoDir, r, indexDir);
   }
@@ -343,12 +400,35 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
 
     FacetsCollector sfc =
         newSearcher(r).search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Test SUM:
     Facets facets =
-        new TaxonomyFacetSumValueSource(
-            taxoReader, config, sfc, DoubleValuesSource.fromLongField("price"));
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            sfc,
+            AssociationAggregationFunction.SUM,
+            DoubleValuesSource.fromLongField("price"),
+            null);
     assertEquals(
         "dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n",
         facets.getTopChildren(10, "a").toString());
+
+    // Test MAX:
+    facets =
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            sfc,
+            AssociationAggregationFunction.MAX,
+            DoubleValuesSource.fromLongField("price"),
+            null);
+    assertEquals(
+        "dim=a path=[] value=4.0 childCount=2\n  1 (4.0)\n  0 (3.0)\n",
+        facets.getTopChildren(10, "a").toString());
+
     iw.close();
     IOUtils.close(taxoWriter, taxoReader, taxoDir, r, indexDir);
   }
@@ -381,12 +461,35 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     // categories easier
     Query q = new FunctionQuery(new LongFieldSource("price"));
     FacetsCollector.search(newSearcher(r), q, 10, fc);
-    Facets facets =
-        new TaxonomyFacetSumValueSource(taxoReader, config, fc, DoubleValuesSource.SCORES);
 
+    // Test SUM:
+    Facets facets =
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            fc,
+            AssociationAggregationFunction.SUM,
+            DoubleValuesSource.SCORES,
+            null);
     assertEquals(
         "dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n",
         facets.getTopChildren(10, "a").toString());
+
+    // Test MAX:
+    facets =
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            fc,
+            AssociationAggregationFunction.MAX,
+            DoubleValuesSource.SCORES,
+            null);
+    assertEquals(
+        "dim=a path=[] value=4.0 childCount=2\n  1 (4.0)\n  0 (3.0)\n",
+        facets.getTopChildren(10, "a").toString());
+
     iw.close();
     IOUtils.close(taxoWriter, taxoReader, taxoDir, r, indexDir);
   }
@@ -413,13 +516,35 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
 
     FacetsCollector sfc =
         newSearcher(r).search(new MatchAllDocsQuery(), new FacetsCollectorManager());
-    Facets facets =
-        new TaxonomyFacetSumValueSource(
-            taxoReader, config, sfc, DoubleValuesSource.fromLongField("price"));
 
+    // Test SUM:
+    Facets facets =
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            sfc,
+            AssociationAggregationFunction.SUM,
+            DoubleValuesSource.fromLongField("price"),
+            null);
     assertEquals(
         "dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n",
         facets.getTopChildren(10, "a").toString());
+
+    // Test MAX:
+    facets =
+        getFacets(
+            null,
+            taxoReader,
+            config,
+            sfc,
+            AssociationAggregationFunction.MAX,
+            DoubleValuesSource.fromLongField("price"),
+            null);
+    assertEquals(
+        "dim=a path=[] value=4.0 childCount=2\n  1 (4.0)\n  0 (3.0)\n",
+        facets.getTopChildren(10, "a").toString());
+
     iw.close();
     IOUtils.close(taxoWriter, taxoReader, taxoDir, r, indexDir);
   }
@@ -451,11 +576,24 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     Facets facets2;
     if (random().nextBoolean()) {
       facets2 =
-          new TaxonomyFacetSumValueSource(
-              new DocValuesOrdinalsReader("$b"), taxoReader, config, fc, DoubleValuesSource.SCORES);
+          getFacets(
+              null,
+              taxoReader,
+              config,
+              fc,
+              AssociationAggregationFunction.SUM,
+              DoubleValuesSource.SCORES,
+              new DocValuesOrdinalsReader("$b"));
     } else {
       facets2 =
-          new TaxonomyFacetSumValueSource("$b", taxoReader, config, fc, DoubleValuesSource.SCORES);
+          getFacets(
+              "$b",
+              taxoReader,
+              config,
+              fc,
+              AssociationAggregationFunction.SUM,
+              DoubleValuesSource.SCORES,
+              null);
     }
 
     assertEquals(r.maxDoc(), facets1.getTopChildren(10, "a").value.intValue());
@@ -502,75 +640,147 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
       }
       FacetsCollector fc = new FacetsCollector();
       FacetsCollector.search(searcher, new TermQuery(new Term("content", searchToken)), 10, fc);
-      Facets facets =
-          new TaxonomyFacetSumValueSource(
-              tr, config, fc, DoubleValuesSource.fromFloatField("value"));
-
-      // Slow, yet hopefully bug-free, faceting:
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      Map<String, Float>[] expectedValues = new HashMap[numDims];
-      for (int i = 0; i < numDims; i++) {
-        expectedValues[i] = new HashMap<>();
-      }
 
-      for (TestDoc doc : testDocs) {
-        if (doc.content.equals(searchToken)) {
-          for (int j = 0; j < numDims; j++) {
-            if (doc.dims[j] != null) {
-              Float v = expectedValues[j].get(doc.dims[j]);
-              if (v == null) {
-                expectedValues[j].put(doc.dims[j], doc.value);
-              } else {
-                expectedValues[j].put(doc.dims[j], v + doc.value);
-              }
+      checkResults(
+          numDims,
+          testDocs,
+          searchToken,
+          tr,
+          config,
+          fc,
+          DoubleValuesSource.fromFloatField("value"),
+          AssociationAggregationFunction.SUM);
+      checkResults(
+          numDims,
+          testDocs,
+          searchToken,
+          tr,
+          config,
+          fc,
+          DoubleValuesSource.fromFloatField("value"),
+          AssociationAggregationFunction.MAX);
+    }
+
+    w.close();
+    IOUtils.close(tw, searcher.getIndexReader(), tr, indexDir, taxoDir);
+  }
+
+  private void checkResults(
+      int numDims,
+      List<TestDoc> testDocs,
+      String searchToken,
+      TaxonomyReader taxoReader,
+      FacetsConfig facetsConfig,
+      FacetsCollector facetsCollector,
+      DoubleValuesSource valuesSource,
+      AssociationAggregationFunction aggregationFunction)
+      throws IOException {
+    // Slow, yet hopefully bug-free, faceting:
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    Map<String, Float>[] expectedValues = new HashMap[numDims];
+    for (int i = 0; i < numDims; i++) {
+      expectedValues[i] = new HashMap<>();
+    }
+
+    for (TestDoc doc : testDocs) {
+      if (doc.content.equals(searchToken)) {
+        for (int j = 0; j < numDims; j++) {
+          if (doc.dims[j] != null) {
+            Float v = expectedValues[j].get(doc.dims[j]);
+            if (v == null) {
+              expectedValues[j].put(doc.dims[j], doc.value);
+            } else {
+              float newValue = aggregationFunction.aggregate(v, doc.value);
+              expectedValues[j].put(doc.dims[j], newValue);
             }
           }
         }
       }
+    }
 
-      List<FacetResult> expected = new ArrayList<>();
-      for (int i = 0; i < numDims; i++) {
-        List<LabelAndValue> labelValues = new ArrayList<>();
-        double totValue = 0;
-        for (Map.Entry<String, Float> ent : expectedValues[i].entrySet()) {
-          if (ent.getValue() > 0) {
-            labelValues.add(new LabelAndValue(ent.getKey(), ent.getValue()));
-            totValue += ent.getValue();
-          }
-        }
-        sortLabelValues(labelValues);
-        if (totValue > 0) {
-          expected.add(
-              new FacetResult(
-                  "dim" + i,
-                  new String[0],
-                  totValue,
-                  labelValues.toArray(new LabelAndValue[labelValues.size()]),
-                  labelValues.size()));
-        }
+    List<FacetResult> expected = new ArrayList<>();
+    for (int i = 0; i < numDims; i++) {
+      List<LabelAndValue> labelValues = new ArrayList<>();
+      float aggregatedValue = 0;
+      for (Map.Entry<String, Float> ent : expectedValues[i].entrySet()) {
+        labelValues.add(new LabelAndValue(ent.getKey(), ent.getValue()));
+        aggregatedValue = aggregationFunction.aggregate(aggregatedValue, ent.getValue());
+      }
+      sortLabelValues(labelValues);
+      if (aggregatedValue > 0) {
+        expected.add(
+            new FacetResult(
+                "dim" + i,
+                new String[0],
+                aggregatedValue,
+                labelValues.toArray(new LabelAndValue[labelValues.size()]),
+                labelValues.size()));
       }
+    }
 
-      // Sort by highest value, tie break by value:
-      sortFacetResults(expected);
+    // Sort by highest value, tie break by value:
+    sortFacetResults(expected);
 
-      List<FacetResult> actual = facets.getAllDims(10);
-      // test default implementation of getTopDims
-      if (actual.size() > 0) {
-        List<FacetResult> topDimsResults1 = facets.getTopDims(1, 10);
-        assertEquals(actual.get(0), topDimsResults1.get(0));
-      }
-      // Messy: fixup ties
-      sortTies(actual);
+    Facets facets =
+        getFacets(
+            null,
+            taxoReader,
+            facetsConfig,
+            facetsCollector,
+            aggregationFunction,
+            valuesSource,
+            null);
 
-      if (VERBOSE) {
-        System.out.println("expected=\n" + expected.toString());
-        System.out.println("actual=\n" + actual.toString());
-      }
+    List<FacetResult> actual = facets.getAllDims(10);
 
-      assertFloatValuesEquals(expected, actual);
+    // test default implementation of getTopDims
+    if (actual.size() > 0) {
+      List<FacetResult> topDimsResults1 = facets.getTopDims(1, 10);
+      assertEquals(actual.get(0), topDimsResults1.get(0));
     }
 
-    w.close();
-    IOUtils.close(tw, searcher.getIndexReader(), tr, indexDir, taxoDir);
+    // Messy: fixup ties
+    sortTies(actual);
+
+    if (VERBOSE) {
+      System.out.println("expected=\n" + expected.toString());
+      System.out.println("actual=\n" + actual.toString());
+    }
+
+    assertFloatValuesEquals(expected, actual);
+  }
+
+  private Facets getFacets(
+      String fieldName,
+      TaxonomyReader taxoReader,
+      FacetsConfig facetsConfig,
+      FacetsCollector fc,
+      AssociationAggregationFunction aggregationFunction,
+      DoubleValuesSource values,
+      OrdinalsReader ordinalsReader)
+      throws IOException {
+    if (aggregationFunction != AssociationAggregationFunction.SUM || random().nextBoolean()) {
+      if (ordinalsReader != null) {
+        assert fieldName == null;
+        return new TaxonomyFacetFloatAssociations(
+            ordinalsReader, taxoReader, facetsConfig, fc, aggregationFunction, values);
+      } else if (fieldName == null) {
+        return new TaxonomyFacetFloatAssociations(
+            taxoReader, facetsConfig, fc, aggregationFunction, values);
+      } else {
+        return new TaxonomyFacetFloatAssociations(
+            fieldName, taxoReader, facetsConfig, fc, aggregationFunction, values);
+      }
+    } else {
+      if (ordinalsReader != null) {
+        assert fieldName == null;
+        return new TaxonomyFacetSumValueSource(
+            ordinalsReader, taxoReader, facetsConfig, fc, values);
+      } else if (fieldName == null) {
+        return new TaxonomyFacetSumValueSource(taxoReader, facetsConfig, fc, values);
+      } else {
+        return new TaxonomyFacetSumValueSource(fieldName, taxoReader, facetsConfig, fc, values);
+      }
+    }
   }
 }