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/01/03 13:56:29 UTC

svn commit: r1428320 [2/2] - in /lucene/dev/trunk/lucene: ./ benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ facet/src/examples/org/apache/lucene/facet/example/association/ facet...

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/PathPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/PathPolicy.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/PathPolicy.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/PathPolicy.java Thu Jan  3 12:56:27 2013
@@ -2,7 +2,7 @@ package org.apache.lucene.facet.index.ca
 
 import java.io.Serializable;
 
-import org.apache.lucene.facet.index.streaming.CategoryParentsStream;
+import org.apache.lucene.facet.index.DrillDownStream;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 
 /*
@@ -24,7 +24,7 @@ import org.apache.lucene.facet.taxonomy.
 
 /**
  * Determines which {@link CategoryPath categories} should be added as terms to
- * the {@link CategoryParentsStream}. The default approach is implemented by
+ * the {@link DrillDownStream}. The default approach is implemented by
  * {@link #ALL_CATEGORIES}.
  * 
  * @lucene.experimental

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java Thu Jan  3 12:56:27 2013
@@ -130,7 +130,7 @@ public class FacetIndexingParams {
   public int getPartitionSize() {
     return partitionSize;
   }
-
+  
   /**
    * Returns a list of all {@link CategoryListParams categoryListParams} that
    * are used for facets indexing.

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java?rev=1428320&r1=1426915&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java Thu Jan  3 12:56:27 2013
@@ -1,8 +1,9 @@
-package org.apache.lucene.facet.search.aggregator.association;
+package org.apache.lucene.facet.search.aggregator.associations;
 
 import java.io.IOException;
 
-import org.apache.lucene.facet.enhancements.association.AssociationsPayloadIterator;
+import org.apache.lucene.facet.associations.CategoryFloatAssociation;
+import org.apache.lucene.facet.associations.FloatAssociationsPayloadIterator;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.index.IndexReader;
@@ -25,8 +26,8 @@ import org.apache.lucene.index.IndexRead
  */
 
 /**
- * An {@link Aggregator} which updates the weight of a category by summing the
- * weights of the float association it finds for every document.
+ * An {@link Aggregator} which computes the weight of a category as the sum of
+ * the float values associated with it in the result documents.
  * 
  * @lucene.experimental
  */
@@ -34,7 +35,7 @@ public class AssociationFloatSumAggregat
 
   protected final String field;
   protected final float[] sumArray;
