You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2012/11/23 13:01:26 UTC

svn commit: r1412849 [6/13] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/maven/ dev-tools/maven/solr/contrib/dataimporthandler/ dev-tool...

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java Fri Nov 23 12:00:32 2012
@@ -3,6 +3,9 @@ package org.apache.lucene.facet.taxonomy
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.store.AlreadyClosedException;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -60,13 +63,13 @@ import java.util.Map;
  * 
  * @lucene.experimental
  */
-public interface TaxonomyReader extends Closeable {
+public abstract class TaxonomyReader implements Closeable {
   
   /**
-   * The root category (the category with the empty path) always has the
-   * ordinal 0, to which we give a name ROOT_ORDINAL.
-   * getOrdinal() of an empty path will always return ROOT_ORDINAL, and
-   * getCategory(ROOT_ORDINAL) will return the empty path.
+   * The root category (the category with the empty path) always has the ordinal
+   * 0, to which we give a name ROOT_ORDINAL. {@link #getOrdinal(CategoryPath)}
+   * of an empty path will always return {@code ROOT_ORDINAL}, and
+   * {@link #getPath(int)} with {@code ROOT_ORDINAL} will return the empty path.
    */
   public final static int ROOT_ORDINAL = 0;
   
@@ -77,207 +80,189 @@ public interface TaxonomyReader extends 
   public final static int INVALID_ORDINAL = -1;
   
   /**
-   * getOrdinal() returns the ordinal of the category given as a path.
-   * The ordinal is the category's serial number, an integer which starts
-   * with 0 and grows as more categories are added (note that once a category
-   * is added, it can never be deleted).
-   * <P>
-   * If the given category wasn't found in the taxonomy, INVALID_ORDINAL is
-   * returned.
-   */
-  public int getOrdinal(CategoryPath categoryPath) throws IOException;
-  
-  /**
-   * getPath() returns the path name of the category with the given
-   * ordinal. The path is returned as a new CategoryPath object - to
-   * reuse an existing object, use {@link #getPath(int, CategoryPath)}.
-   * <P>
-   * A null is returned if a category with the given ordinal does not exist. 
-   */
-  public CategoryPath getPath(int ordinal) throws IOException;
+   * If the taxonomy has changed since the provided reader was opened, open and
+   * return a new {@link TaxonomyReader}; else, return {@code null}. The new
+   * reader, if not {@code null}, will be the same type of reader as the one
+   * given to this method.
+   * 
+   * <p>
+   * This method is typically far less costly than opening a fully new
+   * {@link TaxonomyReader} as it shares resources with the provided
+   * {@link TaxonomyReader}, when possible.
+   */
+  public static <T extends TaxonomyReader> T openIfChanged(T oldTaxoReader) throws IOException {
+    @SuppressWarnings("unchecked")
+    final T newTaxoReader = (T) oldTaxoReader.doOpenIfChanged();
+    assert newTaxoReader != oldTaxoReader;
+    return newTaxoReader;
+  }
+
+  private volatile boolean closed = false;
 
+  // set refCount to 1 at start
+  private final AtomicInteger refCount = new AtomicInteger(1);
+  
+  /**
+   * performs the actual task of closing the resources that are used by the
+   * taxonomy reader.
+   */
+  protected abstract void doClose() throws IOException;
+  
   /**
-   * getPath() returns the path name of the category with the given
-   * ordinal. The path is written to the given CategoryPath object (which
-   * is cleared first).
-   * <P>
-   * If a category with the given ordinal does not exist, the given
-   * CategoryPath object is not modified, and the method returns
-   * <code>false</code>. Otherwise, the method returns <code>true</code>. 
+   * Implements the actual opening of a new {@link TaxonomyReader} instance if
+   * the taxonomy has changed.
+   * 
+   * @see #openIfChanged(TaxonomyReader)
    */
-  public boolean getPath(int ordinal, CategoryPath result) throws IOException;
+  protected abstract TaxonomyReader doOpenIfChanged() throws IOException;
+  
+  /**
+   * @throws AlreadyClosedException if this IndexReader is closed
+   */
+  protected final void ensureOpen() throws AlreadyClosedException {
+    if (getRefCount() <= 0) {
+      throw new AlreadyClosedException("this TaxonomyReader is closed");
+    }
+  }
 
+  @Override
+  public final void close() throws IOException {
+    if (!closed) {
+      synchronized (this) {
+        if (!closed) {
+          decRef();
+          closed = true;
+        }
+      }
+    }
+  }
+  
   /**
-   * refresh() re-reads the taxonomy information if there were any changes to
-   * the taxonomy since this instance was opened or last refreshed. Calling
-   * refresh() is more efficient than close()ing the old instance and opening a
-   * new one.
-   * <P>
-   * If there were no changes since this instance was opened or last refreshed,
-   * then this call does nothing. Note, however, that this is still a relatively
-   * slow method (as it needs to verify whether there have been any changes on
-   * disk to the taxonomy), so it should not be called too often needlessly. In
-   * faceted search, the taxonomy reader's refresh() should be called only after
-   * a reopen() of the main index.
-   * <P>
-   * Refreshing the taxonomy might fail in some cases, for example 
-   * if the taxonomy was recreated since this instance was opened or last refreshed.
-   * In this case an {@link InconsistentTaxonomyException} is thrown,
-   * suggesting that in order to obtain up-to-date taxonomy data a new
-   * {@link TaxonomyReader} should be opened. Note: This {@link TaxonomyReader} 
-   * instance remains unchanged and usable in this case, and the application can
-   * continue to use it, and should still {@link #close()} when no longer needed.  
-   * <P>
-   * It should be noted that refresh() is similar in purpose to
-   * IndexReader.reopen(), but the two methods behave differently. refresh()
-   * refreshes the existing TaxonomyReader object, rather than opening a new one
-   * in addition to the old one as reopen() does. The reason is that in a
-   * taxonomy, one can only add new categories and cannot modify or delete
-   * existing categories; Therefore, there is no reason to keep an old snapshot
-   * of the taxonomy open - refreshing the taxonomy to the newest data and using
-   * this new snapshots in all threads (whether new or old) is fine. This saves
-   * us needing to keep multiple copies of the taxonomy open in memory.
-   * @return true if anything has changed, false otherwise. 
-   */
-  public boolean refresh() throws IOException, InconsistentTaxonomyException;
-  
-  /**
-   * getParent() returns the ordinal of the parent category of the category
-   * with the given ordinal.
-   * <P>
-   * When a category is specified as a path name, finding the path of its
-   * parent is as trivial as dropping the last component of the path.
-   * getParent() is functionally equivalent to calling getPath() on the
-   * given ordinal, dropping the last component of the path, and then calling
-   * getOrdinal() to get an ordinal back. However, implementations are
-   * expected to provide a much more efficient implementation:
-   * <P>
-   * getParent() should be a very quick method, as it is used during the
-   * facet aggregation process in faceted search. Implementations will most
-   * likely want to serve replies to this method from a pre-filled cache.
-   * <P>
-   * If the given ordinal is the ROOT_ORDINAL, an INVALID_ORDINAL is returned.
-   * If the given ordinal is a top-level category, the ROOT_ORDINAL is returned.
-   * If an invalid ordinal is given (negative or beyond the last available
-   * ordinal), an ArrayIndexOutOfBoundsException is thrown. However, it is
-   * expected that getParent will only be called for ordinals which are
-   * already known to be in the taxonomy.
-   */
-  public int getParent(int ordinal) throws IOException;
-  
-  /**
-   * getParentArray() returns an int array of size getSize() listing the
-   * ordinal of the parent category of each category in the taxonomy.
-   * <P>
-   * The caller can hold on to the array it got indefinitely - it is
-   * guaranteed that no-one else will modify it. The other side of the
-   * same coin is that the caller must treat the array it got as read-only
-   * and <B>not modify it</B>, because other callers might have gotten the
-   * same array too (and getParent() calls might be answered from the
-   * same array).
-   * <P>
-   * If you use getParentArray() instead of getParent(), remember that
-   * the array you got is (naturally) not modified after a refresh(),
-   * so you should always call getParentArray() again after a refresh().
-   * <P>
-   * This method's function is similar to allocating an array of size
-   * getSize() and filling it with getParent() calls, but implementations
-   * are encouraged to implement it much more efficiently, with O(1)
-   * complexity. This can be done, for example, by the implementation
-   * already keeping the parents in an array, and just returning this
-   * array (without any allocation or copying) when requested.
-   */
-  public int[] getParentArray() throws IOException;
-  
-  /**
-   * Equivalent representations of the taxonomy's parent info, 
-   * used internally for efficient computation of facet results: 
-   * "youngest child" and "oldest sibling"   
-   */
-  public static interface ChildrenArrays {
-    /**
-     * getYoungestChildArray() returns an int array of size getSize()
-     * listing the ordinal of the youngest (highest numbered) child
-     * category of each category in the taxonomy. The value for a leaf
-     * category (a category without children) is
-     * <code>INVALID_ORDINAL</code>.
-     */
-    public int[] getYoungestChildArray();
-    /**
-     * getOlderSiblingArray() returns an int array of size getSize()
-     * listing for each category the ordinal of its immediate older
-     * sibling (the sibling in the taxonomy tree with the highest ordinal
-     * below that of the given ordinal). The value for a category with no
-     * older sibling is <code>INVALID_ORDINAL</code>.
-     */
-    public int[] getOlderSiblingArray();
+   * Expert: decreases the refCount of this TaxonomyReader instance. If the
+   * refCount drops to 0 this taxonomy reader is closed.
+   */
+  public final void decRef() throws IOException {
+    ensureOpen();
+    final int rc = refCount.decrementAndGet();
+    if (rc == 0) {
+      boolean success = false;
+      try {
+        doClose();
+        closed = true;
+        success = true;
+      } finally {
+        if (!success) {
+          // Put reference back on failure
+          refCount.incrementAndGet();
+        }
+      }
+    } else if (rc < 0) {
+      throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
+    }
   }
   
   /**
-   * getChildrenArrays() returns a {@link ChildrenArrays} object which can
-   * be used together to efficiently enumerate the children of any category. 
-   * <P>
-   * The caller can hold on to the object it got indefinitely - it is
-   * guaranteed that no-one else will modify it. The other side of the
-   * same coin is that the caller must treat the object which it got (and
-   * the arrays it contains) as read-only and <B>not modify it</B>, because
-   * other callers might have gotten the same object too.
-   * <P>
-   * Implementations should have O(getSize()) time for the first call or
-   * after a refresh(), but O(1) time for further calls. In neither case
-   * there should be a need to read new data from disk. These guarantees
-   * are most likely achieved by calculating this object (based on the
-   * getParentArray()) when first needed, and later (if the taxonomy was not
-   * refreshed) returning the same object (without any allocation or copying)
-   * when requested.
-   * <P>
-   * The reason we have one method returning one object, rather than two
-   * methods returning two arrays, is to avoid race conditions in a multi-
-   * threaded application: We want to avoid the possibility of returning one
-   * new array and one old array, as those could not be used together.
+   * Returns a {@link ChildrenArrays} object which can be used together to
+   * efficiently enumerate the children of any category.
+   * <p>
+   * The caller can hold on to the object it got indefinitely - it is guaranteed
+   * that no-one else will modify it. The other side of the same coin is that
+   * the caller must treat the object which it got (and the arrays it contains)
+   * as read-only and <b>not modify it</b>, because other callers might have
+   * gotten the same object too.
    */
-  public ChildrenArrays getChildrenArrays();
+  public abstract ChildrenArrays getChildrenArrays() throws IOException;
   
   /**
    * Retrieve user committed data.
+   * 
    * @see TaxonomyWriter#commit(Map)
    */
-  public Map<String, String> getCommitUserData() throws IOException;
-
+  public abstract Map<String, String> getCommitUserData() throws IOException;
+  
   /**
-   * Expert: increments the refCount of this TaxonomyReader instance. 
-   * RefCounts can be used to determine when a taxonomy reader can be closed 
-   * safely, i.e. as soon as there are no more references. 
-   * Be sure to always call a corresponding decRef(), in a finally clause; 
-   * otherwise the reader may never be closed. 
+   * Returns the ordinal of the category given as a path. The ordinal is the
+   * category's serial number, an integer which starts with 0 and grows as more
+   * categories are added (note that once a category is added, it can never be
+   * deleted).
+   * 
+   * @return the category's ordinal or {@link #INVALID_ORDINAL} if the category
+   *         wasn't foun.
    */
-  public void incRef();
-
+  public abstract int getOrdinal(CategoryPath categoryPath) throws IOException;
+  
   /**
-   * Expert: decreases the refCount of this TaxonomyReader instance. 
-   * If the refCount drops to 0, then pending changes (if any) can be  
-   * committed to the taxonomy index and this reader can be closed. 
-   * @throws IOException If there is a low-level I/O error.
+   * Returns the ordinal of the parent category of the category with the given
+   * ordinal, according to the following rules:
+   * 
+   * 
+   * <ul>
+   * <li>If the given ordinal is the {@link #ROOT_ORDINAL}, an
+   * {@link #INVALID_ORDINAL} is returned.
+   * <li>If the given ordinal is a top-level category, the {@link #ROOT_ORDINAL}
+   * is returned.
+   * <li>If the given ordinal is an existing category, returns the ordinal of
+   * its parent
+   * </ul>
+   * 
+   * @throws ArrayIndexOutOfBoundsException
+   *           if an invalid ordinal is given (negative or beyond the last
+   *           available ordinal)
    */
-  public void decRef() throws IOException;
+  public abstract int getParent(int ordinal) throws IOException;
   
   /**
-   * Expert: returns the current refCount for this taxonomy reader
+   * Returns an {@code int[]} the size of the taxonomy listing the ordinal of
+   * the parent category of each category in the taxonomy.
+   * <p>
+   * The caller can hold on to the array it got indefinitely - it is guaranteed
+   * that no-one else will modify it. The other side of the same coin is that
+   * the caller must treat the array it got as read-only and <b>not modify
+   * it</b>, because other callers might have gotten the same array too (and
+   * getParent() calls might be answered from the same array).
    */
-  public int getRefCount();
+  public abstract int[] getParentArray() throws IOException;
+  
+  /**
+   * Returns the path name of the category with the given ordinal. The path is
+   * returned as a new CategoryPath object - to reuse an existing object, use
+   * {@link #getPath(int, CategoryPath)}.
+   * 
+   * @return a {@link CategoryPath} with the required path, or {@code null} if
+   *         the given ordinal is unknown to the taxonomy.
+   */
+  public abstract CategoryPath getPath(int ordinal) throws IOException;
+  
+  /**
+   * Same as {@link #getPath(int)}, only reuses the given {@link CategoryPath}
+   * instances.
+   */
+  public abstract boolean getPath(int ordinal, CategoryPath result) throws IOException;
 
+  /** Returns the current refCount for this taxonomy reader. */
+  public final int getRefCount() {
+    return refCount.get();
+  }
+  
   /**
-   * getSize() returns the number of categories in the taxonomy.
-   * <P>
-   * Because categories are numbered consecutively starting with 0, it
-   * means the taxonomy contains ordinals 0 through getSize()-1.
-   * <P>
-   * Note that the number returned by getSize() is often slightly higher
-   * than the number of categories inserted into the taxonomy; This is
-   * because when a category is added to the taxonomy, its ancestors
-   * are also added automatically (including the root, which always get
-   * ordinal 0).
+   * Returns the number of categories in the taxonomy. Note that the number of
+   * categories returned is often slightly higher than the number of categories
+   * inserted into the taxonomy; This is because when a category is added to the
+   * taxonomy, its ancestors are also added automatically (including the root,
+   * which always get ordinal 0).
    */
-  public int getSize();
+  public abstract int getSize();
+  
+  /**
+   * Expert: increments the refCount of this TaxonomyReader instance. RefCounts
+   * can be used to determine when a taxonomy reader can be closed safely, i.e.
+   * as soon as there are no more references. Be sure to always call a
+   * corresponding decRef(), in a finally clause; otherwise the reader may never
+   * be closed.
+   */
+  public final void incRef() {
+    ensureOpen();
+    refCount.incrementAndGet();
+  }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/Consts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/Consts.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/Consts.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/Consts.java Fri Nov 23 12:00:32 2012
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.taxonomy
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.util.BytesRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -28,6 +29,7 @@ abstract class Consts {
   static final String FULL = "$full_path$";
   static final String FIELD_PAYLOADS = "$payloads$";
   static final String PAYLOAD_PARENT = "p";
+  static final BytesRef PAYLOAD_PARENT_BYTES_REF = new BytesRef(PAYLOAD_PARENT);
   static final char[] PAYLOAD_PARENT_CHARS = PAYLOAD_PARENT.toCharArray();
 
   /**

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java Fri Nov 23 12:00:32 2012
@@ -1,29 +1,23 @@
 package org.apache.lucene.facet.taxonomy.directory;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.lucene.facet.taxonomy.CategoryPath;
-import org.apache.lucene.facet.taxonomy.InconsistentTaxonomyException;
+import org.apache.lucene.facet.taxonomy.ChildrenArrays;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.Consts.LoadFullPathOnly;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.collections.LRUHashMap;
 
 /*
@@ -55,169 +49,308 @@ import org.apache.lucene.util.collection
  * 
  * @lucene.experimental
  */
-public class DirectoryTaxonomyReader implements TaxonomyReader {
+public class DirectoryTaxonomyReader extends TaxonomyReader {
 
   private static final Logger logger = Logger.getLogger(DirectoryTaxonomyReader.class.getName());
-  
-  private DirectoryReader indexReader;
 
-  // The following lock is used to allow multiple threads to read from the
-  // index concurrently, while having them block during the very short
-  // critical moment of refresh() (see comments below). Note, however, that
-  // we only read from the index when we don't have the entry in our cache,
-  // and the caches are locked separately.
-  private ReadWriteLock indexReaderLock = new ReentrantReadWriteLock();
-
-  // The following are the limited-size LRU caches used to cache the latest
-  // results from getOrdinal() and getLabel().
-  // Because LRUHashMap is not thread-safe, we need to synchronize on this
-  // object when using it. Unfortunately, this is not optimal under heavy
-  // contention because it means that while one thread is using the cache
-  // (reading or modifying) others are blocked from using it - or even
-  // starting to do benign things like calculating the hash function. A more
-  // efficient approach would be to use a non-locking (as much as possible)
-  // concurrent solution, along the lines of java.util.concurrent.ConcurrentHashMap
-  // but with LRU semantics.
-  // However, even in the current sub-optimal implementation we do not make
-  // the mistake of locking out readers while waiting for disk in a cache
-  // miss - below, we do not hold cache lock while reading missing data from
-  // disk.
-  private final LRUHashMap<String, Integer> ordinalCache;
-  private final LRUHashMap<Integer, String> categoryCache;
-
-  // getParent() needs to be extremely efficient, to the point that we need
-  // to fetch all the data in advance into memory, and answer these calls
-  // from memory. Currently we use a large integer array, which is
-  // initialized when the taxonomy is opened, and potentially enlarged
-  // when it is refresh()ed.
-  // These arrays are not syncrhonized. Rather, the reference to the array
-  // is volatile, and the only writing operation (refreshPrefetchArrays)
-  // simply creates a new array and replaces the reference. The volatility
-  // of the reference ensures the correct atomic replacement and its
-  // visibility properties (the content of the array is visible when the
-  // new reference is visible).
-  private ParentArray parentArray;
+  private static final int DEFAULT_CACHE_VALUE = 4000;
+  
+  private final DirectoryTaxonomyWriter taxoWriter;
+  private final long taxoEpoch; // used in doOpenIfChanged 
+  private final DirectoryReader indexReader;
+
+  // TODO: test DoubleBarrelLRUCache and consider using it instead
+  private LRUHashMap<String, Integer> ordinalCache;
+  private LRUHashMap<Integer, String> categoryCache;
+
+  // TODO: consolidate these objects into one ParentInfo or something?
+  private volatile ParentArray parentArray;
+  private volatile ChildrenArrays childrenArrays;
 
   private char delimiter = Consts.DEFAULT_DELIMITER;
 
-  private volatile boolean closed = false;
-  
-  // set refCount to 1 at start
-  private final AtomicInteger refCount = new AtomicInteger(1);
+  /**
+   * Called only from {@link #doOpenIfChanged()}. If the taxonomy has been
+   * recreated, you should pass {@code null} as the caches and parent/children
+   * arrays.
+   */
+  DirectoryTaxonomyReader(DirectoryReader indexReader, DirectoryTaxonomyWriter taxoWriter,
+      LRUHashMap<String,Integer> ordinalCache,
+      LRUHashMap<Integer,String> categoryCache, ParentArray parentArray,
+      ChildrenArrays childrenArrays) throws IOException {
+    this.indexReader = indexReader;
+    this.taxoWriter = taxoWriter;
+    this.taxoEpoch = taxoWriter == null ? -1 : taxoWriter.getTaxonomyEpoch();
+    
+    // use the same instance of the cache, note the protective code in getOrdinal and getPath
+    this.ordinalCache = ordinalCache == null ? new LRUHashMap<String,Integer>(DEFAULT_CACHE_VALUE) : ordinalCache;
+    this.categoryCache = categoryCache == null ? new LRUHashMap<Integer,String>(DEFAULT_CACHE_VALUE) : categoryCache;
+    
+    this.parentArray = null;
+    this.childrenArrays = null;
+    if (parentArray != null) {
+      this.parentArray = new ParentArray(indexReader, parentArray);
+      if (childrenArrays != null) {
+        this.childrenArrays = new ChildrenArrays(this.parentArray.getArray(), childrenArrays);
+      }
+    }
+  }
   
   /**
    * Open for reading a taxonomy stored in a given {@link Directory}.
+   * 
    * @param directory
-   *    The {@link Directory} in which to the taxonomy lives. Note that
-   *    the taxonomy is read directly to that directory (not from a
-   *    subdirectory of it).
-   * @throws CorruptIndexException if the Taxonomy is corrupted.
-   * @throws IOException if another error occurred.
+   *          The {@link Directory} in which the taxonomy resides.
+   * @throws CorruptIndexException
+   *           if the Taxonomy is corrupt.
+   * @throws IOException
+   *           if another error occurred.
    */
   public DirectoryTaxonomyReader(Directory directory) throws IOException {
-    this.indexReader = openIndexReader(directory);
+    indexReader = openIndexReader(directory);
+    taxoWriter = null;
+    taxoEpoch = -1;
 
     // These are the default cache sizes; they can be configured after
     // construction with the cache's setMaxSize() method
-    ordinalCache = new LRUHashMap<String, Integer>(4000);
-    categoryCache = new LRUHashMap<Integer, String>(4000);
-
-    // TODO (Facet): consider lazily create parent array when asked, not in the constructor
-    parentArray = new ParentArray();
-    parentArray.refresh(indexReader);
+    ordinalCache = new LRUHashMap<String, Integer>(DEFAULT_CACHE_VALUE);
+    categoryCache = new LRUHashMap<Integer, String>(DEFAULT_CACHE_VALUE);
   }
-
-  protected DirectoryReader openIndexReader(Directory directory) throws IOException {
-    return DirectoryReader.open(directory);
-  }
-
+  
   /**
-   * @throws AlreadyClosedException if this IndexReader is closed
+   * Opens a {@link DirectoryTaxonomyReader} over the given
+   * {@link DirectoryTaxonomyWriter} (for NRT).
+   * 
+   * @param taxoWriter
+   *          The {@link DirectoryTaxonomyWriter} from which to obtain newly
+   *          added categories, in real-time.
    */
-  protected final void ensureOpen() throws AlreadyClosedException {
-    if (getRefCount() <= 0) {
-      throw new AlreadyClosedException("this TaxonomyReader is closed");
+  public DirectoryTaxonomyReader(DirectoryTaxonomyWriter taxoWriter) throws IOException {
+    this.taxoWriter = taxoWriter;
+    taxoEpoch = taxoWriter.getTaxonomyEpoch();
+    indexReader = openIndexReader(taxoWriter.getInternalIndexWriter());
+    
+    // These are the default cache sizes; they can be configured after
+    // construction with the cache's setMaxSize() method
+    ordinalCache = new LRUHashMap<String, Integer>(DEFAULT_CACHE_VALUE);
+    categoryCache = new LRUHashMap<Integer, String>(DEFAULT_CACHE_VALUE);
+  }
+  
+  private String getLabel(int catID) throws IOException {
+    ensureOpen();
+
+    // Since the cache is shared with DTR instances allocated from
+    // doOpenIfChanged, we need to ensure that the ordinal is one that this DTR
+    // instance recognizes. Therefore we do this check up front, before we hit
+    // the cache.
+    if (catID < 0 || catID >= indexReader.maxDoc()) {
+      return null;
+    }
+    
+    // TODO: can we use an int-based hash impl, such as IntToObjectMap,
+    // wrapped as LRU?
+    Integer catIDInteger = Integer.valueOf(catID);
+    synchronized (categoryCache) {
+      String res = categoryCache.get(catIDInteger);
+      if (res != null) {
+        return res;
+      }
     }
+
+    final LoadFullPathOnly loader = new LoadFullPathOnly();
+    indexReader.document(catID, loader);
+    String ret = loader.getFullPath();
+    synchronized (categoryCache) {
+      categoryCache.put(catIDInteger, ret);
+    }
+
+    return ret;
+  }
+  
+  @Override
+  protected void doClose() throws IOException {
+    indexReader.close();
+    parentArray = null;
+    childrenArrays = null;
+    // do not clear() the caches, as they may be used by other DTR instances.
+    ordinalCache = null;
+    categoryCache = null;
   }
   
   /**
-   * setCacheSize controls the maximum allowed size of each of the caches
-   * used by {@link #getPath(int)} and {@link #getOrdinal(CategoryPath)}.
-   * <P>
-   * Currently, if the given size is smaller than the current size of
-   * a cache, it will not shrink, and rather we be limited to its current
-   * size.
-   * @param size the new maximum cache size, in number of entries.
+   * Implements the opening of a new {@link DirectoryTaxonomyReader} instance if
+   * the taxonomy has changed.
+   * 
+   * <p>
+   * <b>NOTE:</b> the returned {@link DirectoryTaxonomyReader} shares the
+   * ordinal and category caches with this reader. This is not expected to cause
+   * any issues, unless the two instances continue to live. The reader
+   * guarantees that the two instances cannot affect each other in terms of
+   * correctness of the caches, however if the size of the cache is changed
+   * through {@link #setCacheSize(int)}, it will affect both reader instances.
    */
-  public void setCacheSize(int size) {
+  @Override
+  protected DirectoryTaxonomyReader doOpenIfChanged() throws IOException {
     ensureOpen();
-    synchronized(categoryCache) {
-      categoryCache.setMaxSize(size);
+    
+    final DirectoryReader r2;
+    if (taxoWriter == null) {
+      // not NRT
+      r2 = DirectoryReader.openIfChanged(indexReader);
+    } else {
+      // NRT
+      r2 = DirectoryReader.openIfChanged(indexReader, taxoWriter.getInternalIndexWriter(), false);
     }
-    synchronized(ordinalCache) {
-      ordinalCache.setMaxSize(size);
+    if (r2 == null) {
+      return null; // no changes, nothing to do
+    }
+
+    // check if the taxonomy was recreated
+    boolean success = false;
+    try {
+      boolean recreated = false;
+      if (taxoWriter == null) {
+        // not NRT, check epoch from commit data
+        String t1 = indexReader.getIndexCommit().getUserData().get(DirectoryTaxonomyWriter.INDEX_EPOCH);
+        String t2 = r2.getIndexCommit().getUserData().get(DirectoryTaxonomyWriter.INDEX_EPOCH);
+        if (t1 == null) {
+          if (t2 != null) {
+            recreated = true;
+          }
+        } else if (!t1.equals(t2)) {
+          // t1 != null and t2 cannot be null b/c DirTaxoWriter always puts the commit data.
+          // it's ok to use String.equals because we require the two epoch values to be the same.
+          recreated = true;
+        }
+      } else {
+        // NRT, compare current taxoWriter.epoch() vs the one that was given at construction
+        if (taxoEpoch != taxoWriter.getTaxonomyEpoch()) {
+          recreated = true;
+        }
+      }
+
+      final DirectoryTaxonomyReader newtr;
+      if (recreated) {
+        // if recreated, do not reuse anything from this instace. the information
+        // will be lazily computed by the new instance when needed.
+        newtr = new DirectoryTaxonomyReader(r2, taxoWriter, null, null, null, null);
+      } else {
+        newtr = new DirectoryTaxonomyReader(r2, taxoWriter, ordinalCache, categoryCache, parentArray, childrenArrays);
+      }
+      
+      success = true;
+      return newtr;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(r2);
+      }
     }
   }
 
+  protected DirectoryReader openIndexReader(Directory directory) throws IOException {
+    return DirectoryReader.open(directory);
+  }
+
+  protected DirectoryReader openIndexReader(IndexWriter writer) throws IOException {
+    return DirectoryReader.open(writer, false);
+  }
+
   /**
-   * setDelimiter changes the character that the taxonomy uses in its
-   * internal storage as a delimiter between category components. Do not
-   * use this method unless you really know what you are doing.
-   * <P>
-   * If you do use this method, make sure you call it before any other
-   * methods that actually queries the taxonomy. Moreover, make sure you
-   * always pass the same delimiter for all LuceneTaxonomyWriter and
-   * LuceneTaxonomyReader objects you create.
+   * Expert: returns the underlying {@link DirectoryReader} instance that is
+   * used by this {@link TaxonomyReader}.
    */
-  public void setDelimiter(char delimiter) {
+  DirectoryReader getInternalIndexReader() {
     ensureOpen();
-    this.delimiter = delimiter;
+    return indexReader;
   }
 
+  @Override
+  public ChildrenArrays getChildrenArrays() throws IOException {
+    ensureOpen();
+    if (childrenArrays == null) {
+      synchronized (this) {
+        if (childrenArrays == null) {
+          childrenArrays = new ChildrenArrays(getParentArray());
+        }
+      }      
+    }
+    return childrenArrays;
+  }
+
+  @Override
+  public Map<String, String> getCommitUserData() throws IOException {
+    ensureOpen();
+    return indexReader.getIndexCommit().getUserData();
+  }
+
+  @Override
   public int getOrdinal(CategoryPath categoryPath) throws IOException {
     ensureOpen();
-    if (categoryPath.length()==0) {
+    if (categoryPath.length() == 0) {
       return ROOT_ORDINAL;
     }
     String path = categoryPath.toString(delimiter);
 
     // First try to find the answer in the LRU cache:
-    synchronized(ordinalCache) {
+    synchronized (ordinalCache) {
       Integer res = ordinalCache.get(path);
-      if (res!=null) {
-        return res.intValue();
+      if (res != null) {
+        if (res.intValue() < indexReader.maxDoc()) {
+          // Since the cache is shared with DTR instances allocated from
+          // doOpenIfChanged, we need to ensure that the ordinal is one that
+          // this DTR instance recognizes.
+          return res.intValue();
+        } else {
+          // if we get here, it means that the category was found in the cache,
+          // but is not recognized by this TR instance. Therefore there's no
+          // need to continue search for the path on disk, because we won't find
+          // it there too.
+          return TaxonomyReader.INVALID_ORDINAL;
+        }
       }
     }
 
     // If we're still here, we have a cache miss. We need to fetch the
     // value from disk, and then also put it in the cache:
     int ret = TaxonomyReader.INVALID_ORDINAL;
-    try {
-      indexReaderLock.readLock().lock();
-      // TODO (Facet): avoid Multi*?
-      Bits liveDocs = MultiFields.getLiveDocs(indexReader);
-      DocsEnum docs = MultiFields.getTermDocsEnum(indexReader, liveDocs, Consts.FULL, new BytesRef(path), 0);
-      if (docs != null && docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
-        ret = docs.docID();
+    DocsEnum docs = MultiFields.getTermDocsEnum(indexReader, null, Consts.FULL, new BytesRef(path), 0);
+    if (docs != null && docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+      ret = docs.docID();
+      
+      // we only store the fact that a category exists, not its inexistence.
+      // This is required because the caches are shared with new DTR instances
+      // that are allocated from doOpenIfChanged. Therefore, if we only store
+      // information about found categories, we cannot accidently tell a new
+      // generation of DTR that a category does not exist.
+      synchronized (ordinalCache) {
+        ordinalCache.put(path, Integer.valueOf(ret));
       }
-    } finally {
-      indexReaderLock.readLock().unlock();
-    }
-
-    // Put the new value in the cache. Note that it is possible that while
-    // we were doing the above fetching (without the cache locked), some
-    // other thread already added the same category to the cache. We do
-    // not care about this possibilty, as LRUCache replaces previous values
-    // of the same keys (it doesn't store duplicates).
-    synchronized(ordinalCache) {
-      // GB: new Integer(int); creates a new object each and every time.
-      // Integer.valueOf(int) might not (See JavaDoc). 
-      ordinalCache.put(path, Integer.valueOf(ret));
     }
 
     return ret;
   }
 
+  // TODO: move to a ParentInfo class? (see TODO for parentArray)
+  @Override
+  public int getParent(int ordinal) throws IOException {
+    ensureOpen();
+    return getParentArray()[ordinal];
+  }
+
+  @Override
+  public int[] getParentArray() throws IOException {
+    ensureOpen();
+    if (parentArray == null) {
+      synchronized (this) {
+        if (parentArray == null) {
+          parentArray = new ParentArray(indexReader);
+        }
+      }
+    }
+    return parentArray.getArray();
+  }
+  
+  @Override
   public CategoryPath getPath(int ordinal) throws IOException {
     ensureOpen();
     // TODO (Facet): Currently, the LRU cache we use (getCategoryCache) holds
@@ -229,16 +362,17 @@ public class DirectoryTaxonomyReader imp
     // interface that CategoryPath implements, and this method should
     // return this interface, not the writable CategoryPath.
     String label = getLabel(ordinal);
-    if (label==null) {
-      return null;  
+    if (label == null) {
+      return null;
     }
     return new CategoryPath(label, delimiter);
   }
 
+  @Override
   public boolean getPath(int ordinal, CategoryPath result) throws IOException {
     ensureOpen();
     String label = getLabel(ordinal);
-    if (label==null) {
+    if (label == null) {
       return false;
     }
     result.clear();
@@ -246,276 +380,50 @@ public class DirectoryTaxonomyReader imp
     return true;
   }
 
-  private String getLabel(int catID) throws IOException {
-    ensureOpen();
-    // First try to find the answer in the LRU cache. It is very
-    // unfortunate that we need to allocate an Integer object here -
-    // it would have been better if we used a hash table specifically
-    // designed for int keys...
-    // GB: new Integer(int); creates a new object each and every time.
-    // Integer.valueOf(int) might not (See JavaDoc). 
-    Integer catIDInteger = Integer.valueOf(catID);
-
-    synchronized(categoryCache) {
-      String res = categoryCache.get(catIDInteger);
-      if (res!=null) {
-        return res;
-      }
-    }
-
-    // If we're still here, we have a cache miss. We need to fetch the
-    // value from disk, and then also put it in the cache:
-    String ret;
-    try {
-      indexReaderLock.readLock().lock();
-      // The taxonomy API dictates that if we get an invalid category
-      // ID, we should return null, If we don't check this here, we
-      // can some sort of an exception from the document() call below.
-      // NOTE: Currently, we *do not* cache this return value; There
-      // isn't much point to do so, because checking the validity of
-      // the docid doesn't require disk access - just comparing with
-      // the number indexReader.maxDoc().
-      if (catID<0 || catID>=indexReader.maxDoc()) {
-        return null;
-      }
-      final LoadFullPathOnly loader = new LoadFullPathOnly();
-      indexReader.document(catID, loader);
-      ret = loader.getFullPath();
-    } finally {
-      indexReaderLock.readLock().unlock();
-    }
-    // Put the new value in the cache. Note that it is possible that while
-    // we were doing the above fetching (without the cache locked), some
-    // other thread already added the same category to the cache. We do
-    // not care about this possibility, as LRUCache replaces previous
-    // values of the same keys (it doesn't store duplicates).
-    synchronized (categoryCache) {
-      categoryCache.put(catIDInteger, ret);
-    }
-
-    return ret;
-  }
-
-  public int getParent(int ordinal) {
+  @Override
+  public int getSize() {
     ensureOpen();
-    // Note how we don't need to hold the read lock to do the following,
-    // because the array reference is volatile, ensuring the correct
-    // visibility and ordering: if we get the new reference, the new
-    // data is also visible to this thread.
-    return getParentArray()[ordinal];
+    return indexReader.numDocs();
   }
-
+  
   /**
-   * getParentArray() returns an int array of size getSize() listing the
-   * ordinal of the parent category of each category in the taxonomy.
-   * <P>
-   * The caller can hold on to the array it got indefinitely - it is
-   * guaranteed that no-one else will modify it. The other side of the
-   * same coin is that the caller must treat the array it got as read-only
-   * and <B>not modify it</B>, because other callers might have gotten the
-   * same array too, and getParent() calls are also answered from the
-   * same array.
-   * <P>
-   * The getParentArray() call is extremely efficient, merely returning
-   * a reference to an array that already exists. For a caller that plans
-   * to call getParent() for many categories, using getParentArray() and
-   * the array it returns is a somewhat faster approach because it avoids
-   * the overhead of method calls and volatile dereferencing.
+   * setCacheSize controls the maximum allowed size of each of the caches
+   * used by {@link #getPath(int)} and {@link #getOrdinal(CategoryPath)}.
    * <P>
-   * If you use getParentArray() instead of getParent(), remember that
-   * the array you got is (naturally) not modified after a refresh(),
-   * so you should always call getParentArray() again after a refresh().
+   * Currently, if the given size is smaller than the current size of
+   * a cache, it will not shrink, and rather we be limited to its current
+   * size.
+   * @param size the new maximum cache size, in number of entries.
    */
-
-  public int[] getParentArray() {
+  public void setCacheSize(int size) {
     ensureOpen();
-    // Note how we don't need to hold the read lock to do the following,
-    // because the array reference is volatile, ensuring the correct
-    // visibility and ordering: if we get the new reference, the new
-    // data is also visible to this thread.
-    return parentArray.getArray();
-  }
-
-  // Note that refresh() is synchronized (it is the only synchronized
-  // method in this class) to ensure that it never gets called concurrently
-  // with itself.
-  public synchronized boolean refresh() throws IOException, InconsistentTaxonomyException {
-    ensureOpen();
-    /*
-     * Since refresh() can be a lengthy operation, it is very important that we
-     * avoid locking out all readers for its duration. This is why we don't hold
-     * the indexReaderLock write lock for the entire duration of this method. In
-     * fact, it is enough to hold it only during a single assignment! Other
-     * comments in this method will explain this.
-     */
-
-    // note that the lengthy operation indexReader.reopen() does not
-    // modify the reader, so we can do it without holding a lock. We can
-    // safely read indexReader without holding the write lock, because
-    // no other thread can be writing at this time (this method is the
-    // only possible writer, and it is "synchronized" to avoid this case).
-    DirectoryReader r2 = DirectoryReader.openIfChanged(indexReader);
-    if (r2 == null) {
-      return false; // no changes, nothing to do
-    } 
-    
-    // validate that a refresh is valid at this point, i.e. that the taxonomy 
-    // was not recreated since this reader was last opened or refresshed.
-    String t1 = indexReader.getIndexCommit().getUserData().get(DirectoryTaxonomyWriter.INDEX_CREATE_TIME);
-    String t2 = r2.getIndexCommit().getUserData().get(DirectoryTaxonomyWriter.INDEX_CREATE_TIME);
-    if (t1==null) {
-      if (t2!=null) {
-        r2.close();
-        throw new InconsistentTaxonomyException("Taxonomy was recreated at: "+t2);
-      }
-    } else if (!t1.equals(t2)) {
-      r2.close();
-      throw new InconsistentTaxonomyException("Taxonomy was recreated at: "+t2+"  !=  "+t1);
-    }
-    
-      IndexReader oldreader = indexReader;
-      // we can close the old searcher, but need to synchronize this
-      // so that we don't close it in the middle that another routine
-      // is reading from it.
-      indexReaderLock.writeLock().lock();
-      indexReader = r2;
-      indexReaderLock.writeLock().unlock();
-      // We can close the old reader, but need to be certain that we
-      // don't close it while another method is reading from it.
-      // Luckily, we can be certain of that even without putting the
-      // oldreader.close() in the locked section. The reason is that
-      // after lock() succeeded above, we know that all existing readers
-      // had finished (this is what a read-write lock ensures). New
-      // readers, starting after the unlock() we just did, already got
-      // the new indexReader we set above. So nobody can be possibly
-      // using the old indexReader, and we can close it:
-      oldreader.close();
-
-      // We prefetch some of the arrays to make requests much faster.
-      // Let's refresh these prefetched arrays; This refresh is much
-      // is made more efficient by assuming that it is enough to read
-      // the values for new categories (old categories could not have been
-      // changed or deleted)
-      // Note that this this done without the write lock being held,
-      // which means that it is possible that during a refresh(), a
-      // reader will have some methods (like getOrdinal and getCategory)
-      // return fresh information, while getParent()
-      // (only to be prefetched now) still return older information.
-      // We consider this to be acceptable. The important thing,
-      // however, is that refreshPrefetchArrays() itself writes to
-      // the arrays in a correct manner (see discussion there)
-      parentArray.refresh(indexReader);
-
-      // Remove any INVALID_ORDINAL values from the ordinal cache,
-      // because it is possible those are now answered by the new data!
-      Iterator<Entry<String, Integer>> i = ordinalCache.entrySet().iterator();
-      while (i.hasNext()) {
-        Entry<String, Integer> e = i.next();
-        if (e.getValue().intValue() == INVALID_ORDINAL) {
-          i.remove();
-        }
-      }
-      return true;
-    }
-
-  public void close() throws IOException {
-    if (!closed) {
-      synchronized (this) {
-        if (!closed) {
-          decRef();
-          closed = true;
-        }
-      }
+    synchronized (categoryCache) {
+      categoryCache.setMaxSize(size);
     }
-  }
-  
-  /** Do the actual closing, free up resources */
-  private void doClose() throws IOException {
-    indexReader.close();
-    closed = true;
-
-    parentArray = null;
-    childrenArrays = null;
-    categoryCache.clear();
-    ordinalCache.clear();
-  }
-
-  public int getSize() {
-    ensureOpen();
-    indexReaderLock.readLock().lock();
-    try {
-      return indexReader.numDocs();
-    } finally {
-      indexReaderLock.readLock().unlock();
+    synchronized (ordinalCache) {
+      ordinalCache.setMaxSize(size);
     }
   }
 
-  public Map<String, String> getCommitUserData() throws IOException {
+  /**
+   * setDelimiter changes the character that the taxonomy uses in its
+   * internal storage as a delimiter between category components. Do not
+   * use this method unless you really know what you are doing.
+   * <P>
+   * If you do use this method, make sure you call it before any other
+   * methods that actually queries the taxonomy. Moreover, make sure you
+   * always pass the same delimiter for all LuceneTaxonomyWriter and
+   * LuceneTaxonomyReader objects you create.
+   */
+  public void setDelimiter(char delimiter) {
     ensureOpen();
-    return indexReader.getIndexCommit().getUserData();
+    this.delimiter = delimiter;
   }
   
-  private ChildrenArrays childrenArrays;
-  Object childrenArraysRebuild = new Object();
-
-  public ChildrenArrays getChildrenArrays() {
-    ensureOpen();
-    // Check if the taxonomy grew since we built the array, and if it
-    // did, create new (and larger) arrays and fill them as required.
-    // We do all this under a lock, two prevent to concurrent calls to
-    // needlessly do the same array building at the same time.
-    synchronized(childrenArraysRebuild) {
-      int num = getSize();
-      int first;
-      if (childrenArrays==null) {
-        first = 0;
-      } else {
-        first = childrenArrays.getYoungestChildArray().length;
-      }
-      // If the taxonomy hasn't grown, we can return the existing object
-      // immediately
-      if (first == num) {
-        return childrenArrays;
-      }
-      // Otherwise, build new arrays for a new ChildrenArray object.
-      // These arrays start with an enlarged copy of the previous arrays,
-      // and then are modified to take into account the new categories:
-      int[] newYoungestChildArray = new int[num];
-      int[] newOlderSiblingArray = new int[num];
-      // In Java 6, we could just do Arrays.copyOf()...
-      if (childrenArrays!=null) {
-        System.arraycopy(childrenArrays.getYoungestChildArray(), 0,
-            newYoungestChildArray, 0, childrenArrays.getYoungestChildArray().length);
-        System.arraycopy(childrenArrays.getOlderSiblingArray(), 0,
-            newOlderSiblingArray, 0, childrenArrays.getOlderSiblingArray().length);
-      }
-      int[] parents = getParentArray();
-      for (int i=first; i<num; i++) {
-        newYoungestChildArray[i] = INVALID_ORDINAL;
-      }
-      // In the loop below we can ignore the root category (0) because
-      // it has no parent
-      if (first==0) {
-        first = 1;
-        newOlderSiblingArray[0] = INVALID_ORDINAL;
-      }
-      for (int i=first; i<num; i++) {
-        // Note that parents[i] is always < i, so the right-hand-side of
-        // the following line is already set when we get here.
-        newOlderSiblingArray[i] = newYoungestChildArray[parents[i]];
-        newYoungestChildArray[parents[i]] = i;
-      }
-      // Finally switch to the new arrays
-      childrenArrays = new ChildrenArraysImpl(newYoungestChildArray,
-          newOlderSiblingArray);
-      return childrenArrays;
-    }
-  }
-
   public String toString(int max) {
     ensureOpen();
     StringBuilder sb = new StringBuilder();
-    int upperl = Math.min(max, this.indexReader.maxDoc());
+    int upperl = Math.min(max, indexReader.maxDoc());
     for (int i = 0; i < upperl; i++) {
       try {
         CategoryPath category = this.getPath(i);
@@ -536,70 +444,5 @@ public class DirectoryTaxonomyReader imp
     }
     return sb.toString();
   }
-
-  private static final class ChildrenArraysImpl implements ChildrenArrays {
-    private int[] youngestChildArray, olderSiblingArray;
-    public ChildrenArraysImpl(int[] youngestChildArray, int[] olderSiblingArray) {
-      this.youngestChildArray = youngestChildArray;
-      this.olderSiblingArray = olderSiblingArray;
-    }
-    public int[] getOlderSiblingArray() {
-      return olderSiblingArray;
-    }
-    public int[] getYoungestChildArray() {
-      return youngestChildArray;
-    }    
-  }
-
-  /**
-   * Expert:  This method is only for expert use.
-   * Note also that any call to refresh() will invalidate the returned reader,
-   * so the caller needs to take care of appropriate locking.
-   * 
-   * @return lucene indexReader
-   */
-  DirectoryReader getInternalIndexReader() {
-    ensureOpen();
-    return this.indexReader;
-  }
-
-  /**
-   * Expert: decreases the refCount of this TaxonomyReader instance. If the
-   * refCount drops to 0, then this reader is closed.
-   */
-  public void decRef() throws IOException {
-    ensureOpen();
-    final int rc = refCount.decrementAndGet();
-    if (rc == 0) {
-      boolean success = false;
-      try {
-        doClose();
-        success = true;
-      } finally {
-        if (!success) {
-          // Put reference back on failure
-          refCount.incrementAndGet();
-        }
-      }
-    } else if (rc < 0) {
-      throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
-    }
-  }
-  
-  /** Expert: returns the current refCount for this taxonomy reader */
-  public int getRefCount() {
-    return refCount.get();
-  }
   
-  /**
-   * Expert: increments the refCount of this TaxonomyReader instance. 
-   * RefCounts are used to determine when a taxonomy reader can be closed 
-   * safely, i.e. as soon as there are no more references. 
-   * Be sure to always call a corresponding decRef(), in a finally clause; 
-   * otherwise the reader may never be closed. 
-   */
-  public void incRef() {
-    ensureOpen();
-    refCount.incrementAndGet();
-  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java Fri Nov 23 12:00:32 2012
@@ -86,23 +86,24 @@ import org.apache.lucene.util.Version;
  * @lucene.experimental
  */
 public class DirectoryTaxonomyWriter implements TaxonomyWriter {
-
+  
   /**
-   * Property name of user commit data that contains the creation time of a
-   * taxonomy index.
+   * Property name of user commit data that contains the index epoch. The epoch
+   * changes whenever the taxonomy is recreated (i.e. opened with
+   * {@link OpenMode#CREATE}.
    * <p>
    * Applications should not use this property in their commit data because it
    * will be overridden by this taxonomy writer.
    */
-  public static final String INDEX_CREATE_TIME = "index.create.time";
-
+  public static final String INDEX_EPOCH = "index.epoch";
+  
   private final Directory dir;
   private final IndexWriter indexWriter;
   private final TaxonomyWriterCache cache;
   private final AtomicInteger cacheMisses = new AtomicInteger(0);
   
-  /** Records the taxonomy index creation time, updated on replaceTaxonomy as well. */
-  private String createTime;
+  // Records the taxonomy index epoch, updated on replaceTaxonomy as well.
+  private long indexEpoch;
   
   private char delimiter = Consts.DEFAULT_DELIMITER;
   private SinglePositionTokenStream parentStream = new SinglePositionTokenStream(Consts.PAYLOAD_PARENT);
@@ -200,28 +201,34 @@ public class DirectoryTaxonomyWriter imp
   public DirectoryTaxonomyWriter(Directory directory, OpenMode openMode,
       TaxonomyWriterCache cache) throws IOException {
 
-    if (!DirectoryReader.indexExists(directory) || openMode==OpenMode.CREATE) {
-      createTime = Long.toString(System.nanoTime());
-    } else {
-      Map<String, String> commitData = readCommitData(directory);
-      if (commitData != null) {
-        // It is ok if an existing index doesn't have commitData, or the
-        // INDEX_CREATE_TIME property. If ever it will be recreated, we'll set
-        // createTime accordingly in the above 'if'. 
-        createTime = commitData.get(INDEX_CREATE_TIME);
-      } else {
-        createTime = null;
-      }
-    }
-    
     dir = directory;
     IndexWriterConfig config = createIndexWriterConfig(openMode);
     indexWriter = openIndexWriter(dir, config);
-    
+
     // verify (to some extent) that merge policy in effect would preserve category docids 
     assert !(indexWriter.getConfig().getMergePolicy() instanceof TieredMergePolicy) : 
       "for preserving category docids, merging none-adjacent segments is not allowed";
     
+    // after we opened the writer, and the index is locked, it's safe to check
+    // the commit data and read the index epoch
+    openMode = config.getOpenMode();
+    if (!DirectoryReader.indexExists(directory)) {
+      indexEpoch = 1;
+    } else {
+      String epochStr = null;
+      Map<String, String> commitData = readCommitData(directory);
+      if (commitData != null) {
+        epochStr = commitData.get(INDEX_EPOCH);
+      }
+      // no commit data, or no epoch in it means an old taxonomy, so set its epoch to 1, for lack
+      // of a better value.
+      indexEpoch = epochStr == null ? 1 : Long.parseLong(epochStr);
+    }
+    
+    if (openMode == OpenMode.CREATE) {
+      ++indexEpoch;
+    }
+    
     FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
     ft.setOmitNorms(true);
     parentStreamField = new Field(Consts.FIELD_PAYLOADS, parentStream, ft);
@@ -287,6 +294,9 @@ public class DirectoryTaxonomyWriter imp
    * @param openMode see {@link OpenMode}
    */
   protected IndexWriterConfig createIndexWriterConfig(OpenMode openMode) {
+    // TODO: should we use a more optimized Codec, e.g. Pulsing (or write custom)?
+    // The taxonomy has a unique structure, where each term is associated with one document
+ 
     // Make sure we use a MergePolicy which always merges adjacent segments and thus
     // keeps the doc IDs ordered as well (this is crucial for the taxonomy index).
     return new IndexWriterConfig(Version.LUCENE_50,
@@ -576,7 +586,7 @@ public class DirectoryTaxonomyWriter imp
     addToCache(categoryPath, length, id);
     
     // also add to the parent array
-    getParentArray().add(id, parent);
+    parentArray = getParentArray().add(id, parent);
 
     return id;
   }
@@ -670,9 +680,7 @@ public class DirectoryTaxonomyWriter imp
     if (userData != null) {
       m.putAll(userData);
     }
-    if (createTime != null) {
-      m.put(INDEX_CREATE_TIME, createTime);
-    }
+    m.put(INDEX_EPOCH, Long.toString(indexEpoch));
     return m;
   }
   
@@ -806,10 +814,9 @@ public class DirectoryTaxonomyWriter imp
       synchronized (this) {
         if (parentArray == null) {
           initReaderManager();
-          parentArray = new ParentArray();
           DirectoryReader reader = readerManager.acquire();
           try {
-            parentArray.refresh(reader);
+            parentArray = new ParentArray(reader);
           } finally {
             readerManager.release(reader);
           }
@@ -1022,13 +1029,29 @@ public class DirectoryTaxonomyWriter imp
     cacheIsComplete = false;
     shouldFillCache = true;
     
-    // update createTime as a taxonomy replace is just like it has be recreated
-    createTime = Long.toString(System.nanoTime());
+    // update indexEpoch as a taxonomy replace is just like it has be recreated
+    ++indexEpoch;
   }
 
   /** Returns the {@link Directory} of this taxonomy writer. */
   public Directory getDirectory() {
     return dir;
   }
-
+  
+  /**
+   * Used by {@link DirectoryTaxonomyReader} to support NRT.
+   * <p>
+   * <b>NOTE:</b> you should not use the obtained {@link IndexWriter} in any
+   * way, other than opening an IndexReader on it, or otherwise, the taxonomy
+   * index may become corrupt!
+   */
+  final IndexWriter getInternalIndexWriter() {
+    return indexWriter;
+  }
+  
+  /** Used by {@link DirectoryTaxonomyReader} to support NRT. */
+  final long getTaxonomyEpoch() {
+    return indexEpoch;
+  }
+  
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/ParentArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/ParentArray.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/ParentArray.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/ParentArray.java Fri Nov 23 12:00:32 2012
@@ -2,15 +2,13 @@ package org.apache.lucene.facet.taxonomy
 
 import java.io.IOException;
 
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.util.ArrayUtil;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -29,55 +27,23 @@ import org.apache.lucene.facet.taxonomy.
  * limitations under the License.
  */
 
-// getParent() needs to be extremely efficient, to the point that we need
-// to fetch all the data in advance into memory, and answer these calls
-// from memory. Currently we use a large integer array, which is
-// initialized when the taxonomy is opened, and potentially enlarged
-// when it is refresh()ed.
 /**
  * @lucene.experimental
  */
 class ParentArray {
 
-  // These arrays are not syncrhonized. Rather, the reference to the array
-  // is volatile, and the only writing operation (refreshPrefetchArrays)
-  // simply creates a new array and replaces the reference. The volatility
-  // of the reference ensures the correct atomic replacement and its
-  // visibility properties (the content of the array is visible when the
-  // new reference is visible).
-  private volatile int prefetchParentOrdinal[] = null;
+  // TODO: maybe use PackedInts?
+  private final int[] parentOrdinals;
 
-  public int[] getArray() {
-    return prefetchParentOrdinal;
+  /** Used by {@link #add(int, int)} when the array needs to grow. */
+  ParentArray(int[] parentOrdinals) {
+    this.parentOrdinals = parentOrdinals;
   }
 
-  /**
-   * refreshPrefetch() refreshes the parent array. Initially, it fills the
-   * array from the positions of an appropriate posting list. If called during
-   * a refresh(), when the arrays already exist, only values for new documents
-   * (those beyond the last one in the array) are read from the positions and
-   * added to the arrays (that are appropriately enlarged). We assume (and
-   * this is indeed a correct assumption in our case) that existing categories
-   * are never modified or deleted.
-   */
-  void refresh(IndexReader indexReader) throws IOException {
-    // Note that it is not necessary for us to obtain the read lock.
-    // The reason is that we are only called from refresh() (precluding
-    // another concurrent writer) or from the constructor (when no method
-    // could be running).
-    // The write lock is also not held during the following code, meaning
-    // that reads *can* happen while this code is running. The "volatile"
-    // property of the prefetchParentOrdinal and prefetchDepth array
-    // references ensure the correct visibility property of the assignment
-    // but other than that, we do *not* guarantee that a reader will not
-    // use an old version of one of these arrays (or both) while a refresh
-    // is going on. But we find this acceptable - until a refresh has
-    // finished, the reader should not expect to see new information
-    // (and the old information is the same in the old and new versions).
-    int first;
-    int num = indexReader.maxDoc();
-    if (prefetchParentOrdinal==null) {
-      prefetchParentOrdinal = new int[num];
+  public ParentArray(IndexReader reader) throws IOException {
+    parentOrdinals = new int[reader.maxDoc()];
+    if (parentOrdinals.length > 0) {
+      initFromReader(reader, 0);
       // Starting Lucene 2.9, following the change LUCENE-1542, we can
       // no longer reliably read the parent "-1" (see comment in
       // LuceneTaxonomyWriter.SinglePositionTokenStream). We have no way
@@ -85,78 +51,80 @@ class ParentArray {
       // with existing indexes, so what we'll do instead is just
       // hard-code the parent of ordinal 0 to be -1, and assume (as is
       // indeed the case) that no other parent can be -1.
-      if (num>0) {
-        prefetchParentOrdinal[0] = TaxonomyReader.INVALID_ORDINAL;
-      }
-      first = 1;
-    } else {
-      first = prefetchParentOrdinal.length;
-      if (first==num) {
-        return; // nothing to do - no category was added
-      }
-      // In Java 6, we could just do Arrays.copyOf()...
-      int[] newarray = new int[num];
-      System.arraycopy(prefetchParentOrdinal, 0, newarray, 0,
-          prefetchParentOrdinal.length);
-      prefetchParentOrdinal = newarray;
+      parentOrdinals[0] = TaxonomyReader.INVALID_ORDINAL;
     }
+  }
+  
+  public ParentArray(IndexReader reader, ParentArray copyFrom) throws IOException {
+    assert copyFrom != null;
+
+    // note that copyParents.length may be equal to reader.maxDoc(). this is not a bug
+    // it may be caused if e.g. the taxonomy segments were merged, and so an updated
+    // NRT reader was obtained, even though nothing was changed. this is not very likely
+    // to happen.
+    int[] copyParents = copyFrom.getArray();
+    this.parentOrdinals = new int[reader.maxDoc()];
+    System.arraycopy(copyParents, 0, parentOrdinals, 0, copyParents.length);
+    initFromReader(reader, copyParents.length);
+  }
 
-    // Read the new part of the parents array from the positions:
-    // TODO (Facet): avoid Multi*?
-    Bits liveDocs = MultiFields.getLiveDocs(indexReader);
-    DocsAndPositionsEnum positions = MultiFields.getTermPositionsEnum(indexReader, liveDocs,
-                                                                      Consts.FIELD_PAYLOADS, new BytesRef(Consts.PAYLOAD_PARENT),
-                                                                      DocsAndPositionsEnum.FLAG_PAYLOADS);
-      if ((positions == null || positions.advance(first) == DocIdSetIterator.NO_MORE_DOCS) && first < num) {
-        throw new CorruptIndexException("Missing parent data for category " + first);
-      }
-      for (int i=first; i<num; i++) {
-        // Note that we know positions.doc() >= i (this is an
-        // invariant kept throughout this loop)
-        if (positions.docID()==i) {
-          if (positions.freq() == 0) { // shouldn't happen
-            throw new CorruptIndexException(
-                "Missing parent data for category "+i);
-          }
-
-          // TODO (Facet): keep a local (non-volatile) copy of the prefetchParentOrdinal
-          // reference, because access to volatile reference is slower (?).
-          // Note: The positions we get here are one less than the position
-          // increment we added originally, so we get here the right numbers:
-          prefetchParentOrdinal[i] = positions.nextPosition();
-
-          if (positions.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) {
-            if ( i+1 < num ) {
-              throw new CorruptIndexException(
-                  "Missing parent data for category "+(i+1));
-            }
-            break;
+  // Read the parents of the new categories
+  private void initFromReader(IndexReader reader, int first) throws IOException {
+    if (reader.maxDoc() == first) {
+      return;
+    }
+    
+    // it's ok to use MultiFields because we only iterate on one posting list.
+    // breaking it to loop over the leaves() only complicates code for no
+    // apparent gain.
+    DocsAndPositionsEnum positions = MultiFields.getTermPositionsEnum(reader, null,
+        Consts.FIELD_PAYLOADS, Consts.PAYLOAD_PARENT_BYTES_REF,
+        DocsAndPositionsEnum.FLAG_PAYLOADS);
+
+    // shouldn't really happen, if it does, something's wrong
+    if (positions == null || positions.advance(first) == DocIdSetIterator.NO_MORE_DOCS) {
+      throw new CorruptIndexException("Missing parent data for category " + first);
+    }
+    
+    int num = reader.maxDoc();
+    for (int i = first; i < num; i++) {
+      if (positions.docID() == i) {
+        if (positions.freq() == 0) { // shouldn't happen
+          throw new CorruptIndexException("Missing parent data for category " + i);
+        }
+        
+        parentOrdinals[i] = positions.nextPosition();
+        
+        if (positions.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) {
+          if (i + 1 < num) {
+            throw new CorruptIndexException("Missing parent data for category "+ (i + 1));
           }
-        } else { // this shouldn't happen
-        throw new CorruptIndexException(
-            "Missing parent data for category "+i);
+          break;
+        }
+      } else { // this shouldn't happen
+        throw new CorruptIndexException("Missing parent data for category " + i);
       }
     }
   }
-
+  
+  public int[] getArray() {
+    return parentOrdinals;
+  }
+  
   /**
-   * add() is used in LuceneTaxonomyWriter, not in LuceneTaxonomyReader.
-   * It is only called from a synchronized method, so it is not reentrant,
-   * and also doesn't need to worry about reads happening at the same time.
-   * 
-   * NOTE: add() and refresh() CANNOT be used together. If you call add(),
-   * this changes the arrays and refresh() can no longer be used.
+   * Adds the given ordinal/parent info and returns either a new instance if the
+   * underlying array had to grow, or this instance otherwise.
+   * <p>
+   * <b>NOTE:</b> you should call this method from a thread-safe code.
    */
-  void add(int ordinal, int parentOrdinal) {
-    if (ordinal >= prefetchParentOrdinal.length) {
-      // grow the array, if necessary.
-      // In Java 6, we could just do Arrays.copyOf()...
-      int[] newarray = new int[ordinal*2+1];
-      System.arraycopy(prefetchParentOrdinal, 0, newarray, 0,
-          prefetchParentOrdinal.length);
-      prefetchParentOrdinal = newarray;
+  ParentArray add(int ordinal, int parentOrdinal) {
+    if (ordinal >= parentOrdinals.length) {
+      int[] newarray = ArrayUtil.grow(parentOrdinals);
+      newarray[ordinal] = parentOrdinal;
+      return new ParentArray(newarray);
     }
-    prefetchParentOrdinal[ordinal] = parentOrdinal;
+    parentOrdinals[ordinal] = parentOrdinal;
+    return this;
   }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/LRUHashMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/LRUHashMap.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/LRUHashMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/LRUHashMap.java Fri Nov 23 12:00:32 2012
@@ -102,4 +102,10 @@ public class LRUHashMap<K,V> extends Lin
     return size() > maxSize;
   }
 
+  @SuppressWarnings("unchecked")
+  @Override
+  public LRUHashMap<K,V> clone() {
+    return (LRUHashMap<K,V>) super.clone();
+  }
+  
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java Fri Nov 23 12:00:32 2012
@@ -131,7 +131,7 @@ public class FacetTestUtils {
 
   public static class IndexTaxonomyReaderPair {
     public DirectoryReader indexReader;
-    public TaxonomyReader taxReader;
+    public DirectoryTaxonomyReader taxReader;
     public IndexSearcher indexSearcher;
 
     public void close() throws IOException {

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCounts.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCounts.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCounts.java Fri Nov 23 12:00:32 2012
@@ -78,11 +78,9 @@ public class TestTotalFacetCounts extend
     TestTotalFacetCountsCache.addFacets(iParams, writers[0].indexWriter, writers[0].taxWriter, "b", "c");
 
     // Commit Changes
-    writers[0].commit();
     writers[0].close();
 
-    IndexTaxonomyReaderPair[] readers = 
-      FacetTestUtils.createIndexTaxonomyReaderPair(dirs);
+    IndexTaxonomyReaderPair[] readers = FacetTestUtils.createIndexTaxonomyReaderPair(dirs);
     
     int[] intArray = new int[iParams.getPartitionSize()];
 
@@ -93,8 +91,7 @@ public class TestTotalFacetCounts extend
     tfcc.load(tmpFile, readers[0].indexReader, readers[0].taxReader, iParams);
     
     // now retrieve the one just loaded
-    TotalFacetCounts totalCounts = 
-      tfcc.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+    TotalFacetCounts totalCounts = tfcc.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
 
     int partition = 0;
     for (int i=0; i<expectedCounts.length; i+=partitionSize) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java Fri Nov 23 12:00:32 2012
@@ -297,23 +297,17 @@ public class TestTotalFacetCountsCache e
     writers[0].indexWriter.close();
     writers[0].taxWriter.close();
 
-    readers[0].taxReader.refresh();
+    DirectoryTaxonomyReader newTaxoReader = TaxonomyReader.openIfChanged(readers[0].taxReader);
+    assertNotNull(newTaxoReader);
+    assertTrue("should have received more cagtegories in updated taxonomy", newTaxoReader.getSize() > readers[0].taxReader.getSize());
+    readers[0].taxReader.close();
+    readers[0].taxReader = newTaxoReader;
+    
     DirectoryReader r2 = DirectoryReader.openIfChanged(readers[0].indexReader);
     assertNotNull(r2);
-    // Hold on to the 'original' reader so we can do some checks with it
-    IndexReader origReader = null;
-
-    assertTrue("Reader must be updated!", readers[0].indexReader != r2);
-    
-    // Set the 'original' reader
-    origReader = readers[0].indexReader;
-    // Set the new master index Reader
+    readers[0].indexReader.close();
     readers[0].indexReader = r2;
 
-    // Try to get total-counts the originalReader AGAIN, just for sanity. Should pull from the cache - not recomputed. 
-    assertTrue("Should be obtained from cache at 6th attempt",totalCounts == 
-      TFC.getTotalCounts(origReader, readers[0].taxReader, iParams, null));
-
     // now use the new reader - should recompute
     totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
     prevGen = assertRecomputed(totalCounts, prevGen, "after updating the index - 7th attempt!");
@@ -322,9 +316,7 @@ public class TestTotalFacetCountsCache e
     assertTrue("Should be obtained from cache at 8th attempt",totalCounts == 
       TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null));
     
-    origReader.close();
     readers[0].close();
-    r2.close();
     outputFile.delete();
     IOUtils.close(dirs[0]);
   }
@@ -380,7 +372,10 @@ public class TestTotalFacetCountsCache e
       writers[0].taxWriter.addCategory(new CategoryPath("foo", Integer.toString(i)));
     }
     writers[0].taxWriter.commit();
-    readers[0].taxReader.refresh();
+    DirectoryTaxonomyReader newTaxoReader = TaxonomyReader.openIfChanged(readers[0].taxReader);
+    assertNotNull(newTaxoReader);
+    readers[0].taxReader.close();
+    readers[0].taxReader = newTaxoReader;
 
     initCache();