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/06 20:07:45 UTC

svn commit: r1429570 [2/2] - in /lucene/dev/trunk/lucene: ./ benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ facet/src/java/org/apache/lucene/facet/associations/ facet/src/java/org/apache/lucene/facet/index/ facet/src/java/org/apache/lucen...

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CollisionMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CollisionMap.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CollisionMap.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CollisionMap.java Sun Jan  6 19:07:44 2013
@@ -1,6 +1,5 @@
 package org.apache.lucene.facet.taxonomy.writercache.cl2o;
 
-import java.io.IOException;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
@@ -108,25 +107,11 @@ public class CollisionMap {
     int bucketIndex = indexFor(hash, this.capacity);
     Entry e = this.entries[bucketIndex];
 
-    while (e != null && !(hash == e.hash && label.equalsToSerialized(this.labelRepository, e.offset))) { 
+    while (e != null && !(hash == e.hash && CategoryPathUtils.equalsToSerialized(label, labelRepository, e.offset))) { 
       e = e.next;
     }
     if (e == null) {
-      return LabelToOrdinal.InvalidOrdinal;
-    }
-
-    return e.cid;
-  }
-
-  public int get(CategoryPath label, int prefixLen, int hash) {
-    int bucketIndex = indexFor(hash, this.capacity);
-    Entry e = this.entries[bucketIndex];
-
-    while (e != null && !(hash == e.hash && label.equalsToSerialized(prefixLen, this.labelRepository, e.offset))) { 
-      e = e.next;
-    }
-    if (e == null) {
-      return LabelToOrdinal.InvalidOrdinal;
+      return LabelToOrdinal.INVALID_ORDINAL;
     }
 
     return e.cid;
@@ -135,47 +120,22 @@ public class CollisionMap {
   public int addLabel(CategoryPath label, int hash, int cid) {
     int bucketIndex = indexFor(hash, this.capacity);
     for (Entry e = this.entries[bucketIndex]; e != null; e = e.next) {
-      if (e.hash == hash && label.equalsToSerialized(this.labelRepository, e.offset)) {
-        return e.cid;
-      }
-    }
-
-    // new string; add to label repository
-    int offset = this.labelRepository.length();
-    try {
-      label.serializeAppendTo(labelRepository);
-    } catch (IOException e) {
-      // can't happen, because labelRepository.append() doesn't throw an exception
-    }
-
-    addEntry(offset, cid, hash, bucketIndex);
-    return cid;
-  }
-
-  public int addLabel(CategoryPath label, int prefixLen, int hash, int cid) {
-    int bucketIndex = indexFor(hash, this.capacity);
-    for (Entry e = this.entries[bucketIndex]; e != null; e = e.next) {
-      if (e.hash == hash && label.equalsToSerialized(prefixLen, this.labelRepository, e.offset)) {
+      if (e.hash == hash && CategoryPathUtils.equalsToSerialized(label, labelRepository, e.offset)) {
         return e.cid;
       }
     }
 
     // new string; add to label repository
-    int offset = this.labelRepository.length();
-    try {
-      label.serializeAppendTo(prefixLen, labelRepository);
-    } catch (IOException e) {
-      // can't happen, because labelRepository.append() doesn't throw an exception
-    }
-
+    int offset = labelRepository.length();
+    CategoryPathUtils.serialize(label, labelRepository);
     addEntry(offset, cid, hash, bucketIndex);
     return cid;
   }
 
   /**
-   * This method does not check if the same value is already
-   * in the map because we pass in an char-array offset, so
-   * so we now that we're in resize-mode here. 
+   * This method does not check if the same value is already in the map because
+   * we pass in an char-array offset, so so we now that we're in resize-mode
+   * here.
    */
   public void addLabelOffset(int hash, int offset, int cid) {
     int bucketIndex = indexFor(hash, this.capacity);

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CompactLabelToOrdinal.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CompactLabelToOrdinal.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CompactLabelToOrdinal.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CompactLabelToOrdinal.java Sun Jan  6 19:07:44 2013
@@ -29,8 +29,6 @@ import java.util.Iterator;
 
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 
-// TODO: maybe this could use an FST instead...
-
 /**
  * This is a very efficient LabelToOrdinal implementation that uses a
  * CharBlockArray to store all labels and a configurable number of HashArrays to
@@ -59,8 +57,8 @@ public class CompactLabelToOrdinal exten
 
   public static final float DefaultLoadFactor = 0.15f;
 
-  static final char TerminatorChar = 0xffff;
-  private static final int Collision = -5;
+  static final char TERMINATOR_CHAR = 0xffff;
+  private static final int COLLISION = -5;
 
   private HashArray[] hashArrays;
   private CollisionMap collisionMap;
@@ -103,9 +101,7 @@ public class CompactLabelToOrdinal exten
 
   private void init() {
     labelRepository = new CharBlockArray();
-    try {
-      new CategoryPath().serializeAppendTo(labelRepository);
-    } catch (IOException e) { }  //can't happen 
+    CategoryPathUtils.serialize(CategoryPath.EMPTY, labelRepository);
 
     int c = this.capacity;
     for (int i = 0; i < this.hashArrays.length; i++) {
@@ -116,7 +112,7 @@ public class CompactLabelToOrdinal exten
 
   @Override
   public void addLabel(CategoryPath label, int ordinal) {
-    if (this.collisionMap.size() > this.threshold) {
+    if (collisionMap.size() > threshold) {
       grow();
     }
 
@@ -127,43 +123,22 @@ public class CompactLabelToOrdinal exten
       }
     }
 
-    int prevVal = this.collisionMap.addLabel(label, hash, ordinal);
-    if (prevVal != ordinal) {
-      throw new IllegalArgumentException("Label already exists: " +
-          label.toString('/') + " prev ordinal " + prevVal);
-    }
-  }
-
-  @Override
-  public void addLabel(CategoryPath label, int prefixLen, int ordinal) {
-    if (this.collisionMap.size() > this.threshold) {
-      grow();
-    }
-
-    int hash = CompactLabelToOrdinal.stringHashCode(label, prefixLen);
-    for (int i = 0; i < this.hashArrays.length; i++) {
-      if (addLabel(this.hashArrays[i], label, prefixLen, hash, ordinal)) {
-        return;
-      }
-    }
-
-    int prevVal = this.collisionMap.addLabel(label, prefixLen, hash, ordinal);
+    int prevVal = collisionMap.addLabel(label, hash, ordinal);
     if (prevVal != ordinal) {
-      throw new IllegalArgumentException("Label already exists: " +
-          label.toString('/', prefixLen) + " prev ordinal " + prevVal);
+      throw new IllegalArgumentException("Label already exists: " + label.toString('/') + " prev ordinal " + prevVal);
     }
   }
 
   @Override
   public int getOrdinal(CategoryPath label) {
     if (label == null) {
-      return LabelToOrdinal.InvalidOrdinal;
+      return LabelToOrdinal.INVALID_ORDINAL;
     }
 
     int hash = CompactLabelToOrdinal.stringHashCode(label);
     for (int i = 0; i < this.hashArrays.length; i++) {
       int ord = getOrdinal(this.hashArrays[i], label, hash);
-      if (ord != Collision) {
+      if (ord != COLLISION) {
         return ord;
       }
     }
@@ -171,23 +146,6 @@ public class CompactLabelToOrdinal exten
     return this.collisionMap.get(label, hash);
   }
 
-  @Override
-  public int getOrdinal(CategoryPath label, int prefixLen) {
-    if (label == null) {
-      return LabelToOrdinal.InvalidOrdinal;
-    }
-
-    int hash = CompactLabelToOrdinal.stringHashCode(label, prefixLen);
-    for (int i = 0; i < this.hashArrays.length; i++) {
-      int ord = getOrdinal(this.hashArrays[i], label, prefixLen, hash);
-      if (ord != Collision) {
-        return ord;
-      }
-    }
-
-    return this.collisionMap.get(label, prefixLen, hash);
-  }
-
   private void grow() {
     HashArray temp = this.hashArrays[this.hashArrays.length - 1];
 
@@ -241,39 +199,13 @@ public class CompactLabelToOrdinal exten
     }
   }
 
-  private boolean addLabel(HashArray a, CategoryPath label, int hash,
-                            int ordinal) {
+  private boolean addLabel(HashArray a, CategoryPath label, int hash, int ordinal) {
     int index = CompactLabelToOrdinal.indexFor(hash, a.offsets.length);
     int offset = a.offsets[index];
 
     if (offset == 0) {
       a.offsets[index] = this.labelRepository.length();
-      try {
-        label.serializeAppendTo(this.labelRepository);
-      } catch (IOException e) {
-        // can't happen - LabelRepository.append() never throws an
-        // exception
-      }
-      a.cids[index] = ordinal;
-      return true;
-    }
-
-    return false;
-  }
-
-  private boolean addLabel(HashArray a, CategoryPath label, int prefixLen,
-                            int hash, int ordinal) {
-    int index = CompactLabelToOrdinal.indexFor(hash, a.offsets.length);
-    int offset = a.offsets[index];
-
-    if (offset == 0) {
-      a.offsets[index] = this.labelRepository.length();
-      try {
-        label.serializeAppendTo(prefixLen, this.labelRepository);
-      } catch (IOException e) {
-        // can't happen - LabelRepository.append() never throws an
-        // exception
-      }
+      CategoryPathUtils.serialize(label, labelRepository);
       a.cids[index] = ordinal;
       return true;
     }
@@ -313,43 +245,23 @@ public class CompactLabelToOrdinal exten
 
   private int getOrdinal(HashArray a, CategoryPath label, int hash) {
     if (label == null) {
-      return LabelToOrdinal.InvalidOrdinal;
-    }
-
-    int index = CompactLabelToOrdinal.indexFor(hash, a.offsets.length);
-    int offset = a.offsets[index];
-    if (offset == 0) {
-      return LabelToOrdinal.InvalidOrdinal;
-    }
-
-    if (label.equalsToSerialized(labelRepository, offset)) {
-      return a.cids[index];
-    }
-
-    return Collision;
-  }
-
-  private int getOrdinal(HashArray a, CategoryPath label, int prefixLen, int hash) {
-    if (label == null) {
-      return LabelToOrdinal.InvalidOrdinal;
+      return LabelToOrdinal.INVALID_ORDINAL;
     }
 
-    int index = CompactLabelToOrdinal.indexFor(hash, a.offsets.length);
+    int index = indexFor(hash, a.offsets.length);
     int offset = a.offsets[index];
     if (offset == 0) {
-      return LabelToOrdinal.InvalidOrdinal;
+      return LabelToOrdinal.INVALID_ORDINAL;
     }
 
-    if (label.equalsToSerialized(prefixLen, labelRepository, offset)) {
+    if (CategoryPathUtils.equalsToSerialized(label, labelRepository, offset)) {
       return a.cids[index];
     }
 
-    return Collision;
+    return COLLISION;
   }
 
-  /**
-   * Returns index for hash code h.
-   */
+  /** Returns index for hash code h. */
   static int indexFor(int h, int length) {
     return h & (length - 1);
   }
@@ -378,22 +290,10 @@ public class CompactLabelToOrdinal exten
 
   }
 
-  static int stringHashCode(CategoryPath label, int prefixLen) {
-    int hash = label.hashCode(prefixLen);
-
-    hash = hash ^ ((hash >>> 20) ^ (hash >>> 12));
-    hash = hash ^ (hash >>> 7) ^ (hash >>> 4);
-
-    return hash;
-
-  }
-
   static int stringHashCode(CharBlockArray labelRepository, int offset) {
-    int hash = CategoryPath.hashCodeOfSerialized(labelRepository, offset);
-
+    int hash = CategoryPathUtils.hashCodeOfSerialized(labelRepository, offset);
     hash = hash ^ ((hash >>> 20) ^ (hash >>> 12));
     hash = hash ^ (hash >>> 7) ^ (hash >>> 4);
-
     return hash;
   }
 
@@ -495,25 +395,16 @@ public class CompactLabelToOrdinal exten
       // that array offsets will work).  Since the initial file is machine 
       // generated, I think this should be OK.
       while (offset < l2o.labelRepository.length()) {
-        // First component is numcomponents, so we initialize the hash
-        // to this
-        int ncomponents = l2o.labelRepository.charAt(offset++);
-        int hash = ncomponents;
-        // If ncomponents is 0, then we are done?
-        if (ncomponents != 0) {
-
-          // usedchars is always the last member of the 'ends' array
-          // in serialization. Rather than rebuild the entire array,
-          // assign usedchars to the last value we read in. This will
-          // be slightly more memory efficient.
-          int usedchars = 0;
-          for (int i = 0; i < ncomponents; i++) {
-            usedchars = l2o.labelRepository.charAt(offset++);
-            hash = hash * 31 + usedchars;
-          }
-          // Hash the usedchars for this label
-          for (int i = 0; i < usedchars; i++) {
-            hash = hash * 31 + l2o.labelRepository.charAt(offset++);
+        // identical code to CategoryPath.hashFromSerialized. since we need to
+        // advance offset, we cannot call the method directly. perhaps if we
+        // could pass a mutable Integer or something...
+        int length = (short) l2o.labelRepository.charAt(offset++);
+        int hash = length;
+        if (length != 0) {
+          for (int i = 0; i < length; i++) {
+            int len = (short) l2o.labelRepository.charAt(offset++);
+            hash = hash * 31 + l2o.labelRepository.subSequence(offset, offset + len).hashCode();
+            offset += len;
           }
         }
         // Now that we've hashed the components of the label, do the

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/LabelToOrdinal.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/LabelToOrdinal.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/LabelToOrdinal.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/LabelToOrdinal.java Sun Jan  6 19:07:44 2013
@@ -27,7 +27,7 @@ import org.apache.lucene.facet.taxonomy.
 public abstract class LabelToOrdinal {
 
   protected int counter;
-  public static final int InvalidOrdinal = -2;
+  public static final int INVALID_ORDINAL = -2;
 
   /**
    * return the maximal Ordinal assigned so far
@@ -52,22 +52,9 @@ public abstract class LabelToOrdinal {
   public abstract void addLabel(CategoryPath label, int ordinal);
 
   /**
-   * Adds a new label if its not yet in the table.
-   * Throws an {@link IllegalArgumentException} if the same label with
-   * a different ordinal was previoulsy added to this table.
-   */
-  public abstract void addLabel(CategoryPath label, int prefixLen, int ordinal);
-
-  /**
    * @return the ordinal assigned to the given label, 
-   * or {@link #InvalidOrdinal} if the label cannot be found in this table.
+   * or {@link #INVALID_ORDINAL} if the label cannot be found in this table.
    */
   public abstract int getOrdinal(CategoryPath label);
 
-  /**
-   * @return the ordinal assigned to the given label, 
-   * or {@link #InvalidOrdinal} if the label cannot be found in this table.
-   */
-  public abstract int getOrdinal(CategoryPath label, int prefixLen);
-
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/LruTaxonomyWriterCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/LruTaxonomyWriterCache.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/LruTaxonomyWriterCache.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/LruTaxonomyWriterCache.java Sun Jan  6 19:07:44 2013
@@ -87,23 +87,6 @@ public class LruTaxonomyWriterCache impl
   }
 
   @Override
-  public synchronized int get(CategoryPath categoryPath, int length) {
-    if (length<0 || length>categoryPath.length()) {
-      length = categoryPath.length();
-    }
-    // TODO (Facet): unfortunately, we make a copy here! we can avoid part of
-    // the copy by creating a wrapper object (but this still creates a new
-    // object). A better implementation of the cache would not use Java's
-    // hash table, but rather some other hash table we can control, and
-    // pass the length parameter into it...
-    Integer res = cache.get(new CategoryPath(categoryPath, length));
-    if (res==null) {
-      return -1;
-    }
-    return res.intValue();
-  }
-
-  @Override
   public synchronized boolean put(CategoryPath categoryPath, int ordinal) {
     boolean ret = cache.put(categoryPath, new Integer(ordinal));
     // If the cache is full, we need to clear one or more old entries
@@ -119,20 +102,4 @@ public class LruTaxonomyWriterCache impl
     return ret;
   }
 
-  @Override
-  public synchronized boolean put(CategoryPath categoryPath, int prefixLen, int ordinal) {
-    boolean ret = cache.put(categoryPath, prefixLen, new Integer(ordinal));
-    // If the cache is full, we need to clear one or more old entries
-    // from the cache. However, if we delete from the cache a recent
-    // addition that isn't yet in our reader, for this entry to be
-    // visible to us we need to make sure that the changes have been
-    // committed and we reopen the reader. Because this is a slow
-    // operation, we don't delete entries one-by-one but rather in bulk
-    // (put() removes the 2/3rd oldest entries).
-    if (ret) { 
-      cache.makeRoomLRU(); 
-    }
-    return ret;
-  }
-
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameHashIntCacheLRU.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameHashIntCacheLRU.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameHashIntCacheLRU.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameHashIntCacheLRU.java Sun Jan  6 19:07:44 2013
@@ -41,6 +41,7 @@ public class NameHashIntCacheLRU extends
 
   @Override
   Object key(CategoryPath name, int prefixLen) {
-    return new Long(name.longHashCode(prefixLen));
+    return new Long(name.subpath(prefixLen).longHashCode());
   }
+  
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameIntCacheLRU.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameIntCacheLRU.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameIntCacheLRU.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameIntCacheLRU.java Sun Jan  6 19:07:44 2013
@@ -68,23 +68,13 @@ class NameIntCacheLRU {
     return res;
   }
 
-  /**
-   * Subclasses can override this to provide caching by e.g. hash of the string.
-   */
+  /** Subclasses can override this to provide caching by e.g. hash of the string. */
   Object key(CategoryPath name) {
-    // Note that a copy constructor (cloning) here is necessary, because a
-    // CategoryPath object is mutable, so we cannot save a reference to an
-    // existing CategoryPath. Subclasses which override this method can
-    // avoid this cloning by, e.g., hashing the name.
-    return new CategoryPath(name);
+    return name;
   }
 
   Object key(CategoryPath name, int prefixLen) {
-    // Note that a copy constructor (cloning) here is necessary, because a
-    // CategoryPath object is mutable, so we cannot save a reference to an
-    // existing CategoryPath. Subclasses which override this method can
-    // avoid this cloning by, e.g., hashing the name.
-    return new CategoryPath(name, prefixLen);
+    return name.subpath(prefixLen);
   }
 
   /**

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=1429570&r1=1429569&r2=1429570&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 Sun Jan  6 19:07:44 2013
@@ -77,7 +77,7 @@ public class OrdinalMappingReaderTest ex
     FacetResultNode node = result.getFacetResultNode();
     for (FacetResultNode facet: node.getSubResults()) {
       int weight = (int)facet.getValue();
-      int label = Integer.parseInt(facet.getLabel().getComponent(1));
+      int label = Integer.parseInt(facet.getLabel().components[1]);
       //System.out.println(label + ": " + weight);
       if (VERBOSE) {
         System.out.println(label + ": " + weight);

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/categorypolicy/OrdinalPolicyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/categorypolicy/OrdinalPolicyTest.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/categorypolicy/OrdinalPolicyTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/categorypolicy/OrdinalPolicyTest.java Sun Jan  6 19:07:44 2013
@@ -31,12 +31,10 @@ public class OrdinalPolicyTest extends L
   public void testDefaultOrdinalPolicy() {
     // check ordinal policy
     OrdinalPolicy ordinalPolicy = OrdinalPolicy.ALL_PARENTS;
-    assertFalse("default ordinal policy should not match root", ordinalPolicy
-        .shouldAdd(TaxonomyReader.ROOT_ORDINAL));
+    assertFalse("default ordinal policy should not match root", ordinalPolicy.shouldAdd(TaxonomyReader.ROOT_ORDINAL));
     for (int i = 0; i < 300; i++) {
       int ordinal = 1 + random().nextInt(Integer.MAX_VALUE - 1);
-      assertTrue("default ordinal policy should match " + ordinal,
-          ordinalPolicy.shouldAdd(ordinal));
+      assertTrue("default ordinal policy should match " + ordinal, ordinalPolicy.shouldAdd(ordinal));
     }
   }
 
@@ -50,8 +48,7 @@ public class OrdinalPolicyTest extends L
     String[] topLevelStrings = new String[10];
     for (int i = 0; i < 10; i++) {
       topLevelStrings[i] = Integer.valueOf(random().nextInt(30)).toString();
-      topLevelOrdinals[i] = taxonomy.addCategory(new CategoryPath(
-          topLevelStrings[i]));
+      topLevelOrdinals[i] = taxonomy.addCategory(new CategoryPath(topLevelStrings[i]));
     }
     int[] nonTopLevelOrdinals = new int[300];
     for (int i = 0; i < 300; i++) {
@@ -61,22 +58,18 @@ public class OrdinalPolicyTest extends L
       for (int j = 1; j < components.length; j++) {
         components[j] = (Integer.valueOf(random().nextInt(30))).toString();
       }
-      nonTopLevelOrdinals[i] = taxonomy.addCategory(new CategoryPath(
-          components));
+      nonTopLevelOrdinals[i] = taxonomy.addCategory(new CategoryPath(components));
     }
     // check ordinal policy
     OrdinalPolicy ordinalPolicy = new NonTopLevelOrdinalPolicy();
     ordinalPolicy.init(taxonomy);
-    assertFalse("top level ordinal policy should not match root", ordinalPolicy
-        .shouldAdd(TaxonomyReader.ROOT_ORDINAL));
+    assertFalse("top level ordinal policy should not match root", ordinalPolicy.shouldAdd(TaxonomyReader.ROOT_ORDINAL));
     for (int i = 0; i < 10; i++) {
-      assertFalse("top level ordinal policy should not match "
-          + topLevelOrdinals[i],
+      assertFalse("top level ordinal policy should not match " + topLevelOrdinals[i], 
           ordinalPolicy.shouldAdd(topLevelOrdinals[i]));
     }
     for (int i = 0; i < 300; i++) {
-      assertTrue("top level ordinal policy should match "
-          + nonTopLevelOrdinals[i],
+      assertTrue("top level ordinal policy should match " + nonTopLevelOrdinals[i],
           ordinalPolicy.shouldAdd(nonTopLevelOrdinals[i]));
     }
 

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/categorypolicy/PathPolicyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/categorypolicy/PathPolicyTest.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/categorypolicy/PathPolicyTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/categorypolicy/PathPolicyTest.java Sun Jan  6 19:07:44 2013
@@ -29,10 +29,9 @@ public class PathPolicyTest extends Luce
   @Test
   public void testDefaultPathPolicy() {
     // check path policy
-    CategoryPath cp = new CategoryPath();
+    CategoryPath cp = CategoryPath.EMPTY;
     PathPolicy pathPolicy = PathPolicy.ALL_CATEGORIES;
-    assertFalse("default path policy should not accept root", 
-        pathPolicy.shouldAdd(cp));
+    assertFalse("default path policy should not accept root", pathPolicy.shouldAdd(cp));
     for (int i = 0; i < 300; i++) {
       int nComponents = 1 + random().nextInt(10);
       String[] components = new String[nComponents];
@@ -40,9 +39,7 @@ public class PathPolicyTest extends Luce
         components[j] = (Integer.valueOf(random().nextInt(30))).toString();
       }
       cp = new CategoryPath(components);
-      assertTrue("default path policy should accept "
-          + cp.toString('/'),
-          pathPolicy.shouldAdd(cp));
+      assertTrue("default path policy should accept " + cp.toString('/'), pathPolicy.shouldAdd(cp));
     }
   }
 
@@ -74,7 +71,7 @@ public class PathPolicyTest extends Luce
     // check ordinal policy
     PathPolicy pathPolicy = new NonTopLevelPathPolicy();
     assertFalse("top level path policy should not match root",
-        pathPolicy.shouldAdd(new CategoryPath()));
+        pathPolicy.shouldAdd(CategoryPath.EMPTY));
     for (int i = 0; i < 10; i++) {
       assertFalse("top level path policy should not match "
           + topLevelPaths[i],

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/FacetIndexingParamsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/FacetIndexingParamsTest.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/FacetIndexingParamsTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/FacetIndexingParamsTest.java Sun Jan  6 19:07:44 2013
@@ -74,7 +74,7 @@ public class FacetIndexingParamsTest ext
   public void testCategoryPolicies() {
     FacetIndexingParams dfip = FacetIndexingParams.ALL_PARENTS;
     // check path policy
-    CategoryPath cp = new CategoryPath();
+    CategoryPath cp = CategoryPath.EMPTY;
     PathPolicy pathPolicy = PathPolicy.ALL_CATEGORIES;
     assertEquals("path policy does not match default for root", pathPolicy.shouldAdd(cp), dfip.getPathPolicy().shouldAdd(cp));
     for (int i = 0; i < 30; i++) {

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=1429570&r1=1429569&r2=1429570&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 Sun Jan  6 19:07:44 2013
@@ -129,7 +129,7 @@ public class TestDemoFacets extends Luce
   }
 
   private void toSimpleString(int depth, StringBuilder sb, FacetResultNode node, String indent) {
-    sb.append(indent + node.getLabel().getComponent(depth) + " (" + (int) node.getValue() + ")\n");
+    sb.append(indent + node.getLabel().components[depth] + " (" + (int) node.getValue() + ")\n");
     for(FacetResultNode childNode : node.getSubResults()) {
       toSimpleString(depth+1, sb, childNode, indent + "  ");
     }

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=1429570&r1=1429569&r2=1429570&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 Sun Jan  6 19:07:44 2013
@@ -178,7 +178,7 @@ public class TestTopKInEachNodeResultHan
       }
       
       FacetResult fr = facetResults.get(0); // a, depth=3, K=2
-      boolean hasDoctor = "Doctor".equals(fr.getFacetRequest().getCategoryPath().getComponent(0));
+      boolean hasDoctor = "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
       assertEquals(9, fr.getNumValidDescendants());
       FacetResultNode parentRes = fr.getFacetResultNode();
       assertEquals(16.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -219,7 +219,7 @@ public class TestTopKInEachNodeResultHan
       }
 
       fr = facetResults.get(1); // a, depth=2, K=2. same result as before
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().getComponent(0));
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
       assertEquals(9, fr.getNumValidDescendants());
       parentRes = fr.getFacetResultNode();
       assertEquals(16.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -239,7 +239,7 @@ public class TestTopKInEachNodeResultHan
       }
 
       fr = facetResults.get(2); // a, depth=1, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().getComponent(0));
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
       assertEquals(4, fr.getNumValidDescendants(), 4);
       parentRes = fr.getFacetResultNode();
       assertEquals(16.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -257,7 +257,7 @@ public class TestTopKInEachNodeResultHan
       }
       
       fr = facetResults.get(3); // a/b, depth=3, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().getComponent(0));
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
       assertEquals(4, fr.getNumValidDescendants());
       parentRes = fr.getFacetResultNode();
       assertEquals(8.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -272,7 +272,7 @@ public class TestTopKInEachNodeResultHan
       }
 
       fr = facetResults.get(4); // a/b, depth=2, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().getComponent(0));
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
       assertEquals(4, fr.getNumValidDescendants());
       parentRes = fr.getFacetResultNode();
       assertEquals(8.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -286,7 +286,7 @@ public class TestTopKInEachNodeResultHan
       }
 
       fr = facetResults.get(5); // a/b, depth=1, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().getComponent(0));
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
       assertEquals(4, fr.getNumValidDescendants());
       parentRes = fr.getFacetResultNode();
       assertEquals(8.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -300,13 +300,13 @@ public class TestTopKInEachNodeResultHan
       }
       
       fr = facetResults.get(6); // a/b, depth=0, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().getComponent(0));
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
       assertEquals(0, fr.getNumValidDescendants()); // 0 descendants but rootnode
       parentRes = fr.getFacetResultNode();
       assertEquals(8.0, parentRes.getValue(), Double.MIN_VALUE);
       assertEquals(0.0, parentRes.getResidue(), Double.MIN_VALUE);
       assertEquals(0, parentRes.getNumSubResults());
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().getComponent(0));
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
 
       // doctor, depth=1, K=2
       assertFalse("Shouldn't have found anything for a FacetRequest " +

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=1429570&r1=1429569&r2=1429570&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 Sun Jan  6 19:07:44 2013
@@ -231,7 +231,7 @@ public class MultiIteratorsPerCLParamsTe
       if (requestedPath == null) {
         parentOrdinal = 0;
       } else {
-        CategoryPath cp = new CategoryPath(requestedPath.getComponent(0));
+        CategoryPath cp = new CategoryPath(requestedPath.components[0]);
         parentOrdinal = taxo.getOrdinal(cp);
       }
       parentArray = taxo.getParallelTaxonomyArrays().parents();

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java Sun Jan  6 19:07:44 2013
@@ -1,15 +1,7 @@
 package org.apache.lucene.facet.taxonomy;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-
-import org.junit.Test;
-
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.junit.Test;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -32,843 +24,145 @@ public class TestCategoryPath extends Lu
   
   @Test 
   public void testBasic() {
-    CategoryPath p = new CategoryPath(0,0);
-    assertEquals(0, p.length());
-    for (int i=0; i<1000; i++) {
-      p.add("hello");
-      assertEquals(i+1, p.length());
-    }
-  }
-  
-  @Test 
-  public void testConstructorCapacity() {
-    CategoryPath p = new CategoryPath(0,0);
-    assertEquals(0, p.capacityChars());
-    assertEquals(0, p.capacityComponents());
-    assertEquals(0, p.length());
-    p = new CategoryPath(5,18);
-    assertEquals(5, p.capacityChars());
-    assertEquals(18, p.capacityComponents());
-    assertEquals(0, p.length());
-    p = new CategoryPath(27,13);
-    assertEquals(27, p.capacityChars());
-    assertEquals(13, p.capacityComponents());
-    assertEquals(0, p.length());
-  }
-  
-  @Test 
-  public void testClear() {
-    CategoryPath p = new CategoryPath(0,0);
-    p.add("hi");
-    p.add("there");
-    assertEquals(2, p.length());
-    p.clear();
-    assertEquals(0, p.length());
-    p.add("yo!");
-    assertEquals(1, p.length());
-  }
-
-  @Test 
-  public void testTrim() {
-    CategoryPath p = new CategoryPath(0,0);
-    p.add("this");
-    p.add("message");
-    p.add("will");
-    p.add("self");
-    p.add("destruct");
-    p.add("in");
-    p.add("five");
-    p.add("seconds");
-    assertEquals(8, p.length());
-    p.trim(3);
-    assertEquals(5, p.length());
-    p.trim(0); // no-op
-    assertEquals(5, p.length());
-    p.trim(-3);  // no-op
-    assertEquals(5, p.length());
-    p.trim(1);
-    assertEquals(4, p.length());
-    p.trim(8); // clear
-    assertEquals(0, p.length());
-    p.add("yo!");
-    assertEquals(1, p.length());
-    p.trim(1); // clear
-    assertEquals(0, p.length());
-  }
-
-  @Test 
-  public void testComponentsLimit() {
-    // Test that we can add up to 2^15-1 components
-    CategoryPath p = new CategoryPath(0,0);
-    for (int i=0; i<32767; i++) {
-      p.add("");
-      assertEquals(i+1, p.length());
-    }
-    // Also see that in the current implementation, this is actually
-    // the limit: if we add one more component, things break (because
-    // we used a short to hold ncomponents). See that it breaks in the
-    // way we expect it to:
-    p.add(""); // this still works, but...
-    assertEquals(-32768, p.length()); // now the length is wrong and negative
-  }
-  
-  @Test 
-  public void testCharsLimit() {
-    // Test that we can add up to 2^15-1 characters
-    CategoryPath p = new CategoryPath(0,0);
-    for (int i=0; i<8192; i++) {
-      p.add("aaaa");
-    }
-    // Also see that in the current implementation, this is actually the
-    // limit: If we add one more character, things break (because ends[]
-    // is an array of shorts), and we actually get an exception.
-    try {
-      p.add("a");
-      fail("Should have thrown an exception");
-    } catch (ArrayIndexOutOfBoundsException e) {
-      // good.
-    }
+    assertEquals(0, CategoryPath.EMPTY.length);
+    assertEquals(1, new CategoryPath("hello").length);
+    assertEquals(2, new CategoryPath("hello", "world").length);
   }
   
   @Test 
   public void testToString() {
-    CategoryPath p = new CategoryPath(0,0);
     // When the category is empty, we expect an empty string
-    assertEquals("", p.toString('/'));
-    // This is (deliberately, in our implementation) indistinguishable
-    // from the case of a single empty component:
-    p.add("");
-    assertEquals("", p.toString('/'));
-    // Check just one category (so no delimiter needed):
-    p.clear();
-    p.add("hello");
-    assertEquals("hello", p.toString('/'));
-    // Now for two categories:
-    p.clear();
-    p.add("hello");
-    p.add("world");
-    assertEquals("hello/world", p.toString('/'));
-    // And for a thousand...
-    p.clear();
-    p.add("0");
-    StringBuilder expected = new StringBuilder("0");
-    for (int i=1; i<1000; i++) {
-      String num = Integer.toString(i);
-      p.add(num);
-      expected.append('/');
-      expected.append(num);
-    }
-    assertEquals(expected.toString(), p.toString('/'));
-    // Check that toString() without a parameter just defaults to '/':
-    assertEquals(expected.toString(), p.toString());
+    assertEquals("", CategoryPath.EMPTY.toString('/'));
+    // one category (so no delimiter needed)
+    assertEquals("hello", new CategoryPath("hello").toString('/'));
+    // more than one category (so no delimiter needed)
+    assertEquals("hello/world", new CategoryPath("hello", "world").toString('/'));
   }
 
-  // testing toString() and its variants already test most of the appendTo()
-  // code, but not all of it (the "eclemma" code-coverage tool discovered
-  // this for us). Here we complete the coverage of the appendTo() methods:
-  @Test 
-  public void testAppendTo() throws IOException {
-    CategoryPath p = new CategoryPath(0,0);
-    StringBuilder sb = new StringBuilder();
-    p.appendTo(sb, '/');
-    assertEquals(0, sb.length());
-    p.appendTo(sb, '/', -1);
-    assertEquals(0, sb.length());
-    p.appendTo(sb, '/', 1);
-    assertEquals(0, sb.length());
-    p.appendTo(sb, '/', -1, 1);
-    assertEquals(0, sb.length());
-  }
-  
-  @Test 
-  public void testLastComponent() {
-    CategoryPath p = new CategoryPath(1000,1000);
-    // When the category is empty, we expect a null
-    assertNull(p.lastComponent());
-    for (int i=0; i<=100; i++) {
-      String num = Integer.toString(i);
-      p.add(num);
-      assertEquals(num, p.lastComponent());
-    }
-  }
-  
   @Test 
   public void testGetComponent() {
-    CategoryPath p = new CategoryPath(1000,1000);
-    // When the category is empty, we expect a null
-    assertNull(p.getComponent(0));
-    assertNull(p.getComponent(1));
-    assertNull(p.getComponent(-1));
-    for (int i=0; i<=100; i++) {
-      p.add(Integer.toString(i));
-      for (int j=0; j<=i; j++) {
-        assertEquals(j, Integer.parseInt(p.getComponent(j)));
-      }
-      assertNull(p.getComponent(-1));
-      assertNull(p.getComponent(i+1));
+    String[] components = new String[atLeast(10)];
+    for (int i = 0; i < components.length; i++) {
+      components[i] = Integer.toString(i);
+    }
+    CategoryPath cp = new CategoryPath(components);
+    for (int i = 0; i < components.length; i++) {
+      assertEquals(i, Integer.parseInt(cp.components[i]));
     }
   }
 
-  @Test 
-  public void testToStringPrefix() {
-    CategoryPath p = new CategoryPath(0,0);
-    p.add("hi");
-    p.add("there");
-    p.add("man");
-    assertEquals("hi/there/man", p.toString('/'));
-    assertEquals("", p.toString('/', 0));
-    assertEquals("hi", p.toString('/', 1));
-    assertEquals("hi/there", p.toString('/', 2));
-    assertEquals("hi/there/man", p.toString('/', 3));
-    assertEquals("hi/there/man", p.toString('/', 4));
-    assertEquals("hi/there/man", p.toString('/', -1));
-  }
-
-  @Test 
-  public void testToStringSubpath() {
-    CategoryPath p = new CategoryPath(0,0);
-    assertEquals("", p.toString('/', 0, 0));
-    p.add("hi");
-    p.add("there");
-    p.add("man");
-    assertEquals("", p.toString('/', 0, 0));
-    assertEquals("hi", p.toString('/', 0, 1));
-    assertEquals("hi/there", p.toString('/', 0, 2));
-    assertEquals("hi/there/man", p.toString('/', 0, 3));
-    assertEquals("hi/there/man", p.toString('/', 0, 4));
-    assertEquals("hi/there/man", p.toString('/', 0, -1));
-    assertEquals("hi/there/man", p.toString('/', -1, -1));
-    assertEquals("there/man", p.toString('/', 1, -1));
-    assertEquals("man", p.toString('/', 2, -1));
-    assertEquals("", p.toString('/', 3, -1));
-    assertEquals("there/man", p.toString('/', 1, 3));
-    assertEquals("there", p.toString('/', 1, 2));
-    assertEquals("", p.toString('/', 1, 1));
-  }
-
-  @Test 
+  @Test
   public void testDelimiterConstructor() {
-    // Test that the constructor that takes a string and a delimiter
-    // works correctly. Also check that it allocates exactly the needed
-    // needed size for the array - not more.
     CategoryPath p = new CategoryPath("", '/');
-    assertEquals(p.length(), 0);
-    assertEquals(p.capacityChars(), 0);
-    assertEquals(p.capacityComponents(), 0);
+    assertEquals(0, p.length);
     p = new CategoryPath("hello", '/');
-    assertEquals(p.length(), 1);
-    assertEquals(p.capacityChars(), 5);
-    assertEquals(p.capacityComponents(), 1);
+    assertEquals(p.length, 1);
     assertEquals(p.toString('@'), "hello");
     p = new CategoryPath("hi/there", '/');
-    assertEquals(p.length(), 2);
-    assertEquals(p.capacityChars(), 7);
-    assertEquals(p.capacityComponents(), 2);
+    assertEquals(p.length, 2);
     assertEquals(p.toString('@'), "hi@there");
     p = new CategoryPath("how/are/you/doing?", '/');
-    assertEquals(p.length(), 4);
-    assertEquals(p.capacityChars(), 15);
-    assertEquals(p.capacityComponents(), 4);
+    assertEquals(p.length, 4);
     assertEquals(p.toString('@'), "how@are@you@doing?");
   }
   
-  @Test 
+  @Test
   public void testDefaultConstructor() {
     // test that the default constructor (no parameters) currently
     // defaults to creating an object with a 0 initial capacity.
     // If we change this default later, we also need to change this
     // test.
-    CategoryPath p = new CategoryPath();
-    assertEquals(0, p.capacityChars());
-    assertEquals(0, p.capacityComponents());
-    assertEquals(0, p.length());
+    CategoryPath p = CategoryPath.EMPTY;
+    assertEquals(0, p.length);
     assertEquals("", p.toString('/'));
   }
   
   @Test 
-  public void testAddEmpty() {
-    // In the current implementation, p.add("") should add en empty
-    // component (which is, admitingly, not a useful case. On the other
-    // hand, p.add("", delimiter) should add no components at all.
-    // Verify this:
-    CategoryPath p = new CategoryPath(0, 0);
-    p.add("");
-    assertEquals(1, p.length());
-    p.add("");
-    assertEquals(2, p.length());
-    p.add("", '/');
-    assertEquals(2, p.length());
-    p.clear();
-    p.add("", '/');
-    assertEquals(0, p.length());
-  }
-  
-  @Test 
-  public void testDelimiterAdd() {
-    // Test that the add() that takes a string and a delimiter
-    // works correctly. Note that unlike the constructor test above,
-    // we can't expect the capacity to grow to exactly the length of
-    // the given category, so we do not test this.
-    CategoryPath p = new CategoryPath(0, 0);
-    p.add("", '/');
-    assertEquals(0, p.length());
-    assertEquals("", p.toString('@'), "");
-    p.clear();
-    p.add("hello", '/');
-    assertEquals(p.length(), 1);
-    assertEquals(p.toString('@'), "hello");
-    p.clear();
-    p.add("hi/there", '/');
-    assertEquals(p.length(), 2);
-    assertEquals(p.toString('@'), "hi@there");
-    p.clear();
-    p.add("how/are/you/doing?", '/');
-    assertEquals(p.length(), 4);
-    assertEquals(p.toString('@'), "how@are@you@doing?");
-    // See that this is really an add, not replace:
-    p.clear();
-    p.add("hi/there", '/');
-    assertEquals(p.length(), 2);
-    assertEquals(p.toString('@'), "hi@there");
-    p.add("how/are/you/doing", '/');
-    assertEquals(p.length(), 6);
-    assertEquals(p.toString('@'), "hi@there@how@are@you@doing");
-  }
-  
-  @Test 
-  public void testCopyConstructor() {
-    CategoryPath p = new CategoryPath(0,0);
-    int expectedchars=0;
-    for (int i=0; i<1000; i++) {
-      CategoryPath clone = new CategoryPath(p);
-      assertEquals(p.length(), clone.length());
-      assertEquals(p.toString('/'), clone.toString('/'));
-      // verify that the newly created clone has exactly the right
-      // capacity, with no spare (while the original path p probably
-      // does have spare)
-      assertEquals(i, clone.capacityComponents());
-      assertEquals(expectedchars, clone.capacityChars());
-      // Finally, add another component to the path, for the next
-      // round of this loop
-      String num = Integer.toString(i);
-      p.add(num);
-      expectedchars+=num.length();
-    }
-  }
-
-  @Test 
-  public void testPrefixCopyConstructor() {
-    CategoryPath p = new CategoryPath(0,0);
-    p.add("hi");
-    p.add("there");
-    p.add("man");
-    assertEquals(p.length(), 3);
+  public void testSubPath() {
+    final CategoryPath p = new CategoryPath("hi", "there", "man");
+    assertEquals(p.length, 3);
     
-    CategoryPath p1 = new CategoryPath(p,2);
-    assertEquals(2, p1.length());
+    CategoryPath p1 = p.subpath(2);
+    assertEquals(2, p1.length);
     assertEquals("hi/there", p1.toString('/'));
-    // the new prefix object should only take the space it needs: 
-    assertEquals(2, p1.capacityComponents());
-    assertEquals(7, p1.capacityChars());
 
-    p1 = new CategoryPath(p,1);
-    assertEquals(1, p1.length());
+    p1 = p.subpath(1);
+    assertEquals(1, p1.length);
     assertEquals("hi", p1.toString('/'));
-    assertEquals(1, p1.capacityComponents());
-    assertEquals(2, p1.capacityChars());
 
-    p1 = new CategoryPath(p,0);
-    assertEquals(0, p1.length());
+    p1 = p.subpath(0);
+    assertEquals(0, p1.length);
     assertEquals("", p1.toString('/'));
-    assertEquals(0, p1.capacityComponents());
-    assertEquals(0, p1.capacityChars());
 
-    // with all the following lengths, the prefix should be the whole path: 
+    // with all the following lengths, the prefix should be the whole path 
     int[] lengths = { 3, -1, 4 };
-    for (int i=0; i<lengths.length; i++) {
-      p1 = new CategoryPath(p, lengths[i]);
-      assertEquals(3, p1.length());
+    for (int i = 0; i < lengths.length; i++) {
+      p1 = p.subpath(lengths[i]);
+      assertEquals(3, p1.length);
       assertEquals("hi/there/man", p1.toString('/'));
       assertEquals(p, p1);
-      assertEquals(3, p1.capacityComponents());
-      assertEquals(10, p1.capacityChars());
     }
   }
 
   @Test 
   public void testEquals() {
-    // check that two empty paths are equal, even if they have different
-    // capacities:
-    CategoryPath p1 = new CategoryPath(0,0);
-    CategoryPath p2 = new CategoryPath(1000,300);
-    assertEquals(true, p1.equals(p2));
-    // If we make p2 different, it is no longer equals:
-    p2.add("hi");
-    assertEquals(false, p1.equals(p2));
-    // A categoryPath is definitely not equals to an object of some other
-    // type:
-    assertEquals(false, p1.equals(Integer.valueOf(3)));
-    // Build two paths separately, and compare them
-    p1.clear();
-    p1.add("hello");
-    p1.add("world");
-    p2.clear();
-    p2.add("hello");
-    p2.add("world");
-    assertEquals(true, p1.equals(p2));    
-    // Check that comparison really don't look at old data which might
-    // be stored in the array
-    p1.clear();
-    p1.add("averylongcategoryname");
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hi");
-    assertEquals(true, p1.equals(p2));
-    // Being of the same length is obviously not enough to be equal
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hello");
-    assertEquals(false, p1.equals(p2));
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("ho");
-    assertEquals(false, p1.equals(p2));
+    assertEquals(CategoryPath.EMPTY, CategoryPath.EMPTY);
+    assertFalse(CategoryPath.EMPTY.equals(new CategoryPath("hi")));
+    assertFalse(CategoryPath.EMPTY.equals(Integer.valueOf(3)));
+    assertEquals(new CategoryPath("hello", "world"), new CategoryPath("hello", "world"));    
   }
+  
   @Test 
   public void testHashCode() {
-    // Note: in this test, we assume that if two paths are not equal,
-    // their hash codes should come out differently. This is *not*
-    // always the case, but in the examples we use below, it comes out
-    // fine, and unless we have some really bad luck in changing our
-    // hash function, this should also remain true in the future.
-    
-    // check that two empty paths are equal, even if they have different
-    // capacities:
-    CategoryPath p1 = new CategoryPath(0,0);
-    CategoryPath p2 = new CategoryPath(1000,300);
-    assertEquals(p1.hashCode(), p2.hashCode());
-    // If we make p2 different, it is no longer equals:
-    p2.add("hi");
-    assertEquals(false, p1.hashCode()==p2.hashCode());
-    // Build two paths separately, and compare them
-    p1.clear();
-    p1.add("hello");
-    p1.add("world");
-    p2.clear();
-    p2.add("hello");
-    p2.add("world");
-    assertEquals(p1.hashCode(), p2.hashCode());
-    // Check that comparison really don't look at old data which might
-    // be stored in the array
-    p1.clear();
-    p1.add("averylongcategoryname");
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hi");
-    assertEquals(p1.hashCode(), p2.hashCode());
-    // Being of the same length is obviously not enough to be equal
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hello");
-    assertEquals(false, p1.hashCode()==p2.hashCode());
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("ho");
-    assertEquals(false, p1.hashCode()==p2.hashCode());
+    assertEquals(CategoryPath.EMPTY.hashCode(), CategoryPath.EMPTY.hashCode());
+    assertFalse(CategoryPath.EMPTY.hashCode() == new CategoryPath("hi").hashCode());
+    assertEquals(new CategoryPath("hello", "world").hashCode(), new CategoryPath("hello", "world").hashCode());
   }
   
   @Test 
-  public void testHashCodePrefix() {
-    // First, repeat the tests of testHashCode() using hashCode(-1)
-    // just to make sure nothing was broken in this variant:
-    CategoryPath p1 = new CategoryPath(0,0);
-    CategoryPath p2 = new CategoryPath(1000,300);
-    assertEquals(p1.hashCode(-1), p2.hashCode(-1));
-    p2.add("hi");
-    assertEquals(false, p1.hashCode(-1)==p2.hashCode(-1));
-    p1.clear();
-    p1.add("hello");
-    p1.add("world");
-    p2.clear();
-    p2.add("hello");
-    p2.add("world");
-    assertEquals(p1.hashCode(-1), p2.hashCode(-1));
-    p1.clear();
-    p1.add("averylongcategoryname");
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hi");
-    assertEquals(p1.hashCode(-1), p2.hashCode(-1));
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hello");
-    assertEquals(false, p1.hashCode(-1)==p2.hashCode(-1));
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("ho");
-    assertEquals(false, p1.hashCode(-1)==p2.hashCode(-1));
-    
-    // Now move to testing prefixes:
-    CategoryPath p = new CategoryPath();
-    p.add("this");
-    p.add("is");
-    p.add("a");
-    p.add("test");
-    assertEquals(p.hashCode(), p.hashCode(4));
-    assertEquals(new CategoryPath().hashCode(), p.hashCode(0));
-    assertEquals(new CategoryPath(p, 1).hashCode(), p.hashCode(1));
-    assertEquals(new CategoryPath(p, 2).hashCode(), p.hashCode(2));
-    assertEquals(new CategoryPath(p, 3).hashCode(), p.hashCode(3));
-  }
-
-  @Test 
   public void testLongHashCode() {
-    // Note: in this test, we assume that if two paths are not equal,
-    // their hash codes should come out differently. This is *not*
-    // always the case, but in the examples we use below, it comes out
-    // fine, and unless we have some really bad luck in changing our
-    // hash function, this should also remain true in the future.
-    
-    // check that two empty paths are equal, even if they have different
-    // capacities:
-    CategoryPath p1 = new CategoryPath(0,0);
-    CategoryPath p2 = new CategoryPath(1000,300);
-    assertEquals(p1.longHashCode(), p2.longHashCode());
-    // If we make p2 different, it is no longer equals:
-    p2.add("hi");
-    assertEquals(false, p1.longHashCode()==p2.longHashCode());
-    // Build two paths separately, and compare them
-    p1.clear();
-    p1.add("hello");
-    p1.add("world");
-    p2.clear();
-    p2.add("hello");
-    p2.add("world");
-    assertEquals(p1.longHashCode(), p2.longHashCode());
-    // Check that comparison really don't look at old data which might
-    // be stored in the array
-    p1.clear();
-    p1.add("averylongcategoryname");
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hi");
-    assertEquals(p1.longHashCode(), p2.longHashCode());
-    // Being of the same length is obviously not enough to be equal
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hello");
-    assertEquals(false, p1.longHashCode()==p2.longHashCode());
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("ho");
-    assertEquals(false, p1.longHashCode()==p2.longHashCode());
-  }
-  
-  @Test 
-  public void testLongHashCodePrefix() {
-    // First, repeat the tests of testLongHashCode() using longHashCode(-1)
-    // just to make sure nothing was broken in this variant:
-    
-    // check that two empty paths are equal, even if they have different
-    // capacities:
-    CategoryPath p1 = new CategoryPath(0,0);
-    CategoryPath p2 = new CategoryPath(1000,300);
-    assertEquals(p1.longHashCode(-1), p2.longHashCode(-1));
-    // If we make p2 different, it is no longer equals:
-    p2.add("hi");
-    assertEquals(false, p1.longHashCode(-1)==p2.longHashCode(-1));
-    // Build two paths separately, and compare them
-    p1.clear();
-    p1.add("hello");
-    p1.add("world");
-    p2.clear();
-    p2.add("hello");
-    p2.add("world");
-    assertEquals(p1.longHashCode(-1), p2.longHashCode(-1));
-    // Check that comparison really don't look at old data which might
-    // be stored in the array
-    p1.clear();
-    p1.add("averylongcategoryname");
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hi");
-    assertEquals(p1.longHashCode(-1), p2.longHashCode(-1));
-    // Being of the same length is obviously not enough to be equal
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("hello");
-    assertEquals(false, p1.longHashCode(-1)==p2.longHashCode(-1));
-    p1.clear();
-    p1.add("hi");
-    p2.clear();
-    p2.add("ho");
-    assertEquals(false, p1.longHashCode(-1)==p2.longHashCode(-1));
-    
-    // Now move to testing prefixes:
-    CategoryPath p = new CategoryPath();
-    p.add("this");
-    p.add("is");
-    p.add("a");
-    p.add("test");
-    assertEquals(p.longHashCode(), p.longHashCode(4));
-    assertEquals(new CategoryPath().longHashCode(), p.longHashCode(0));
-    assertEquals(new CategoryPath(p, 1).longHashCode(), p.longHashCode(1));
-    assertEquals(new CategoryPath(p, 2).longHashCode(), p.longHashCode(2));
-    assertEquals(new CategoryPath(p, 3).longHashCode(), p.longHashCode(3));
+    assertEquals(CategoryPath.EMPTY.longHashCode(), CategoryPath.EMPTY.longHashCode());
+    assertFalse(CategoryPath.EMPTY.longHashCode() == new CategoryPath("hi").longHashCode());
+    assertEquals(new CategoryPath("hello", "world").longHashCode(), new CategoryPath("hello", "world").longHashCode());
   }
   
   @Test 
   public void testArrayConstructor() {
     CategoryPath p = new CategoryPath("hello", "world", "yo");
-    assertEquals(3, p.length());
-    assertEquals(12, p.capacityChars());
-    assertEquals(3, p.capacityComponents());
+    assertEquals(3, p.length);
     assertEquals("hello/world/yo", p.toString('/'));
     
     p = new CategoryPath(new String[0]);
-    assertEquals(0, p.length());
-    assertEquals(0, p.capacityChars());
-    assertEquals(0, p.capacityComponents());
+    assertEquals(0, p.length);
   }
   
   @Test 
   public void testCharsNeededForFullPath() {
+    assertEquals(0, CategoryPath.EMPTY.fullPathLength());
     String[] components = { "hello", "world", "yo" };
-    CategoryPath p = new CategoryPath();
-    assertEquals(0, p.charsNeededForFullPath());
+    CategoryPath cp = new CategoryPath(components);
     int expectedCharsNeeded = 0;
-    for (int i=0; i<components.length; i++) {
-      p.add(components[i]);
-      expectedCharsNeeded += components[i].length();
-      if (i>0) {
-        expectedCharsNeeded++;
-      }
-      assertEquals(expectedCharsNeeded, p.charsNeededForFullPath());
+    for (String comp : components) {
+      expectedCharsNeeded += comp.length();
     }
+    expectedCharsNeeded += cp.length - 1; // delimiter chars
+    assertEquals(expectedCharsNeeded, cp.fullPathLength());
   }
   
   @Test 
   public void testCopyToCharArray() {
-    String[] components = { "hello", "world", "yo" };
-    CategoryPath p = new CategoryPath(components);
-    char[] charArray = new char[p.charsNeededForFullPath()];
-    int numCharsCopied = 0;
-    
-    numCharsCopied = p.copyToCharArray(charArray, 0, 0, '.');
-    assertEquals(0, numCharsCopied);
-    assertEquals("", new String(charArray, 0, numCharsCopied));
-    
-    numCharsCopied = p.copyToCharArray(charArray, 0, 1, '.');
-    assertEquals(5, numCharsCopied);
-    assertEquals("hello", new String(charArray, 0, numCharsCopied));
-    
-    numCharsCopied = p.copyToCharArray(charArray, 0, 3, '.');
-    assertEquals(14, numCharsCopied);
-    assertEquals("hello.world.yo", new String(charArray, 0, numCharsCopied));
-    
-    numCharsCopied = p.copyToCharArray(charArray, 0, -1, '.');
-    assertEquals(14, numCharsCopied);
-    assertEquals("hello.world.yo", new String(charArray, 0, numCharsCopied));
-    numCharsCopied = p.copyToCharArray(charArray, 0, 4, '.');
-    assertEquals(14, numCharsCopied);
+    CategoryPath p = new CategoryPath("hello", "world", "yo");
+    char[] charArray = new char[p.fullPathLength()];
+    int numCharsCopied = p.copyFullPath(charArray, 0, '.');
+    assertEquals(p.fullPathLength(), numCharsCopied);
     assertEquals("hello.world.yo", new String(charArray, 0, numCharsCopied));
   }
   
   @Test 
-  public void testCharSerialization() throws Exception {
-    CategoryPath[] testCategories = {
-        new CategoryPath("hi", "there", "man"),
-        new CategoryPath("hello"),
-        new CategoryPath("what's", "up"),
-        // See that an empty category, which generates a (char)0,
-        // doesn't cause any problems in the middle of the serialization:
-        new CategoryPath(),
-        new CategoryPath("another", "example"),
-        new CategoryPath(),
-        new CategoryPath()
-    };
-    StringBuilder sb = new StringBuilder();
-    for (int i=0; i<testCategories.length; i++) {
-      testCategories[i].serializeAppendTo(sb);
-    }
-    
-    CategoryPath tmp = new CategoryPath();
-    int offset=0;
-    for (int i=0; i<testCategories.length; i++) {
-      // check equalsToSerialized, in a equal and non-equal case:
-      assertTrue(testCategories[i].equalsToSerialized(sb, offset));
-      assertFalse(new CategoryPath("Hello", "world").equalsToSerialized(sb, offset));
-      assertFalse(new CategoryPath("world").equalsToSerialized(sb, offset));
-      // and check hashCodeFromSerialized:
-      assertEquals(testCategories[i].hashCode(), CategoryPath.hashCodeOfSerialized(sb, offset));
-      // and check setFromSerialized:
-      offset = tmp.setFromSerialized(sb, offset);
-      assertEquals(testCategories[i], tmp);
-    }
-    assertEquals(offset, sb.length());
-    // A similar test, for a much longer path (though not larger than the
-    // 2^15-1 character limit that CategoryPath allows:
-    sb = new StringBuilder();
-    CategoryPath p = new CategoryPath();
-    for (int i=0; i<1000; i++) {
-      p.add(Integer.toString(i));
-    }
-    p.serializeAppendTo(sb);
-    p.serializeAppendTo(sb);
-    p.serializeAppendTo(sb);
-    offset=0;
-    assertTrue(p.equalsToSerialized(sb, offset));
-    assertEquals(p.hashCode(), CategoryPath.hashCodeOfSerialized(sb, offset));
-    offset = tmp.setFromSerialized(sb, offset);
-    assertEquals(p, tmp);
-    assertTrue(p.equalsToSerialized(sb, offset));
-    assertEquals(p.hashCode(), CategoryPath.hashCodeOfSerialized(sb, offset));
-    offset = tmp.setFromSerialized(sb, offset);
-    assertEquals(p, tmp);
-    assertTrue(p.equalsToSerialized(sb, offset));
-    assertEquals(p.hashCode(), CategoryPath.hashCodeOfSerialized(sb, offset));
-    offset = tmp.setFromSerialized(sb, offset);
-    assertEquals(p, tmp);
-    assertEquals(offset, sb.length());
-    
-    // Test the serializeAppendTo variant with a prefixLen
-    p = new CategoryPath();
-    for (int i=0; i<783; i++) {
-      p.add(Integer.toString(i));
-    }
-    int[] prefixLengths = { 0, 574, 782, 783, 784, -1 };
-    for (int prefixLen : prefixLengths) {
-      sb = new StringBuilder();
-      p.serializeAppendTo(prefixLen, sb);
-      assertTrue(new CategoryPath(p, prefixLen).equalsToSerialized(sb, 0));
-    }
-    
-    // Test the equalsToSerialized variant with a prefixLen
-    // We use p and prefixLengths set above.
-    for (int prefixLen : prefixLengths) {
-      sb = new StringBuilder();
-      new CategoryPath(p, prefixLen).serializeAppendTo(sb);
-      assertTrue(p.equalsToSerialized(prefixLen, sb, 0));
-    }
-    
-    // Check also the false case of equalsToSerialized with prefixLen:
-    sb = new StringBuilder();
-    new CategoryPath().serializeAppendTo(sb);
-    assertTrue(new CategoryPath().equalsToSerialized(0, sb, 0));
-    assertTrue(new CategoryPath("a", "b").equalsToSerialized(0, sb, 0));
-    assertFalse(new CategoryPath("a", "b").equalsToSerialized(1, sb, 0));
-    sb = new StringBuilder();
-    new CategoryPath("a", "b").serializeAppendTo(sb);
-    assertFalse(new CategoryPath().equalsToSerialized(0, sb, 0));
-    assertFalse(new CategoryPath("a").equalsToSerialized(0, sb, 0));
-    assertFalse(new CategoryPath("a").equalsToSerialized(1, sb, 0));
-    assertFalse(new CategoryPath("a", "b").equalsToSerialized(0, sb, 0));
-    assertFalse(new CategoryPath("a", "b").equalsToSerialized(1, sb, 0));
-    assertTrue(new CategoryPath("a", "b").equalsToSerialized(2, sb, 0));
-    assertTrue(new CategoryPath("a", "b", "c").equalsToSerialized(2, sb, 0));
-    assertFalse(new CategoryPath("z", "b", "c").equalsToSerialized(2, sb, 0));
-    assertFalse(new CategoryPath("aa", "b", "c").equalsToSerialized(2, sb, 0));
-  }
-
-  @Test 
-  public void testStreamWriterSerialization() throws Exception {
-    CategoryPath[] testPaths = {
-        new CategoryPath("hi", "there", "man"),
-        new CategoryPath("hello"),
-        new CategoryPath("date", "2009", "May", "13", "14", "59", "00"),
-        // See that an empty category, which generates a (char)0,
-        // doesn't cause any problems in the middle of the serialization:
-        new CategoryPath(),
-        new CategoryPath("another", "example")
-    };
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    OutputStreamWriter osw = new OutputStreamWriter(baos, "UTF-8");  // UTF-8 is always supported.
-    for (CategoryPath cp : testPaths) {
-      cp.serializeToStreamWriter(osw);
-    }
-    osw.flush();
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    InputStreamReader isr = new InputStreamReader(bais, "UTF-8");
-    CategoryPath[] checkPaths = {
-        new CategoryPath(), new CategoryPath(), new CategoryPath(), new CategoryPath(), new CategoryPath()
-    };
-    for (int j = 0; j < checkPaths.length; j++) {
-      checkPaths[j].deserializeFromStreamReader(isr);
-      assertEquals("Paths not equal", testPaths[j], checkPaths[j]);
-    }
-  }
-
-  @Test 
-  public void testCharSequenceCtor() throws Exception {
-    CategoryPath[] testPaths = {
-        new CategoryPath(new CS("hi"), new CS("there"), new CS("man")),
-        new CategoryPath(new CS("hello")),
-        new CategoryPath(new CS("date"), new CS("2009"), new CS("May"), new CS("13"),
-            new CS("14"), new CS("59"), new CS("00")),
-        new CategoryPath(),
-        new CategoryPath(new CS("another"), new CS("example"))
-    };
-    assertEquals("Wrong capacity", 10, testPaths[0].capacityChars());
-    assertEquals("Wrong capacity", 5, testPaths[1].capacityChars());
-    assertEquals("Wrong capacity", 19, testPaths[2].capacityChars());
-    assertEquals("Wrong capacity", 0, testPaths[3].capacityChars());
-    assertEquals("Wrong capacity", 14, testPaths[4].capacityChars());
-
-    assertEquals("Wrong component", "hi", testPaths[0].getComponent(0));
-    assertEquals("Wrong component", "there", testPaths[0].getComponent(1));
-    assertEquals("Wrong component", "man", testPaths[0].getComponent(2));
-    assertEquals("Wrong component", "hello", testPaths[1].getComponent(0));
-    assertEquals("Wrong component", "date", testPaths[2].getComponent(0));
-    assertEquals("Wrong component", "2009", testPaths[2].getComponent(1));
-    assertEquals("Wrong component", "May", testPaths[2].getComponent(2));
-    assertEquals("Wrong component", "13", testPaths[2].getComponent(3));
-    assertEquals("Wrong component", "14", testPaths[2].getComponent(4));
-    assertEquals("Wrong component", "59", testPaths[2].getComponent(5));
-    assertEquals("Wrong component", "00", testPaths[2].getComponent(6));
-    assertNull("Not null component", testPaths[3].getComponent(0));
-    assertEquals("Wrong component", "another", testPaths[4].getComponent(0));
-    assertEquals("Wrong component", "example", testPaths[4].getComponent(1));
-  }
-
-  @Test 
-  public void testIsDescendantOf() throws Exception {
-    CategoryPath[] testPaths = {
-        new CategoryPath(new CS("hi"), new CS("there")),
-        new CategoryPath(new CS("hi"), new CS("there"), new CS("man")),
-        new CategoryPath(new CS("hithere"), new CS("man")),
-        new CategoryPath(new CS("hi"), new CS("there"), new CS("mano")),
-        new CategoryPath(),
-    };
-    assertTrue(testPaths[0].isDescendantOf(testPaths[0]));
-    assertTrue(testPaths[0].isDescendantOf(testPaths[4]));
-    assertFalse(testPaths[4].isDescendantOf(testPaths[0]));
-    assertTrue(testPaths[1].isDescendantOf(testPaths[0]));
-    assertTrue(testPaths[1].isDescendantOf(testPaths[1]));
-    assertTrue(testPaths[3].isDescendantOf(testPaths[0]));
-    assertFalse(testPaths[2].isDescendantOf(testPaths[0]));
-    assertFalse(testPaths[2].isDescendantOf(testPaths[1]));
-    assertFalse(testPaths[3].isDescendantOf(testPaths[1]));
-  }
-
-  @Test 
   public void testCompareTo() {
     CategoryPath p = new CategoryPath("a/b/c/d", '/');
     CategoryPath pother = new CategoryPath("a/b/c/d", '/');
-    assertTrue(pother.compareTo(p) == 0);
+    assertEquals(0, pother.compareTo(p));
     pother = new CategoryPath("", '/');
     assertTrue(pother.compareTo(p) < 0);
     pother = new CategoryPath("a/b_/c/d", '/');
@@ -880,25 +174,5 @@ public class TestCategoryPath extends Lu
     pother = new CategoryPath("a/b/c//e", '/');
     assertTrue(pother.compareTo(p) < 0);
   }
-  
-  private static class CS implements CharSequence {
-    public CS(String s) {
-      this.ca = new char[s.length()];
-      s.getChars(0, s.length(), this.ca, 0);
-    }
-    @Override
-    public char charAt(int index) {
-      return this.ca[index];
-    }
-    @Override
-    public int length() {
-      return this.ca.length;
-    }
-    @Override
-    public CharSequence subSequence(int start, int end) {
-      return null; // not used.
-    }
-    private char[] ca;
-  }
 
 }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyCombined.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyCombined.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyCombined.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyCombined.java Sun Jan  6 19:07:44 2013
@@ -146,7 +146,7 @@ public class TestTaxonomyCombined extend
     if (path==null) {
       return "<null>";
     }
-    if (path.length()==0) {
+    if (path.length==0) {
       return "<empty>";
     }
     return "<"+path.toString('/')+">";
@@ -304,9 +304,9 @@ public class TestTaxonomyCombined extend
     tw.close();
     TaxonomyReader tr = new DirectoryTaxonomyReader(indexDir);
     assertEquals(1, tr.getSize());
-    assertEquals(0, tr.getPath(0).length());
+    assertEquals(0, tr.getPath(0).length);
     assertEquals(TaxonomyReader.INVALID_ORDINAL, tr.getParent(0));
-    assertEquals(0, tr.getOrdinal(new CategoryPath()));
+    assertEquals(0, tr.getOrdinal(CategoryPath.EMPTY));
     tr.close();
     indexDir.close();
   }
@@ -323,9 +323,9 @@ public class TestTaxonomyCombined extend
     tw.commit();
     TaxonomyReader tr = new DirectoryTaxonomyReader(indexDir);
     assertEquals(1, tr.getSize());
-    assertEquals(0, tr.getPath(0).length());
+    assertEquals(0, tr.getPath(0).length);
     assertEquals(TaxonomyReader.INVALID_ORDINAL, tr.getParent(0));
-    assertEquals(0, tr.getOrdinal(new CategoryPath()));
+    assertEquals(0, tr.getOrdinal(CategoryPath.EMPTY));
     tw.close();
     tr.close();
     indexDir.close();
@@ -416,7 +416,7 @@ public class TestTaxonomyCombined extend
         ", but this is not a valid category.");
       }
       // verify that the parent is indeed my parent, according to the strings
-      if (!new CategoryPath(me, me.length()-1).equals(parent)) {
+      if (!me.subpath(me.length-1).equals(parent)) {
         fail("Got parent "+parentOrdinal+" for ordinal "+ordinal+
             " but categories are "+showcat(parent)+" and "+showcat(me)+
             " respectively.");
@@ -506,7 +506,7 @@ public class TestTaxonomyCombined extend
       }
       // verify that the parent is indeed my parent, according to the
       // strings
-      if (!new CategoryPath(me, me.length() - 1).equals(parent)) {
+      if (!me.subpath(me.length - 1).equals(parent)) {
         fail("Got parent " + parentOrdinal + " for ordinal " + ordinal
             + " but categories are " + showcat(parent) + " and "
             + showcat(me) + " respectively.");

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java Sun Jan  6 19:07:44 2013
@@ -81,7 +81,6 @@ public class TestAddTaxonomy extends Luc
   }
 
   private void validate(Directory dest, Directory src, OrdinalMap ordMap) throws Exception {
-    CategoryPath cp = new CategoryPath();
     DirectoryTaxonomyReader destTR = new DirectoryTaxonomyReader(dest);
     try {
       final int destSize = destTR.getSize();
@@ -98,7 +97,7 @@ public class TestAddTaxonomy extends Luc
         // validate that all source categories exist in destination, and their
         // ordinals are as expected.
         for (int j = 1; j < srcSize; j++) {
-          srcTR.getPath(j, cp);
+          CategoryPath cp = srcTR.getPath(j);
           int destOrdinal = destTR.getOrdinal(cp);
           assertTrue(cp + " not found in destination", destOrdinal > 0);
           assertEquals(destOrdinal, map[j]);

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=1429570&r1=1429569&r2=1429570&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 Sun Jan  6 19:07:44 2013
@@ -48,12 +48,8 @@ public class TestConcurrentFacetedIndexi
     @Override
     public int get(CategoryPath categoryPath) { return -1; }
     @Override
-    public int get(CategoryPath categoryPath, int length) { return -1; }
-    @Override
     public boolean put(CategoryPath categoryPath, int ordinal) { return true; }
     @Override
-    public boolean put(CategoryPath categoryPath, int prefixLen, int ordinal) { return true; }
-    @Override
     public boolean isFull() { return true; }
     @Override
     public void clear() {}
@@ -108,9 +104,9 @@ public class TestConcurrentFacetedIndexi
                 CategoryPath cp = newCategory();
                 cats.add(cp);
                 // add all prefixes to values
-                int level = cp.length();
+                int level = cp.length;
                 while (level > 0) {
-                  String s = cp.toString('/', level);
+                  String s = cp.subpath(level).toString('/');
                   values.put(s, s);
                   --level;
                 }
@@ -134,11 +130,11 @@ public class TestConcurrentFacetedIndexi
     for (String cat : values.keySet()) {
       CategoryPath cp = new CategoryPath(cat, '/');
       assertTrue("category not found " + cp, tr.getOrdinal(cp) > 0);
-      int level = cp.length();
+      int level = cp.length;
       int parentOrd = 0; // for root, parent is always virtual ROOT (ord=0)
-      CategoryPath path = new CategoryPath();
+      CategoryPath path = CategoryPath.EMPTY;
       for (int i = 0; i < level; i++) {
-        path.add(cp.getComponent(i));
+        path = cp.subpath(i + 1);
         int ord = tr.getOrdinal(path);
         assertEquals("invalid parent for cp=" + path, parentOrd, parents[ord]);
         parentOrd = ord; // next level should have this parent

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java Sun Jan  6 19:07:44 2013
@@ -154,8 +154,8 @@ public class TestDirectoryTaxonomyReader
       for (int i=0; i<n; i++) {
         int k = random.nextInt(n);
         tw = new DirectoryTaxonomyWriter(dir, OpenMode.CREATE);
-        for (int j=0; j<=k; j++) {
-          tw.addCategory(new CategoryPath(cp[j]));
+        for (int j = 0; j <= k; j++) {
+          tw.addCategory(cp[j]);
         }
         tw.close();
         if (closeReader) {

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java Sun Jan  6 19:07:44 2013
@@ -52,12 +52,8 @@ public class TestDirectoryTaxonomyWriter
     @Override
     public int get(CategoryPath categoryPath) { return -1; }
     @Override
-    public int get(CategoryPath categoryPath, int length) { return -1; }
-    @Override
     public boolean put(CategoryPath categoryPath, int ordinal) { return true; }
     @Override
-    public boolean put(CategoryPath categoryPath, int prefixLen, int ordinal) { return true; }
-    @Override
     public boolean isFull() { return true; }
     @Override
     public void clear() {}
@@ -266,10 +262,10 @@ public class TestDirectoryTaxonomyWriter
                   Integer.toString(value / 100000), Integer.toString(value));
               int ord = tw.addCategory(cp);
               assertTrue("invalid parent for ordinal " + ord + ", category " + cp, tw.getParent(ord) != -1);
-              String l1 = cp.toString('/', 1);
-              String l2 = cp.toString('/', 2);
-              String l3 = cp.toString('/', 3);
-              String l4 = cp.toString('/', 4);
+              String l1 = cp.subpath(1).toString('/');
+              String l2 = cp.subpath(2).toString('/');
+              String l3 = cp.subpath(3).toString('/');
+              String l4 = cp.subpath(4).toString('/');
               values.put(l1, l1);
               values.put(l2, l2);
               values.put(l3, l3);
@@ -292,11 +288,11 @@ public class TestDirectoryTaxonomyWriter
     for (String cat : values.keySet()) {
       CategoryPath cp = new CategoryPath(cat, '/');
       assertTrue("category not found " + cp, dtr.getOrdinal(cp) > 0);
-      int level = cp.length();
+      int level = cp.length;
       int parentOrd = 0; // for root, parent is always virtual ROOT (ord=0)
-      CategoryPath path = new CategoryPath();
+      CategoryPath path = CategoryPath.EMPTY;
       for (int i = 0; i < level; i++) {
-        path.add(cp.getComponent(i));
+        path = cp.subpath(i + 1);
         int ord = dtr.getOrdinal(path);
         assertEquals("invalid parent for cp=" + path, parentOrd, parents[ord]);
         parentOrd = ord; // next level should have this parent

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/cl2o/TestCompactLabelToOrdinal.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/cl2o/TestCompactLabelToOrdinal.java?rev=1429570&r1=1429569&r2=1429570&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/cl2o/TestCompactLabelToOrdinal.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/cl2o/TestCompactLabelToOrdinal.java Sun Jan  6 19:07:44 2013
@@ -6,11 +6,13 @@ import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Random;
 
 import org.junit.Test;
 
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.writercache.cl2o.CompactLabelToOrdinal;
 import org.apache.lucene.facet.taxonomy.writercache.cl2o.LabelToOrdinal;
@@ -46,9 +48,10 @@ public class TestCompactLabelToOrdinal e
     String[] uniqueValues = new String[numUniqueValues];
     byte[] buffer = new byte[50];
 
+    Random random = random();
     for (int i = 0; i < numUniqueValues;) {
-      random().nextBytes(buffer);
-      int size = 1 + random().nextInt(50);
+      random.nextBytes(buffer);
+      int size = 1 + random.nextInt(buffer.length);
 
       // This test is turning random bytes into a string,
       // this is asking for trouble.
@@ -56,16 +59,16 @@ public class TestCompactLabelToOrdinal e
           .onUnmappableCharacter(CodingErrorAction.REPLACE)
           .onMalformedInput(CodingErrorAction.REPLACE);
       uniqueValues[i] = decoder.decode(ByteBuffer.wrap(buffer, 0, size)).toString();
-      if (uniqueValues[i].indexOf(CompactLabelToOrdinal.TerminatorChar) == -1) {
+      if (uniqueValues[i].indexOf(CompactLabelToOrdinal.TERMINATOR_CHAR) == -1) {
         i++;
       }
     }
 
-    TEMP_DIR.mkdirs();
-    File f = new File(TEMP_DIR, "CompactLabelToOrdinalTest.tmp");
+    File tmpDir = _TestUtil.getTempDir("testLableToOrdinal");
+    File f = new File(tmpDir, "CompactLabelToOrdinalTest.tmp");
     int flushInterval = 10;
 
-    for (int i = 0; i < n * 10; i++) {
+    for (int i = 0; i < n; i++) {
       if (i > 0 && i % flushInterval == 0) {
         compact.flush(f);    
         compact = CompactLabelToOrdinal.open(f, 0.15f, 3);
@@ -75,19 +78,16 @@ public class TestCompactLabelToOrdinal e
         }
       }
 
-      int index = random().nextInt(numUniqueValues);
+      int index = random.nextInt(numUniqueValues);
       CategoryPath label = new CategoryPath(uniqueValues[index], '/');
 
       int ord1 = map.getOrdinal(label);
       int ord2 = compact.getOrdinal(label);
 
-      //System.err.println(ord1+" "+ord2);
-
       assertEquals(ord1, ord2);
 
-      if (ord1 == LabelToOrdinal.InvalidOrdinal) {
+      if (ord1 == LabelToOrdinal.INVALID_ORDINAL) {
         ord1 = compact.getNextOrdinal();
-
         map.addLabel(label, ord1);
         compact.addLabel(label, ord1);
       }
@@ -108,25 +108,15 @@ public class TestCompactLabelToOrdinal e
     
     @Override
     public void addLabel(CategoryPath label, int ordinal) {
-      map.put(new CategoryPath(label), ordinal);
-    }
-
-    @Override
-    public void addLabel(CategoryPath label, int prefixLen, int ordinal) {
-      map.put(new CategoryPath(label, prefixLen), ordinal);
+      map.put(label, ordinal);
     }
 
     @Override
     public int getOrdinal(CategoryPath label) {
       Integer value = map.get(label);
-      return (value != null) ? value.intValue() : LabelToOrdinal.InvalidOrdinal;
-    }
-
-    @Override
-    public int getOrdinal(CategoryPath label, int prefixLen) {
-      Integer value = map.get(new CategoryPath(label, prefixLen));
-      return (value != null) ? value.intValue() : LabelToOrdinal.InvalidOrdinal;
+      return (value != null) ? value.intValue() : LabelToOrdinal.INVALID_ORDINAL;
     }
 
   }
+
 }