-  protected final AssociationsPayloadIterator associationsPayloadIterator;
+  protected final FloatAssociationsPayloadIterator associations;
 
   public AssociationFloatSumAggregator(IndexReader reader, float[] sumArray) throws IOException {
     this(CategoryListParams.DEFAULT_TERM.field(), reader, sumArray);
@@ -42,15 +43,15 @@ public class AssociationFloatSumAggregat
   
   public AssociationFloatSumAggregator(String field, IndexReader reader, float[] sumArray) throws IOException {
     this.field = field;
-    associationsPayloadIterator = new AssociationsPayloadIterator(reader, field);
+    associations = new FloatAssociationsPayloadIterator(reader, field, new CategoryFloatAssociation());
     this.sumArray = sumArray;
   }
 
   @Override
   public void aggregate(int ordinal) {
-    long association = associationsPayloadIterator.getAssociation(ordinal);
-    if (association != AssociationsPayloadIterator.NO_ASSOCIATION) {
-      sumArray[ordinal] += Float.intBitsToFloat((int) association);
+    float association = associations.getAssociation(ordinal);
+    if (!Float.isNaN(association)) {
+      sumArray[ordinal] += association;
     }
   }
 
@@ -70,7 +71,7 @@ public class AssociationFloatSumAggregat
 
   @Override
   public void setNextDoc(int docid, float score) throws IOException {
-    associationsPayloadIterator.setNextDoc(docid);
+    associations.setNextDoc(docid);
   }
   
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java?rev=1428320&r1=1426915&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java Thu Jan  3 12:56:27 2013
@@ -1,8 +1,9 @@
-package org.apache.lucene.facet.search.aggregator.association;
+package org.apache.lucene.facet.search.aggregator.associations;
 
 import java.io.IOException;
 
-import org.apache.lucene.facet.enhancements.association.AssociationsPayloadIterator;
+import org.apache.lucene.facet.associations.CategoryIntAssociation;
+import org.apache.lucene.facet.associations.IntAssociationsPayloadIterator;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.index.IndexReader;
@@ -25,8 +26,8 @@ import org.apache.lucene.index.IndexRead
  */
 
 /**
- * An {@link Aggregator} which updates the weight of a category by summing the
- * weights of the integer association it finds for every document.
+ * An {@link Aggregator} which computes the weight of a category as the sum of
+ * the integer values associated with it in the result documents.
  * 
  * @lucene.experimental
  */
@@ -34,7 +35,7 @@ public class AssociationIntSumAggregator
 
   protected final String field;
   protected final int[] sumArray;
-  protected final AssociationsPayloadIterator associationsPayloadIterator;
+  protected final IntAssociationsPayloadIterator associations;
 
   public AssociationIntSumAggregator(IndexReader reader, int[] sumArray) throws IOException {
     this(CategoryListParams.DEFAULT_TERM.field(), reader, sumArray);
@@ -42,14 +43,14 @@ public class AssociationIntSumAggregator
   
   public AssociationIntSumAggregator(String field, IndexReader reader, int[] sumArray) throws IOException {
     this.field = field;
-    associationsPayloadIterator = new AssociationsPayloadIterator(reader, field);
+    associations = new IntAssociationsPayloadIterator(reader, field, new CategoryIntAssociation());
     this.sumArray = sumArray;
   }
 
   @Override
   public void aggregate(int ordinal) {
-    long association = associationsPayloadIterator.getAssociation(ordinal);
-    if (association != AssociationsPayloadIterator.NO_ASSOCIATION) {
+    long association = associations.getAssociation(ordinal);
+    if (association != IntAssociationsPayloadIterator.NO_ASSOCIATION) {
       sumArray[ordinal] += association;
     }
   }
@@ -70,7 +71,7 @@ public class AssociationIntSumAggregator
 
   @Override
   public void setNextDoc(int docid, float score) throws IOException {
-    associationsPayloadIterator.setNextDoc(docid);
+    associations.setNextDoc(docid);
   }
   
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java?rev=1428320&r1=1426915&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java Thu Jan  3 12:56:27 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search.params.association;
+package org.apache.lucene.facet.search.params.associations;
 
 import java.io.IOException;
 
@@ -6,7 +6,7 @@ import org.apache.lucene.index.IndexRead
 
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
-import org.apache.lucene.facet.search.aggregator.association.AssociationFloatSumAggregator;
+import org.apache.lucene.facet.search.aggregator.associations.AssociationFloatSumAggregator;
 import org.apache.lucene.facet.search.params.FacetRequest;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
@@ -29,8 +29,8 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Facet request for weighting facets according to their float association by
- * summing the association values.
+ * A {@link FacetRequest} for weighting facets according to their float
+ * association by summing the association values.
  * 
  * @lucene.experimental
  */
@@ -45,9 +45,8 @@ public class AssociationFloatSumFacetReq
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements,
-                                      FacetArrays arrays, IndexReader reader,
-                                      TaxonomyReader taxonomy) throws IOException {
+  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, IndexReader reader, 
+      TaxonomyReader taxonomy) throws IOException {
     assert !useComplements : "complements are not supported by this FacetRequest";
     return new AssociationFloatSumAggregator(reader, arrays.getFloatArray());
   }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java?rev=1428320&r1=1426915&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java Thu Jan  3 12:56:27 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search.params.association;
+package org.apache.lucene.facet.search.params.associations;
 
 import java.io.IOException;
 
@@ -6,7 +6,7 @@ import org.apache.lucene.index.IndexRead
 
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
-import org.apache.lucene.facet.search.aggregator.association.AssociationIntSumAggregator;
+import org.apache.lucene.facet.search.aggregator.associations.AssociationIntSumAggregator;
 import org.apache.lucene.facet.search.params.FacetRequest;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
@@ -29,8 +29,8 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Facet request for weighting facets according to their integer association by
- * summing the association values.
+ * A {@link FacetRequest} for weighting facets according to their integer
+ * association by summing the association values.
  * 
  * @lucene.experimental
  */
@@ -45,9 +45,8 @@ public class AssociationIntSumFacetReque
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements,
-                                      FacetArrays arrays, IndexReader reader,
-                                      TaxonomyReader taxonomy) throws IOException {
+  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, IndexReader reader, 
+      TaxonomyReader taxonomy) throws IOException {
     assert !useComplements : "complements are not supported by this FacetRequest";
     return new AssociationIntSumAggregator(reader, arrays.getIntArray());
   }

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/collections/IntToFloatMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/collections/IntToFloatMap.java?rev=1428320&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/collections/IntToFloatMap.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/collections/IntToFloatMap.java Thu Jan  3 12:56:27 2013
@@ -0,0 +1,631 @@
+package org.apache.lucene.util.collections;
+
+import java.util.Arrays;
+
+/*
+ * 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.
+ */
+
+/**
+ * An Array-based hashtable which maps primitive int to a primitive float.<br>
+ * The hashtable is constracted with a given capacity, or 16 as a default. In
+ * case there's not enough room for new pairs, the hashtable grows. <br>
+ * Capacity is adjusted to a power of 2, and there are 2 * capacity entries for
+ * the hash.
+ * 
+ * The pre allocated arrays (for keys, values) are at length of capacity + 1,
+ * when index 0 is used as 'Ground' or 'NULL'.<br>
+ * 
+ * The arrays are allocated ahead of hash operations, and form an 'empty space'
+ * list, to which the key,value pair is allocated.
+ * 
+ * @lucene.experimental
+ */
+public class IntToFloatMap {
+
+  public static final float GROUND = Float.NaN;
+
+  /**
+   * Implements an IntIterator which iterates over all the allocated indexes.
+   */
+  private final class IndexIterator implements IntIterator {
+    /**
+     * The last used baseHashIndex. Needed for "jumping" from one hash entry
+     * to another.
+     */
+    private int baseHashIndex = 0;
+
+    /**
+     * The next not-yet-visited index.
+     */
+    private int index = 0;
+
+    /**
+     * Index of the last visited pair. Used in {@link #remove()}.
+     */
+    private int lastIndex = 0;
+
+    /**
+     * Create the Iterator, make <code>index</code> point to the "first"
+     * index which is not empty. If such does not exist (eg. the map is
+     * empty) it would be zero.
+     */
+    public IndexIterator() {
+      for (baseHashIndex = 0; baseHashIndex < baseHash.length; ++baseHashIndex) {
+        index = baseHash[baseHashIndex];
+        if (index != 0) {
+          break;
+        }
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      return (index != 0);
+    }
+
+    @Override
+    public int next() {
+      // Save the last index visited
+      lastIndex = index;
+
+      // next the index
+      index = next[index];
+
+      // if the next index points to the 'Ground' it means we're done with
+      // the current hash entry and we need to jump to the next one. This
+      // is done until all the hash entries had been visited.
+      while (index == 0 && ++baseHashIndex < baseHash.length) {
+        index = baseHash[baseHashIndex];
+      }
+
+      return lastIndex;
+    }
+
+    @Override
+    public void remove() {
+      IntToFloatMap.this.remove(keys[lastIndex]);
+    }
+
+  }
+
+  /**
+   * Implements an IntIterator, used for iteration over the map's keys.
+   */
+  private final class KeyIterator implements IntIterator {
+    private IntIterator iterator = new IndexIterator();
+
+    KeyIterator() { }
+    
+    @Override
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    @Override
+    public int next() {
+      return keys[iterator.next()];
+    }
+
+    @Override
+    public void remove() {
+      iterator.remove();
+    }
+  }
+
+  /**
+   * Implements an Iterator of a generic type T used for iteration over the
+   * map's values.
+   */
+  private final class ValueIterator implements FloatIterator {
+    private IntIterator iterator = new IndexIterator();
+
+    ValueIterator() { }
+    
+    @Override
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    @Override
+    public float next() {
+      return values[iterator.next()];
+    }
+
+    @Override
+    public void remove() {
+      iterator.remove();
+    }
+  }
+
+  /**
+   * Default capacity - in case no capacity was specified in the constructor
+   */
+  private static int defaultCapacity = 16;
+
+  /**
+   * Holds the base hash entries. if the capacity is 2^N, than the base hash
+   * holds 2^(N+1). It can hold
+   */
+  int[] baseHash;
+
+  /**
+   * The current capacity of the map. Always 2^N and never less than 16. We
+   * never use the zero index. It is needed to improve performance and is also
+   * used as "ground".
+   */
+  private int capacity;
+  /**
+   * All objects are being allocated at map creation. Those objects are "free"
+   * or empty. Whenever a new pair comes along, a pair is being "allocated" or
+   * taken from the free-linked list. as this is just a free list.
+   */
+  private int firstEmpty;
+
+  /**
+   * hashFactor is always (2^(N+1)) - 1. Used for faster hashing.
+   */
+  private int hashFactor;
+
+  /**
+   * This array holds the unique keys
+   */
+  int[] keys;
+
+  /**
+   * In case of collisions, we implement a float linked list of the colliding
+   * hash's with the following next[] and prev[]. Those are also used to store
+   * the "empty" list.
+   */
+  int[] next;
+
+  private int prev;
+
+  /**
+   * Number of currently objects in the map.
+   */
+  private int size;
+
+  /**
+   * This array holds the values
+   */
+  float[] values;
+
+  /**
+   * Constructs a map with default capacity.
+   */
+  public IntToFloatMap() {
+    this(defaultCapacity);
+  }
+
+  /**
+   * Constructs a map with given capacity. Capacity is adjusted to a native
+   * power of 2, with minimum of 16.
+   * 
+   * @param capacity
+   *            minimum capacity for the map.
+   */
+  public IntToFloatMap(int capacity) {
+    this.capacity = 16;
+    // Minimum capacity is 16..
+    while (this.capacity < capacity) {
+      // Multiply by 2 as long as we're still under the requested capacity
+      this.capacity <<= 1;
+    }
+
+    // As mentioned, we use the first index (0) as 'Ground', so we need the
+    // length of the arrays to be one more than the capacity
+    int arrayLength = this.capacity + 1;
+
+    this.values = new float[arrayLength];
+    this.keys = new int[arrayLength];
+    this.next = new int[arrayLength];
+
+    // Hash entries are twice as big as the capacity.
+    int baseHashSize = this.capacity << 1;
+
+    this.baseHash = new int[baseHashSize];
+
+    this.values[0] = GROUND;
+
+    // The has factor is 2^M - 1 which is used as an "AND" hashing operator.
+    // {@link #calcBaseHash()}
+    this.hashFactor = baseHashSize - 1;
+
+    this.size = 0;
+
+    clear();
+  }
+
+  /**
+   * Adds a pair to the map. Takes the first empty position from the
+   * empty-linked-list's head - {@link #firstEmpty}.
+   * 
+   * New pairs are always inserted to baseHash, and are followed by the old
+   * colliding pair.
+   * 
+   * @param key
+   *            integer which maps the given Object
+   * @param v
+   *            float value which is being mapped using the given key
+   */
+  private void prvt_put(int key, float v) {
+    // Hash entry to which the new pair would be inserted
+    int hashIndex = calcBaseHashIndex(key);
+
+    // 'Allocating' a pair from the "Empty" list.
+    int objectIndex = firstEmpty;
+
+    // Setting data
+    firstEmpty = next[firstEmpty];
+    values[objectIndex] = v;
+    keys[objectIndex] = key;
+
+    // Inserting the new pair as the first node in the specific hash entry
+    next[objectIndex] = baseHash[hashIndex];
+    baseHash[hashIndex] = objectIndex;
+
+    // Announcing a new pair was added!
+    ++size;
+  }
+
+  /**
+   * Calculating the baseHash index using the internal <code>hashFactor</code>
+   * .
+   */
+  protected int calcBaseHashIndex(int key) {
+    return key & hashFactor;
+  }
+
+  /**
+   * Empties the map. Generates the "Empty" space list for later allocation.
+   */
+  public void clear() {
+    // Clears the hash entries
+    Arrays.fill(this.baseHash, 0);
+
+    // Set size to zero
+    size = 0;
+
+    // Mark all array entries as empty. This is done with
+    // <code>firstEmpty</code> pointing to the first valid index (1 as 0 is
+    // used as 'Ground').
+    firstEmpty = 1;
+
+    // And setting all the <code>next[i]</code> to point at
+    // <code>i+1</code>.
+    for (int i = 1; i < this.capacity;) {
+      next[i] = ++i;
+    }
+
+    // Surly, the last one should point to the 'Ground'.
+    next[this.capacity] = 0;
+  }
+
+  /**
+   * Checks if a given key exists in the map.
+   * 
+   * @param key
+   *            that is checked against the map data.
+   * @return true if the key exists in the map. false otherwise.
+   */
+  public boolean containsKey(int key) {
+    return find(key) != 0;
+  }
+
+  /**
+   * Checks if the given value exists in the map.<br>
+   * This method iterates over the collection, trying to find an equal object.
+   * 
+   * @param value
+   *            float value that is checked against the map data.
+   * @return true if the value exists in the map, false otherwise.
+   */
+  public boolean containsValue(float value) {
+    for (FloatIterator iterator = iterator(); iterator.hasNext();) {
+      float d = iterator.next();
+      if (d == value) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Find the actual index of a given key.
+   * 
+   * @return index of the key. zero if the key wasn't found.
+   */
+  protected int find(int key) {
+    // Calculate the hash entry.
+    int baseHashIndex = calcBaseHashIndex(key);
+
+    // Start from the hash entry.
+    int localIndex = baseHash[baseHashIndex];
+
+    // while the index does not point to the 'Ground'
+    while (localIndex != 0) {
+      // returns the index found in case of of a matching key.
+      if (keys[localIndex] == key) {
+        return localIndex;
+      }
+        
+      // next the local index
+      localIndex = next[localIndex];
+    }
+
+    // If we got this far, it could only mean we did not find the key we
+    // were asked for. return 'Ground' index.
+    return 0;
+  }
+
+  /**
+   * Find the actual index of a given key with it's baseHashIndex.<br>
+   * Some methods use the baseHashIndex. If those call {@link #find} there's
+   * no need to re-calculate that hash.
+   * 
+   * @return the index of the given key, or 0 as 'Ground' if the key wasn't
+   *         found.
+   */
+  private int findForRemove(int key, int baseHashIndex) {
+    // Start from the hash entry.
+    this.prev = 0;
+    int index = baseHash[baseHashIndex];
+
+    // while the index does not point to the 'Ground'
+    while (index != 0) {
+      // returns the index found in case of of a matching key.
+      if (keys[index] == key) {
+        return index;
+      }
+
+      // next the local index
+      prev = index;
+      index = next[index];
+    }
+
+    // If we got this far, it could only mean we did not find the key we
+    // were asked for. return 'Ground' index.
+    this.prev = 0;
+    return 0;
+  }
+
+  /**
+   * Returns the value mapped with the given key.
+   * 
+   * @param key
+   *            int who's mapped object we're interested in.
+   * @return a float value mapped by the given key. float.NaN if the key wasn't found.
+   */
+  public float get(int key) {
+    return values[find(key)];
+  }
+
+  /**
+   * Grows the map. Allocates a new map of float the capacity, and
+   * fast-insert the old key-value pairs.
+   */
+  protected void grow() {
+    IntToFloatMap that = new IntToFloatMap(
+        this.capacity * 2);
+
+    // Iterates fast over the collection. Any valid pair is put into the new
+    // map without checking for duplicates or if there's enough space for
+    // it.
+    for (IndexIterator iterator = new IndexIterator(); iterator.hasNext();) {
+      int index = iterator.next();
+      that.prvt_put(this.keys[index], this.values[index]);
+    }
+
+    // Copy that's data into this.
+    this.capacity = that.capacity;
+    this.size = that.size;
+    this.firstEmpty = that.firstEmpty;
+    this.values = that.values;
+    this.keys = that.keys;
+    this.next = that.next;
+    this.baseHash = that.baseHash;
+    this.hashFactor = that.hashFactor;
+  }
+
+  /**
+   * 
+   * @return true if the map is empty. false otherwise.
+   */
+  public boolean isEmpty() {
+    return size == 0;
+  }
+
+  /**
+   * Returns a new iterator for the mapped float values.
+   */
+  public FloatIterator iterator() {
+    return new ValueIterator();
+  }
+
+  /** Returns an iterator on the map keys. */
+  public IntIterator keyIterator() {
+    return new KeyIterator();
+  }
+
+  /**
+   * Prints the baseHash array, used for debug purposes.
+   */
+  @SuppressWarnings("unused")
+  private String getBaseHashAsString() {
+    return Arrays.toString(this.baseHash);
+  }
+
+  /**
+   * Inserts the &lt;key,value&gt; pair into the map. If the key already exists,
+   * this method updates the mapped value to the given one, returning the old
+   * mapped value.
+   * 
+   * @return the old mapped value, or {@link Float#NaN} if the key didn't exist.
+   */
+  public float put(int key, float v) {
+    // Does key exists?
+    int index = find(key);
+
+    // Yes!
+    if (index != 0) {
+      // Set new data and exit.
+      float old = values[index];
+      values[index] = v;
+      return old;
+    }
+
+    // Is there enough room for a new pair?
+    if (size == capacity) {
+      // No? Than grow up!
+      grow();
+    }
+
+    // Now that everything is set, the pair can be just put inside with no
+    // worries.
+    prvt_put(key, v);
+
+    return Float.NaN;
+  }
+
+  /**
+   * Removes a &lt;key,value&gt; pair from the map and returns the mapped value,
+   * or {@link Float#NaN} if the none existed.
+   * 
+   * @param key used to find the value to remove
+   * @return the removed value or {@link Float#NaN} if none existed.
+   */
+  public float remove(int key) {
+    int baseHashIndex = calcBaseHashIndex(key);
+    int index = findForRemove(key, baseHashIndex);
+    if (index != 0) {
+      // If it is the first in the collision list, we should promote its
+      // next colliding element.
+      if (prev == 0) {
+        baseHash[baseHashIndex] = next[index];
+      }
+
+      next[prev] = next[index];
+      next[index] = firstEmpty;
+      firstEmpty = index;
+      --size;
+      return values[index];
+    }
+
+    return Float.NaN;
+  }
+
+  /**
+   * @return number of pairs currently in the map
+   */
+  public int size() {
+    return this.size;
+  }
+
+  /**
+   * Translates the mapped pairs' values into an array of Objects
+   * 
+   * @return a float array of all the values currently in the map.
+   */
+  public float[] toArray() {
+    int j = -1;
+    float[] array = new float[size];
+
+    // Iterates over the values, adding them to the array.
+    for (FloatIterator iterator = iterator(); iterator.hasNext();) {
+      array[++j] = iterator.next();
+    }
+    return array;
+  }
+
+  /**
+   * Translates the mapped pairs' values into an array of T
+   * 
+   * @param a
+   *            the array into which the elements of the list are to be
+   *            stored. If it is big enough use whatever space we need,
+   *            setting the one after the true data as {@link Float#NaN}.
+   * 
+   * @return an array containing the elements of the list, using the given
+   *         parameter if big enough, otherwise allocate an appropriate array
+   *         and return it.
+   * 
+   */
+  public float[] toArray(float[] a) {
+    int j = 0;
+    if (a.length < this.size()) {
+      a = new float[this.size()];
+    }
+
+    // Iterates over the values, adding them to the array.
+    for (FloatIterator iterator = iterator(); iterator.hasNext(); ++j) {
+      a[j] = iterator.next();
+    }
+
+    if (j < a.length) {
+      a[j] = Float.NaN;
+    }
+
+    return a;
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append('{');
+    IntIterator keyIterator = keyIterator();
+    while (keyIterator.hasNext()) {
+      int key = keyIterator.next();
+      sb.append(key);
+      sb.append('=');
+      sb.append(get(key));
+      if (keyIterator.hasNext()) {
+        sb.append(',');
+        sb.append(' ');
+      }
+    }
+    sb.append('}');
+    return sb.toString();
+  }
+  
+  @Override
+  public int hashCode() {
+    return getClass().hashCode() ^ size();
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    IntToFloatMap that = (IntToFloatMap)o;
+    if (that.size() != this.size()) {
+      return false;
+    }
+    
+    IntIterator it = keyIterator();
+    while (it.hasNext()) {
+      int key = it.next();
+      if (!that.containsKey(key)) {
+        return false;
+      }
+
+      float v1 = this.get(key);
+      float v2 = that.get(key);
+      if (Float.compare(v1, v2) != 0) {
+        return false;
+      }
+    }
+    return true;
+  }
+}
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java Thu Jan  3 12:56:27 2013
@@ -1,5 +1,6 @@
 package org.apache.lucene.util.encoding;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.OutputStream;
 
@@ -43,7 +44,7 @@ import java.io.OutputStream;
  * 
  * @lucene.experimental
  */
-public abstract class IntEncoder {
+public abstract class IntEncoder implements Closeable {
 
   protected OutputStream out = null;
 
@@ -68,6 +69,7 @@ public abstract class IntEncoder {
    * <b>NOTE:</b> overriding classes should make sure they either call
    * <code>super.close()</code> or close the output stream themselves.
    */
+  @Override
   public void close() throws IOException {
     if (out != null) {
       out.close();

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java Thu Jan  3 12:56:27 2013
@@ -15,7 +15,7 @@ import org.apache.lucene.analysis.MockTo
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.params.FacetRequest;
@@ -255,9 +255,8 @@ public abstract class FacetTestBase exte
   protected final void indexDoc(FacetIndexingParams iParams, RandomIndexWriter iw,
       TaxonomyWriter tw, String content, List<CategoryPath> categories) throws IOException {
     Document d = new Document();
-    CategoryDocumentBuilder builder = new CategoryDocumentBuilder(tw, iParams);
-    builder.setCategoryPaths(categories);
-    builder.build(d);
+    FacetFields facetFields = new FacetFields(tw, iParams);
+    facetFields.addFields(d, categories);
     d.add(new TextField("content", content, Field.Store.YES));
     iw.addDocument(d);
   }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java Thu Jan  3 12:56:27 2013
@@ -2,13 +2,14 @@ package org.apache.lucene.facet;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.FacetsCollector;
 import org.apache.lucene.facet.search.params.CountFacetRequest;
@@ -117,11 +118,9 @@ public class FacetTestUtils {
   
   public static void add(FacetIndexingParams iParams, RandomIndexWriter iw,
       TaxonomyWriter tw, String... strings) throws IOException {
-    ArrayList<CategoryPath> cps = new ArrayList<CategoryPath>();
-    CategoryPath cp = new CategoryPath(strings);
-    cps.add(cp);
     Document d = new Document();
-    new CategoryDocumentBuilder(tw, iParams).setCategoryPaths(cps).build(d);
+    FacetFields facetFields = new FacetFields(tw, iParams);
+    facetFields.addFields(d, Collections.singletonList(new CategoryPath(strings)));
     d.add(new TextField("content", "alpha", Field.Store.YES));
     iw.addDocument(d);
   }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/example/TestSimpleExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/example/TestSimpleExample.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/example/TestSimpleExample.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/example/TestSimpleExample.java Thu Jan  3 12:56:27 2013
@@ -34,7 +34,7 @@ import org.apache.lucene.facet.search.re
 public class TestSimpleExample extends LuceneTestCase {
 
   @Test
-  public void testSimple () throws Exception {
+  public void testSimple() throws Exception {
     ExampleResult res = new SimpleMain().runSimple();
     assertNotNull("Null result!", res);
     assertNotNull("Null facet result!", res.getFacetResults());
@@ -47,7 +47,7 @@ public class TestSimpleExample extends L
    * As result, facets that without drill down got count of 2 will now get a count of 1. 
    */
   @Test
-  public void testDrillDown () throws Exception {
+  public void testDrillDown() throws Exception {
     ExampleResult res = new SimpleMain().runDrillDown();
     assertNotNull("Null result!", res);
     assertNotNull("Null facet result!", res.getFacetResults());

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/OrdinalMappingReaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/OrdinalMappingReaderTest.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/OrdinalMappingReaderTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/OrdinalMappingReaderTest.java Thu Jan  3 12:56:27 2013
@@ -101,9 +101,8 @@ public class OrdinalMappingReaderTest ex
         int facetValue = asc? j: NUM_DOCS - j;
         categoryPaths.add(new CategoryPath("tag", Integer.toString(facetValue)));
       }
-      CategoryDocumentBuilder catBuilder = new CategoryDocumentBuilder(taxonomyWriter);
-      catBuilder.setCategoryPaths(categoryPaths);
-      catBuilder.build(doc);
+      FacetFields facetFields = new FacetFields(taxonomyWriter);
+      facetFields.addFields(doc, categoryPaths);
       writer.addDocument(doc);
     }    
     taxonomyWriter.close();

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java Thu Jan  3 12:56:27 2013
@@ -10,7 +10,7 @@ import org.apache.lucene.analysis.MockTo
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.index.params.PerDimensionIndexingParams;
@@ -89,8 +89,8 @@ public class DrillDownTest extends Lucen
       if (i % 5 == 0) { // 20
         paths.add(new CategoryPath("b"));
       }
-      CategoryDocumentBuilder builder = new CategoryDocumentBuilder(taxoWriter);
-      builder.setCategoryPaths(paths).build(doc);
+      FacetFields facetFields = new FacetFields(taxoWriter);
+      facetFields.addFields(doc, paths);
       writer.addDocument(doc);
     }
     

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java Thu Jan  3 12:56:27 2013
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.search.params.CountFacetRequest;
 import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
@@ -43,7 +43,7 @@ public class TestDemoFacets extends Luce
 
   private DirectoryTaxonomyWriter taxoWriter;
   private RandomIndexWriter writer;
-  private CategoryDocumentBuilder docBuilder;
+  private FacetFields docBuilder;
 
   private void add(String ... categoryPaths) throws IOException {
     Document doc = new Document();
@@ -52,8 +52,7 @@ public class TestDemoFacets extends Luce
     for(String categoryPath : categoryPaths) {
       paths.add(new CategoryPath(categoryPath, '/'));
     }
-    docBuilder.setCategoryPaths(paths);
-    docBuilder.build(doc);
+    docBuilder.addFields(doc, paths);
     writer.addDocument(doc);
   }
 
@@ -68,7 +67,7 @@ public class TestDemoFacets extends Luce
 
     // Reused across documents, to add the necessary facet
     // fields:
-    docBuilder = new CategoryDocumentBuilder(taxoWriter);
+    docBuilder = new FacetFields(taxoWriter);
 
     add("Author/Bob", "Publish Date/2010/10/15");
     add("Author/Lisa", "Publish Date/2010/10/20");

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java Thu Jan  3 12:56:27 2013
@@ -1,13 +1,13 @@
 package org.apache.lucene.facet.search;
 
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.StringField;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.params.ScoreFacetRequest;
 import org.apache.lucene.facet.search.results.FacetResult;
@@ -55,13 +55,12 @@ public class TestFacetsCollector extends
     IndexWriter iw = new IndexWriter(indexDir, new IndexWriterConfig(
         TEST_VERSION_CURRENT, new KeywordAnalyzer()));
 
-    CategoryDocumentBuilder cdb = new CategoryDocumentBuilder(taxonomyWriter);
-    Iterable<CategoryPath> cats = Arrays.asList(new CategoryPath("a"));
+    FacetFields facetFields = new FacetFields(taxonomyWriter);
     for(int i = atLeast(2000); i > 0; --i) {
       Document doc = new Document();
       doc.add(new StringField("f", "v", Store.NO));
-      cdb.setCategoryPaths(cats);
-      iw.addDocument(cdb.build(doc));
+      facetFields.addFields(doc, Collections.singletonList(new CategoryPath("a")));
+      iw.addDocument(doc);
     }
     
     taxonomyWriter.close();

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java Thu Jan  3 12:56:27 2013
@@ -2,13 +2,14 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.params.CountFacetRequest;
 import org.apache.lucene.facet.search.params.FacetRequest;
@@ -321,11 +322,9 @@ public class TestTopKInEachNodeResultHan
 
   private void prvt_add(FacetIndexingParams iParams, RandomIndexWriter iw,
       TaxonomyWriter tw, String... strings) throws IOException {
-    ArrayList<CategoryPath> cps = new ArrayList<CategoryPath>();
-    CategoryPath cp = new CategoryPath(strings);
-    cps.add(cp);
     Document d = new Document();
-    new CategoryDocumentBuilder(tw, iParams).setCategoryPaths(cps).build(d);
+    FacetFields facetFields = new FacetFields(tw, iParams);
+    facetFields.addFields(d, Collections.singletonList(new CategoryPath(strings)));
     d.add(new TextField("content", "alpha", Field.Store.YES));
     iw.addDocument(d);
   }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java Thu Jan  3 12:56:27 2013
@@ -2,7 +2,7 @@ package org.apache.lucene.facet.search;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -15,7 +15,7 @@ import org.apache.lucene.facet.example.E
 import org.apache.lucene.facet.example.TestMultiCLExample;
 import org.apache.lucene.facet.example.multiCL.MultiCLIndexer;
 import org.apache.lucene.facet.example.multiCL.MultiCLSearcher;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.TotalFacetCounts.CreationType;
 import org.apache.lucene.facet.search.results.FacetResult;
@@ -88,10 +88,10 @@ public class TestTotalFacetCountsCache e
   /** Utility method to add a document and facets to an index/taxonomy. */
   static void addFacets(FacetIndexingParams iParams, IndexWriter iw,
                         TaxonomyWriter tw, String... strings) throws IOException {
-    ArrayList<CategoryPath> cps = new ArrayList<CategoryPath>();
-    cps.add(new CategoryPath(strings));
-    CategoryDocumentBuilder builder = new CategoryDocumentBuilder(tw, iParams);
-    iw.addDocument(builder.setCategoryPaths(cps).build(new Document()));
+    Document doc = new Document();
+    FacetFields facetFields = new FacetFields(tw, iParams);
+    facetFields.addFields(doc, Collections.singletonList(new CategoryPath(strings)));
+    iw.addDocument(doc);
   }
 
   /** Clears the cache and sets its size to one. */
@@ -143,7 +143,6 @@ public class TestTotalFacetCountsCache e
     MockDirectoryWrapper indexDir = new MockDirectoryWrapper(random(), slowIndexDir);
     SlowRAMDirectory slowTaxoDir = new SlowRAMDirectory(-1, random());
     MockDirectoryWrapper taxoDir = new MockDirectoryWrapper(random(), slowTaxoDir);
-    
 
     // Index documents without the "slowness"
     MultiCLIndexer.index(indexDir, taxoDir);

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/associations/AssociationsFacetRequestTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/associations/AssociationsFacetRequestTest.java?rev=1428320&r1=1426915&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/associations/AssociationsFacetRequestTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/associations/AssociationsFacetRequestTest.java Thu Jan  3 12:56:27 2013
@@ -1,20 +1,18 @@
-package org.apache.lucene.facet.search.association;
+package org.apache.lucene.facet.search.associations;
 
 import java.util.List;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.enhancements.EnhancementsDocumentBuilder;
-import org.apache.lucene.facet.enhancements.association.AssociationEnhancement;
-import org.apache.lucene.facet.enhancements.association.AssociationFloatProperty;
-import org.apache.lucene.facet.enhancements.association.AssociationIntProperty;
-import org.apache.lucene.facet.enhancements.params.EnhancementsIndexingParams;
-import org.apache.lucene.facet.index.CategoryContainer;
+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.search.FacetsCollector;
 import org.apache.lucene.facet.search.params.FacetSearchParams;
-import org.apache.lucene.facet.search.params.association.AssociationFloatSumFacetRequest;
-import org.apache.lucene.facet.search.params.association.AssociationIntSumFacetRequest;
+import org.apache.lucene.facet.search.params.associations.AssociationFloatSumFacetRequest;
+import org.apache.lucene.facet.search.params.associations.AssociationIntSumFacetRequest;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
@@ -70,20 +68,19 @@ public class AssociationsFacetRequestTes
     
     TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
     
-    EnhancementsDocumentBuilder builder = new EnhancementsDocumentBuilder(
-        taxoWriter, new EnhancementsIndexingParams(new AssociationEnhancement()));
+    AssociationsFacetFields assocFacetFields = new AssociationsFacetFields(taxoWriter);
     
     // index documents, 50% have only 'b' and all have 'a'
     for (int i = 0; i < 100; i++) {
       Document doc = new Document();
-      CategoryContainer container = new CategoryContainer();
-      container.addCategory(aint, new AssociationIntProperty(2));
-      container.addCategory(afloat, new AssociationFloatProperty(0.5f));
+      CategoryAssociationsContainer associations = new CategoryAssociationsContainer();
+      associations.setAssociation(aint, new CategoryIntAssociation(2));
+      associations.setAssociation(afloat, new CategoryFloatAssociation(0.5f));
       if (i % 2 == 0) { // 50
-        container.addCategory(bint, new AssociationIntProperty(3));
-        container.addCategory(bfloat, new AssociationFloatProperty(0.2f));
+        associations.setAssociation(bint, new CategoryIntAssociation(3));
+        associations.setAssociation(bfloat, new CategoryFloatAssociation(0.2f));
       }
-      builder.setCategories(container).build(doc);
+      assocFacetFields.addFields(doc, associations);
       writer.addDocument(doc);
     }
     

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiIteratorsPerCLParamsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiIteratorsPerCLParamsTest.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiIteratorsPerCLParamsTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiIteratorsPerCLParamsTest.java Thu Jan  3 12:56:27 2013
@@ -8,7 +8,7 @@ import java.util.List;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.CategoryListIterator;
@@ -171,11 +171,11 @@ public class MultiIteratorsPerCLParamsTe
         newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)));
     TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
 
+    FacetFields facetFields = new FacetFields(taxoWriter, iParams);
     for (CategoryPath[] categories : perDocCategories) {
-      writer.addDocument(new CategoryDocumentBuilder(taxoWriter, iParams)
-          .setCategoryPaths(Arrays.asList(categories)).build(
-              new Document()));
-
+      Document doc = new Document();
+      facetFields.addFields(doc, Arrays.asList(categories));
+      writer.addDocument(doc);
     }
     taxoWriter.commit();
     writer.commit();

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/sampling/OversampleWithDepthTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/sampling/OversampleWithDepthTest.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/sampling/OversampleWithDepthTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/sampling/OversampleWithDepthTest.java Thu Jan  3 12:56:27 2013
@@ -1,11 +1,11 @@
 package org.apache.lucene.facet.search.sampling;
 
 import java.io.IOException;
-import java.util.ArrayList;
+import java.util.Collections;
 
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.search.FacetsAccumulator;
 import org.apache.lucene.facet.search.FacetsCollector;
 import org.apache.lucene.facet.search.params.CountFacetRequest;
@@ -100,14 +100,12 @@ public class OversampleWithDepthTest ext
     IndexWriter w = new IndexWriter(indexDir, iwc);
     TaxonomyWriter tw = new DirectoryTaxonomyWriter(taxoDir);
     
-    CategoryDocumentBuilder cdb = new CategoryDocumentBuilder(tw);
-    ArrayList<CategoryPath> categoryPaths = new ArrayList<CategoryPath>(1);
-    
+    FacetFields facetFields = new FacetFields(tw);
     for (int i = 0; i < 100; i++) {
-      categoryPaths.clear();
-      categoryPaths.add(new CategoryPath("root",Integer.toString(i / 10), Integer.toString(i)));
-      cdb.setCategoryPaths(categoryPaths);
-      w.addDocument(cdb.build(new Document()));
+      Document doc = new Document();
+      CategoryPath cp = new CategoryPath("root",Integer.toString(i / 10), Integer.toString(i));
+      facetFields.addFields(doc, Collections.singletonList(cp));
+      w.addDocument(doc);
     }
     IOUtils.close(tw, w);
   }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestConcurrentFacetedIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestConcurrentFacetedIndexing.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestConcurrentFacetedIndexing.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestConcurrentFacetedIndexing.java Thu Jan  3 12:56:27 2013
@@ -8,7 +8,7 @@ import java.util.concurrent.ConcurrentHa
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.index.CategoryDocumentBuilder;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.writercache.TaxonomyWriterCache;
 import org.apache.lucene.facet.taxonomy.writercache.cl2o.Cl2oTaxonomyWriterCache;
@@ -94,7 +94,7 @@ public class TestConcurrentFacetedIndexi
 
     for (int i = 0; i < indexThreads.length; i++) {
       indexThreads[i] = new Thread() {
-        private final CategoryDocumentBuilder cdb = new CategoryDocumentBuilder(tw);
+        private final FacetFields facetFields = new FacetFields(tw);
         
         @Override
         public void run() {
@@ -115,8 +115,7 @@ public class TestConcurrentFacetedIndexi
                   --level;
                 }
               }
-              cdb.setCategoryPaths(cats);
-              cdb.build(doc);
+              facetFields.addFields(doc, cats);
               iw.addDocument(doc);
             } catch (IOException e) {
               throw new RuntimeException(e);

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/IntHashSetTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/IntHashSetTest.java?rev=1428320&r1=1428319&r2=1428320&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/IntHashSetTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/IntHashSetTest.java Thu Jan  3 12:56:27 2013
@@ -121,10 +121,8 @@ public class IntHashSetTest extends Luce
       }
     }
     assertEquals(set2.size(), set1.size());
-    int i = 0;
     for (int value : set2) {
       assertTrue(set1.contains(value));
-      i++;
     }
   }
 

Added: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/IntToFloatMapTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/IntToFloatMapTest.java?rev=1428320&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/IntToFloatMapTest.java (added)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/IntToFloatMapTest.java Thu Jan  3 12:56:27 2013
@@ -0,0 +1,272 @@
+package org.apache.lucene.util.collections;
+
+import org.junit.Test;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.collections.FloatIterator;
+import org.apache.lucene.util.collections.IntIterator;
+import org.apache.lucene.util.collections.IntToFloatMap;
+
+import java.util.HashSet;
+import java.util.Random;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public class IntToFloatMapTest extends LuceneTestCase {
+  private static void assertGround(float value) {
+    assertEquals(IntToFloatMap.GROUND, value, Float.MAX_VALUE);
+  }
+  
+  @Test
+  public void test0() {
+    IntToFloatMap map = new IntToFloatMap();
+
+    assertGround(map.get(0));
+    
+    for (int i = 0; i < 100; ++i) {
+      int value = 100 + i;
+      assertFalse(map.containsValue(value));
+      map.put(i, value);
+      assertTrue(map.containsValue(value));
+      assertNotNull(map.get(i));
+    }
+
+    assertEquals(100, map.size());
+    for (int i = 0; i < 100; ++i) {
+      assertTrue(map.containsKey(i));
+      assertEquals(100 + i, map.get(i), Float.MAX_VALUE);
+
+    }
+
+    for (int i = 10; i < 90; ++i) {
+      map.remove(i);
+      assertGround(map.get(i));
+    }
+
+    assertEquals(20, map.size());
+    for (int i = 0; i < 100; ++i) {
+      assertEquals(map.containsKey(i), !(i >= 10 && i < 90));
+    }
+
+    for (int i = 5; i < 85; ++i) {
+      map.put(i, Integer.valueOf(5 + i));
+    }
+    assertEquals(95, map.size());
+    for (int i = 0; i < 100; ++i) {
+      assertEquals(map.containsKey(i), !(i >= 85 && i < 90));
+    }
+    for (int i = 0; i < 5; ++i) {
+      assertEquals(map.get(i), (100 + i), Float.MAX_VALUE);
+    }
+    for (int i = 5; i < 85; ++i) {
+      assertEquals(map.get(i), (5 + i), Float.MAX_VALUE);
+    }
+    for (int i = 90; i < 100; ++i) {
+      assertEquals(map.get(i), (100 + i), Float.MAX_VALUE);
+    }
+  }
+
+  @Test
+  public void test1() {
+    IntToFloatMap map = new IntToFloatMap();
+
+    for (int i = 0; i < 100; ++i) {
+      map.put(i, Integer.valueOf(100 + i));
+    }
+    
+    HashSet<Float> set = new HashSet<Float>();
+    
+    for (FloatIterator iterator = map.iterator(); iterator.hasNext();) {
+      set.add(iterator.next());
+    }
+
+    assertEquals(set.size(), map.size());
+    for (int i = 0; i < 100; ++i) {
+      assertTrue(set.contains(Float.valueOf(100+i)));
+    }
+
+    set.clear();
+    for (FloatIterator iterator = map.iterator(); iterator.hasNext();) {
+      float d = iterator.next();
+      if (d % 2 == 1) {
+        iterator.remove();
+        continue;
+      }
+      set.add(d);
+    }
+    assertEquals(set.size(), map.size());
+    for (int i = 0; i < 100; i+=2) {
+      assertTrue(set.contains(Float.valueOf(100+i)));
+    }
+  }
+  
+  @Test
+  public void test2() {
+    IntToFloatMap map = new IntToFloatMap();
+
+    assertTrue(map.isEmpty());
+    assertGround(map.get(0));
+    for (int i = 0; i < 128; ++i) {
+      int value = i * 4096;
+      assertFalse(map.containsValue(value));
+      map.put(i, value);
+      assertTrue(map.containsValue(value));
+      assertNotNull(map.get(i));
+      assertFalse(map.isEmpty());
+    }
+
+    assertEquals(128, map.size());
+    for (int i = 0; i < 128; ++i) {
+      assertTrue(map.containsKey(i));
+      assertEquals(i * 4096, map.get(i), Float.MAX_VALUE);
+    }
+    
+    for (int i = 0 ; i < 200; i+=2) {
+      map.remove(i);
+    }
+    assertEquals(64, map.size());
+    for (int i = 1; i < 128; i+=2) {
+      assertTrue(map.containsKey(i));
+      assertEquals(i * 4096, map.get(i), Float.MAX_VALUE);
+      map.remove(i);
+    }
+    assertTrue(map.isEmpty());
+  }
+  
+  @Test
+  public void test3() {
+    IntToFloatMap map = new IntToFloatMap();
+    int length = 100;
+    for (int i = 0; i < length; ++i) {
+      map.put(i*64, 100 + i);
+    }
+    HashSet<Integer> keySet = new HashSet<Integer>();
+    for (IntIterator iit = map.keyIterator(); iit.hasNext(); ) {
+      keySet.add(iit.next());
+    }
+    assertEquals(length, keySet.size());
+    for (int i = 0; i < length; ++i) {
+      assertTrue(keySet.contains(i * 64));
+    }
+    
+    HashSet<Float> valueSet = new HashSet<Float>();
+    for (FloatIterator iit = map.iterator(); iit.hasNext(); ) {
+      valueSet.add(iit.next());
+    }
+    assertEquals(length, valueSet.size());
+    float[] array = map.toArray();
+    assertEquals(length, array.length);
+    for (float value: array) {
+      assertTrue(valueSet.contains(value));
+    }
+    
+    float[] array2 = new float[80];
+    array2 = map.toArray(array2);
+    assertEquals(length, array2.length);
+    for (float value: array2) {
+      assertTrue(valueSet.contains(value));
+    }
+    
+    float[] array3 = new float[120];
+    array3 = map.toArray(array3);
+    for (int i = 0 ;i < length; ++i) {
+      assertTrue(valueSet.contains(array3[i]));
+    }
+    
+    for (int i = 0; i < length; ++i) {
+      assertTrue(map.containsValue(i + 100));
+      assertTrue(map.containsKey(i*64));
+    }
+    
+    for (IntIterator iit = map.keyIterator(); iit.hasNext(); ) {
+      iit.next();
+      iit.remove();
+    }
+    assertTrue(map.isEmpty());
+    assertEquals(0, map.size());
+    
+  }
+
+  // now with random data.. and lots of it
+  @Test
+  public void test4() {
+    IntToFloatMap map = new IntToFloatMap();
+    int length = ArrayHashMapTest.RANDOM_TEST_NUM_ITERATIONS;
+    // for a repeatable random sequence
+    long seed = random().nextLong();
+    Random random = new Random(seed);
+    
+    for (int i = 0; i < length; ++i) {
+      int value = random.nextInt(Integer.MAX_VALUE);
+      map.put(i*128, value);
+    }
+
+    assertEquals(length, map.size());
+
+    // now repeat
+    random.setSeed(seed);
+
+    for (int i = 0; i < length; ++i) {
+      int value = random.nextInt(Integer.MAX_VALUE);
+      assertTrue(map.containsValue(value));
+      assertTrue(map.containsKey(i*128));
+      assertEquals(0, Float.compare(value, map.remove(i*128)));
+    }
+    assertEquals(0, map.size());
+    assertTrue(map.isEmpty());
+  }
+  
+  @Test
+  public void testEquals() {
+    IntToFloatMap map1 = new IntToFloatMap(100);
+    IntToFloatMap map2 = new IntToFloatMap(100);
+    assertEquals("Empty maps should be equal", map1, map2);
+    assertEquals("hashCode() for empty maps should be equal", 
+        map1.hashCode(), map2.hashCode());
+    
+    for (int i = 0; i < 100; ++i) {
+      map1.put(i, Float.valueOf(1f/i));
+      map2.put(i, Float.valueOf(1f/i));
+    }
+    assertEquals("Identical maps should be equal", map1, map2);
+    assertEquals("hashCode() for identical maps should be equal", 
+        map1.hashCode(), map2.hashCode());
+
+    for (int i = 10; i < 20; i++) {
+      map1.remove(i);
+    }
+    assertFalse("Different maps should not be equal", map1.equals(map2));
+    
+    for (int i = 19; i >=10; --i) {
+      map2.remove(i);
+    }
+    assertEquals("Identical maps should be equal", map1, map2);
+    assertEquals("hashCode() for identical maps should be equal", 
+        map1.hashCode(), map2.hashCode());
+    
+    map1.put(-1,-1f);
+    map2.put(-1,-1.1f);
+    assertFalse("Different maps should not be equal", map1.equals(map2));
+    
+    map2.put(-1,-1f);
+    assertEquals("Identical maps should be equal", map1, map2);
+    assertEquals("hashCode() for identical maps should be equal", 
+        map1.hashCode(), map2.hashCode());
+  }
+  
+}