pylucene 3.5.0-3
[pylucene.git] / lucene-java-3.5.0 / lucene / contrib / facet / src / java / org / apache / lucene / facet / taxonomy / directory / DirectoryTaxonomyWriter.java
diff --git a/lucene-java-3.5.0/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java b/lucene-java-3.5.0/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
new file mode 100644 (file)
index 0000000..aceae06
--- /dev/null
@@ -0,0 +1,1081 @@
+package org.apache.lucene.facet.taxonomy.directory;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.KeywordAnalyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Index;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.LogByteSizeMergePolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermDocs;
+import org.apache.lucene.index.TermEnum;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.store.NativeFSLockFactory;
+import org.apache.lucene.store.SimpleFSLockFactory;
+import org.apache.lucene.util.Version;
+
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
+import org.apache.lucene.facet.taxonomy.writercache.TaxonomyWriterCache;
+import org.apache.lucene.facet.taxonomy.writercache.cl2o.Cl2oTaxonomyWriterCache;
+import org.apache.lucene.facet.taxonomy.writercache.lru.LruTaxonomyWriterCache;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * {@link TaxonomyWriter} which uses a {@link Directory} to store the taxonomy
+ * information on disk, and keeps an additional in-memory cache of some or all
+ * categories.
+ * <p>
+ * In addition to the permanently-stored information in the {@link Directory},
+ * efficiency dictates that we also keep an in-memory cache of <B>recently
+ * seen</B> or <B>all</B> categories, so that we do not need to go back to disk
+ * for every category addition to see which ordinal this category already has,
+ * if any. A {@link TaxonomyWriterCache} object determines the specific caching
+ * algorithm used.
+ * <p>
+ * This class offers some hooks for extending classes to control the
+ * {@link IndexWriter} instance that is used. See {@link #openIndexWriter}.
+ * 
+ * @lucene.experimental
+ */
+public class DirectoryTaxonomyWriter implements TaxonomyWriter {
+
+  /**
+   * Property name of user commit data that contains the creation time of a taxonomy index.
+   * <p>
+   * Applications making use of {@link TaxonomyWriter#commit(Map)} should not use this
+   * particular property name. 
+   */
+  public static final String INDEX_CREATE_TIME = "index.create.time";
+  
+  private IndexWriter indexWriter;
+  private int nextID;
+  private char delimiter = Consts.DEFAULT_DELIMITER;
+  private SinglePositionTokenStream parentStream = new SinglePositionTokenStream(Consts.PAYLOAD_PARENT);
+  private Field parentStreamField;
+  private Field fullPathField;
+
+  private TaxonomyWriterCache cache;
+  /**
+   * We call the cache "complete" if we know that every category in our
+   * taxonomy is in the cache. When the cache is <B>not</B> complete, and
+   * we can't find a category in the cache, we still need to look for it
+   * in the on-disk index; Therefore when the cache is not complete, we
+   * need to open a "reader" to the taxonomy index.
+   * The cache becomes incomplete if it was never filled with the existing
+   * categories, or if a put() to the cache ever returned true (meaning
+   * that some of the cached data was cleared).
+   */
+  private boolean cacheIsComplete;
+  private IndexReader reader;
+  private int cacheMisses;
+
+  /**
+   * When a taxonomy is created, we mark that its create time should be committed in the 
+   * next commit.
+   */
+  private String taxoIndexCreateTime = null;
+  
+  /**
+   * 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. It has nothing to do
+   * with whatever character the application may be using to represent
+   * categories for its own use.
+   * <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 for the same directory.
+   */
+  public void setDelimiter(char delimiter) {
+    ensureOpen();
+    this.delimiter = delimiter;
+  }
+
+  /**
+   * Forcibly unlocks the taxonomy in the named directory.
+   * <P>
+   * Caution: this should only be used by failure recovery code, when it is
+   * known that no other process nor thread is in fact currently accessing
+   * this taxonomy.
+   * <P>
+   * This method is unnecessary if your {@link Directory} uses a
+   * {@link NativeFSLockFactory} instead of the default
+   * {@link SimpleFSLockFactory}. When the "native" lock is used, a lock
+   * does not stay behind forever when the process using it dies. 
+   */
+  public static void unlock(Directory directory) throws IOException {
+    IndexWriter.unlock(directory);
+  }
+
+  /**
+   * Construct a Taxonomy writer.
+   * 
+   * @param directory
+   *    The {@link Directory} in which to store the taxonomy. Note that
+   *    the taxonomy is written directly to that directory (not to a
+   *    subdirectory of it).
+   * @param openMode
+   *    Specifies how to open a taxonomy for writing: <code>APPEND</code>
+   *    means open an existing index for append (failing if the index does
+   *    not yet exist). <code>CREATE</code> means create a new index (first
+   *    deleting the old one if it already existed).
+   *    <code>APPEND_OR_CREATE</code> appends to an existing index if there
+   *    is one, otherwise it creates a new index.
+   * @param cache
+   *    A {@link TaxonomyWriterCache} implementation which determines
+   *    the in-memory caching policy. See for example
+   *    {@link LruTaxonomyWriterCache} and {@link Cl2oTaxonomyWriterCache}.
+   *    If null or missing, {@link #defaultTaxonomyWriterCache()} is used.
+   * @throws CorruptIndexException
+   *     if the taxonomy is corrupted.
+   * @throws LockObtainFailedException
+   *     if the taxonomy is locked by another writer. If it is known
+   *     that no other concurrent writer is active, the lock might
+   *     have been left around by an old dead process, and should be
+   *     removed using {@link #unlock(Directory)}.
+   * @throws IOException
+   *     if another error occurred.
+   */
+  public DirectoryTaxonomyWriter(Directory directory, OpenMode openMode,
+                              TaxonomyWriterCache cache)
+  throws CorruptIndexException, LockObtainFailedException,
+  IOException {
+
+    if (!IndexReader.indexExists(directory) || openMode==OpenMode.CREATE) {
+      taxoIndexCreateTime = Long.toString(System.nanoTime());
+    }
+    
+    indexWriter = openIndexWriter(directory, openMode);
+    reader = null;
+
+    parentStreamField = new Field(Consts.FIELD_PAYLOADS, parentStream);
+    parentStreamField.setOmitNorms(true);
+    fullPathField = new Field(Consts.FULL, "", Store.YES, Index.NOT_ANALYZED_NO_NORMS);
+    fullPathField.setIndexOptions(IndexOptions.DOCS_ONLY);
+
+    this.nextID = indexWriter.maxDoc();
+
+    if (cache==null) {
+      cache = defaultTaxonomyWriterCache();
+    }
+    this.cache = cache;
+
+    if (nextID == 0) {
+      cacheIsComplete = true;
+      // Make sure that the taxonomy always contain the root category
+      // with category id 0.
+      addCategory(new CategoryPath());
+      refreshReader();
+    } else {
+      // There are some categories on the disk, which we have not yet
+      // read into the cache, and therefore the cache is incomplete.
+      // We chose not to read all the categories into the cache now,
+      // to avoid terrible performance when a taxonomy index is opened
+      // to add just a single category. We will do it later, after we
+      // notice a few cache misses.
+      cacheIsComplete = false;
+    }
+    cacheMisses = 0;
+  }
+
+  /**
+   * A hook for extensions of this class to provide their own
+   * {@link IndexWriter} implementation or instance. Extending classes can
+   * instantiate and configure the {@link IndexWriter} as they see fit,
+   * including setting a {@link org.apache.lucene.index.MergeScheduler}, or
+   * {@link org.apache.lucene.index.IndexDeletionPolicy}, different RAM size
+   * etc.<br>
+   * <b>NOTE:</b> the instance this method returns will be closed upon calling
+   * to {@link #close()}.
+   * 
+   * @param directory
+   *          the {@link Directory} on top of which an {@link IndexWriter}
+   *          should be opened.
+   * @param openMode
+   *          see {@link OpenMode}
+   */
+  protected IndexWriter openIndexWriter(Directory directory, OpenMode openMode)
+      throws IOException {
+    // Make sure we use a MergePolicy which merges segments in-order and thus
+    // keeps the doc IDs ordered as well (this is crucial for the taxonomy
+    // index).
+    IndexWriterConfig config = new IndexWriterConfig(Version.LUCENE_30,
+        new KeywordAnalyzer()).setOpenMode(openMode).setMergePolicy(
+        new LogByteSizeMergePolicy());
+    return new IndexWriter(directory, config);
+  }
+
+  // Currently overridden by a unit test that verifies that every index we open
+  // is close()ed.
+  /**
+   * Open an {@link IndexReader} from the {@link #indexWriter} member, by
+   * calling {@link IndexWriter#getReader()}. Extending classes can override
+   * this method to return their own {@link IndexReader}.
+   */
+  protected IndexReader openReader() throws IOException {
+    return IndexReader.open(indexWriter, true); 
+  }
+
+  /**
+   * Creates a new instance with a default cached as defined by
+   * {@link #defaultTaxonomyWriterCache()}.
+   */
+  public DirectoryTaxonomyWriter(Directory directory, OpenMode openMode)
+  throws CorruptIndexException, LockObtainFailedException, IOException {
+    this(directory, openMode, defaultTaxonomyWriterCache());
+  }
+
+  /**
+   * Defines the default {@link TaxonomyWriterCache} to use in constructors
+   * which do not specify one.
+   * <P>  
+   * The current default is {@link Cl2oTaxonomyWriterCache} constructed
+   * with the parameters (1024, 0.15f, 3), i.e., the entire taxonomy is
+   * cached in memory while building it.
+   */
+  public static TaxonomyWriterCache defaultTaxonomyWriterCache() {
+    return new Cl2oTaxonomyWriterCache(1024, 0.15f, 3);
+  }
+
+  // convenience constructors:
+
+  public DirectoryTaxonomyWriter(Directory d)
+  throws CorruptIndexException, LockObtainFailedException,
+  IOException {
+    this(d, OpenMode.CREATE_OR_APPEND);
+  }
+
+  /**
+   * Frees used resources as well as closes the underlying {@link IndexWriter},
+   * which commits whatever changes made to it to the underlying
+   * {@link Directory}.
+   */
+  public synchronized void close() throws CorruptIndexException, IOException {
+    if (indexWriter != null) {
+      if (taxoIndexCreateTime != null) {
+        indexWriter.commit(combinedCommitData(null));
+        taxoIndexCreateTime = null;
+      }
+      doClose();
+    }
+  }
+  
+  private void doClose() throws CorruptIndexException, IOException {
+    indexWriter.close();
+    indexWriter = null;
+    closeResources();
+  }
+
+  /**
+   * Returns the number of memory bytes used by the cache.
+   * @return Number of cache bytes in memory, for CL2O only; zero otherwise.
+   */
+  public int getCacheMemoryUsage() {
+    ensureOpen();
+    if (this.cache == null || !(this.cache instanceof Cl2oTaxonomyWriterCache)) {
+      return 0;
+    }
+    return ((Cl2oTaxonomyWriterCache)this.cache).getMemoryUsage();
+  }
+
+  /**
+   * A hook for extending classes to close additional resources that were used.
+   * The default implementation closes the {@link IndexReader} as well as the
+   * {@link TaxonomyWriterCache} instances that were used. <br>
+   * <b>NOTE:</b> if you override this method, you should include a
+   * <code>super.closeResources()</code> call in your implementation.
+   */
+  protected synchronized void closeResources() throws IOException {
+    if (reader != null) {
+      reader.close();
+      reader = null;
+    }
+    if (cache != null) {
+      cache.close();
+      cache = null;
+    }
+  }
+
+  /**
+   * Look up the given category in the cache and/or the on-disk storage,
+   * returning the category's ordinal, or a negative number in case the
+   * category does not yet exist in the taxonomy.
+   */
+  protected int findCategory(CategoryPath categoryPath) throws IOException {
+    // If we can find the category in our cache, we can return the
+    // response directly from it:
+    int res = cache.get(categoryPath);
+    if (res >= 0) {
+      return res;
+    }
+    // If we know that the cache is complete, i.e., contains every category
+    // which exists, we can return -1 immediately. However, if the cache is
+    // not complete, we need to check the disk.
+    if (cacheIsComplete) {
+      return -1;
+    }
+    cacheMisses++;
+    // After a few cache misses, it makes sense to read all the categories
+    // from disk and into the cache. The reason not to do this on the first
+    // cache miss (or even when opening the writer) is that it will
+    // significantly slow down the case when a taxonomy is opened just to
+    // add one category. The idea only spending a long time on reading
+    // after enough time was spent on cache misses is known as a "online
+    // algorithm".
+    if (perhapsFillCache()) {
+      return cache.get(categoryPath);
+    }
+
+    // We need to get an answer from the on-disk index. If a reader
+    // is not yet open, do it now:
+    if (reader == null) {
+      reader = openReader();
+    }
+
+    TermDocs docs = reader.termDocs(new Term(Consts.FULL, categoryPath
+        .toString(delimiter)));
+    if (!docs.next()) {
+      return -1; // category does not exist in taxonomy
+    }
+    // Note: we do NOT add to the cache the fact that the category
+    // does not exist. The reason is that our only use for this
+    // method is just before we actually add this category. If
+    // in the future this usage changes, we should consider caching
+    // the fact that the category is not in the taxonomy.
+    addToCache(categoryPath, docs.doc());
+    return docs.doc();
+  }
+
+  /**
+   * Look up the given prefix of the given category in the cache and/or the
+   * on-disk storage, returning that prefix's ordinal, or a negative number in
+   * case the category does not yet exist in the taxonomy.
+   */
+  private int findCategory(CategoryPath categoryPath, int prefixLen)
+  throws IOException {
+    int res = cache.get(categoryPath, prefixLen);
+    if (res >= 0) {
+      return res;
+    }
+    if (cacheIsComplete) {
+      return -1;
+    }
+    cacheMisses++;
+    if (perhapsFillCache()) {
+      return cache.get(categoryPath, prefixLen);
+    }
+    if (reader == null) {
+      reader = openReader();
+    }
+    TermDocs docs = reader.termDocs(new Term(Consts.FULL, categoryPath
+        .toString(delimiter, prefixLen)));
+    if (!docs.next()) {
+      return -1; // category does not exist in taxonomy
+    }
+    addToCache(categoryPath, prefixLen, docs.doc());
+    return docs.doc();
+  }
+
+  // TODO (Facet): addCategory() is synchronized. This means that if indexing is
+  // multi-threaded, a new category that needs to be written to disk (and
+  // potentially even trigger a lengthy merge) locks out other addCategory()
+  // calls - even those which could immediately return a cached value.
+  // We definitely need to fix this situation!
+  public synchronized int addCategory(CategoryPath categoryPath) throws IOException {
+    ensureOpen();
+    // If the category is already in the cache and/or the taxonomy, we
+    // should return its existing ordinal:
+    int res = findCategory(categoryPath);
+    if (res < 0) {
+      // This is a new category, and we need to insert it into the index
+      // (and the cache). Actually, we might also need to add some of
+      // the category's ancestors before we can add the category itself
+      // (while keeping the invariant that a parent is always added to
+      // the taxonomy before its child). internalAddCategory() does all
+      // this recursively:
+      res = internalAddCategory(categoryPath, categoryPath.length());
+    }
+    return res;
+
+  }
+
+  /**
+   * Add a new category into the index (and the cache), and return its new
+   * ordinal.
+   * <P>
+   * Actually, we might also need to add some of the category's ancestors
+   * before we can add the category itself (while keeping the invariant that a
+   * parent is always added to the taxonomy before its child). We do this by
+   * recursion.
+   */
+  private int internalAddCategory(CategoryPath categoryPath, int length)
+  throws CorruptIndexException, IOException {
+
+    // Find our parent's ordinal (recursively adding the parent category
+    // to the taxonomy if it's not already there). Then add the parent
+    // ordinal as payloads (rather than a stored field; payloads can be
+    // more efficiently read into memory in bulk by LuceneTaxonomyReader)
+    int parent;
+    if (length > 1) {
+      parent = findCategory(categoryPath, length - 1);
+      if (parent < 0) {
+        parent = internalAddCategory(categoryPath, length - 1);
+      }
+    } else if (length == 1) {
+      parent = TaxonomyReader.ROOT_ORDINAL;
+    } else {
+      parent = TaxonomyReader.INVALID_ORDINAL;
+    }
+    int id = addCategoryDocument(categoryPath, length, parent);
+
+    return id;
+  }
+
+  /**
+   * Verifies that this instance wasn't closed, or throws
+   * {@link AlreadyClosedException} if it is.
+   */
+  protected final void ensureOpen() {
+    if (indexWriter == null) {
+      throw new AlreadyClosedException("The taxonomy writer has already been closed");
+    }
+  }
+  
+  // Note that the methods calling addCategoryDocument() are synchornized,
+  // so this method is effectively synchronized as well, but we'll add
+  // synchronized to be on the safe side, and we can reuse class-local objects
+  // instead of allocating them every time
+  protected synchronized int addCategoryDocument(CategoryPath categoryPath,
+                                                  int length, int parent)
+      throws CorruptIndexException, IOException {
+    // Before Lucene 2.9, position increments >=0 were supported, so we
+    // added 1 to parent to allow the parent -1 (the parent of the root).
+    // Unfortunately, starting with Lucene 2.9, after LUCENE-1542, this is
+    // no longer enough, since 0 is not encoded consistently either (see
+    // comment in SinglePositionTokenStream). But because we must be
+    // backward-compatible with existing indexes, we can't just fix what
+    // we write here (e.g., to write parent+2), and need to do a workaround
+    // in the reader (which knows that anyway only category 0 has a parent
+    // -1).    
+    parentStream.set(parent+1);
+    Document d = new Document();
+    d.add(parentStreamField);
+
+    fullPathField.setValue(categoryPath.toString(delimiter, length));
+    d.add(fullPathField);
+
+    // Note that we do no pass an Analyzer here because the fields that are
+    // added to the Document are untokenized or contains their own TokenStream.
+    // Therefore the IndexWriter's Analyzer has no effect.
+    indexWriter.addDocument(d);
+    int id = nextID++;
+
+    addToCache(categoryPath, length, id);
+
+    // also add to the parent array
+    getParentArray().add(id, parent);
+
+    return id;
+  }
+
+  private static class SinglePositionTokenStream extends TokenStream {
+    private CharTermAttribute termAtt;
+    private PositionIncrementAttribute posIncrAtt;
+    private boolean returned;
+    public SinglePositionTokenStream(String word) {
+      termAtt = addAttribute(CharTermAttribute.class);
+      posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+      termAtt.setEmpty().append(word);
+      returned = true;
+    }
+    /**
+     * Set the value we want to keep, as the position increment.
+     * Note that when TermPositions.nextPosition() is later used to
+     * retrieve this value, val-1 will be returned, not val.
+     * <P>
+     * IMPORTANT NOTE: Before Lucene 2.9, val>=0 were safe (for val==0,
+     * the retrieved position would be -1). But starting with Lucene 2.9,
+     * this unfortunately changed, and only val>0 are safe. val=0 can
+     * still be used, but don't count on the value you retrieve later
+     * (it could be 0 or -1, depending on circumstances or versions).
+     * This change is described in Lucene's JIRA: LUCENE-1542. 
+     */
+    public void set(int val) {
+      posIncrAtt.setPositionIncrement(val);
+      returned = false;
+    }
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (returned) {
+        return false;
+      }
+      returned = true;
+      return true;
+    }
+  }
+
+  private void addToCache(CategoryPath categoryPath, int id)
+  throws CorruptIndexException, IOException {
+    if (cache.put(categoryPath, id)) {
+      // If cache.put() returned true, it means the cache was limited in
+      // size, became full, so parts of it had to be cleared.
+      // Unfortunately we don't know which part was cleared - it is
+      // possible that a relatively-new category that hasn't yet been
+      // committed to disk (and therefore isn't yet visible in our
+      // "reader") was deleted from the cache, and therefore we must
+      // now refresh the reader.
+      // Because this is a slow operation, cache implementations are
+      // expected not to delete entries one-by-one but rather in bulk
+      // (LruTaxonomyWriterCache removes the 2/3rd oldest entries).
+      refreshReader();
+      cacheIsComplete = false;
+    }
+  }
+
+  private void addToCache(CategoryPath categoryPath, int prefixLen, int id)
+  throws CorruptIndexException, IOException {
+    if (cache.put(categoryPath, prefixLen, id)) {
+      refreshReader();
+      cacheIsComplete = false;
+    }
+  }
+
+  private synchronized void refreshReader() throws IOException {
+    if (reader != null) {
+      IndexReader r2 = IndexReader.openIfChanged(reader);
+      if (r2 != null) {
+        reader.close();
+        reader = r2;
+      }
+    }
+  }
+  
+  /**
+   * Calling commit() ensures that all the categories written so far are
+   * visible to a reader that is opened (or reopened) after that call.
+   * When the index is closed(), commit() is also implicitly done.
+   * See {@link TaxonomyWriter#commit()}
+   */ 
+  public synchronized void commit() throws CorruptIndexException, IOException {
+    ensureOpen();
+    if (taxoIndexCreateTime != null) {
+      indexWriter.commit(combinedCommitData(null));
+      taxoIndexCreateTime = null;
+    } else {
+      indexWriter.commit();
+    }
+    refreshReader();
+  }
+
+  /**
+   * Combine original user data with that of the taxonomy creation time
+   */
+  private Map<String,String> combinedCommitData(Map<String,String> userData) {
+    Map<String,String> m = new HashMap<String, String>();
+    if (userData != null) {
+      m.putAll(userData);
+    }
+    m.put(INDEX_CREATE_TIME, taxoIndexCreateTime);
+    return m;
+  }
+  
+  /**
+   * Like commit(), but also store properties with the index. These properties
+   * are retrievable by {@link DirectoryTaxonomyReader#getCommitUserData}.
+   * See {@link TaxonomyWriter#commit(Map)}. 
+   */
+  public synchronized void commit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
+    ensureOpen();
+    if (taxoIndexCreateTime != null) {
+      indexWriter.commit(combinedCommitData(commitUserData));
+      taxoIndexCreateTime = null;
+    } else {
+      indexWriter.commit(commitUserData);
+    }
+    refreshReader();
+  }
+  
+  /**
+   * prepare most of the work needed for a two-phase commit.
+   * See {@link IndexWriter#prepareCommit}.
+   */
+  public synchronized void prepareCommit() throws CorruptIndexException, IOException {
+    ensureOpen();
+    if (taxoIndexCreateTime != null) {
+      indexWriter.prepareCommit(combinedCommitData(null));
+      taxoIndexCreateTime = null;
+    } else {
+      indexWriter.prepareCommit();
+    }
+  }
+
+  /**
+   * Like above, and also prepares to store user data with the index.
+   * See {@link IndexWriter#prepareCommit(Map)}
+   */
+  public synchronized void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
+    ensureOpen();
+    if (taxoIndexCreateTime != null) {
+      indexWriter.prepareCommit(combinedCommitData(commitUserData));
+      taxoIndexCreateTime = null;
+    } else {
+      indexWriter.prepareCommit(commitUserData);
+    }
+  }
+  
+  /**
+   * 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).
+   */
+  synchronized public int getSize() {
+    ensureOpen();
+    return indexWriter.maxDoc();
+  }
+
+  private boolean alreadyCalledFillCache = false;
+
+  /**
+   * Set the number of cache misses before an attempt is made to read the
+   * entire taxonomy into the in-memory cache.
+   * <P> 
+   * LuceneTaxonomyWriter holds an in-memory cache of recently seen
+   * categories to speed up operation. On each cache-miss, the on-disk index
+   * needs to be consulted. When an existing taxonomy is opened, a lot of
+   * slow disk reads like that are needed until the cache is filled, so it
+   * is more efficient to read the entire taxonomy into memory at once.
+   * We do this complete read after a certain number (defined by this method)
+   * of cache misses.
+   * <P>
+   * If the number is set to <CODE>0</CODE>, the entire taxonomy is read
+   * into the cache on first use, without fetching individual categories
+   * first.
+   * <P>
+   * Note that if the memory cache of choice is limited in size, and cannot
+   * hold the entire content of the on-disk taxonomy, then it is never
+   * read in its entirety into the cache, regardless of the setting of this
+   * method. 
+   */
+  public void setCacheMissesUntilFill(int i) {
+    ensureOpen();
+    cacheMissesUntilFill = i;
+  }
+  
+  private int cacheMissesUntilFill = 11;
+
+  private boolean perhapsFillCache() throws IOException {
+    // Note: we assume that we're only called when cacheIsComplete==false.
+    // TODO (Facet): parametrize this criterion:
+    if (cacheMisses < cacheMissesUntilFill) {
+      return false;
+    }
+    // If the cache was already filled (or we decided not to fill it because
+    // there was no room), there is no sense in trying it again.
+    if (alreadyCalledFillCache) {
+      return false;
+    }
+    alreadyCalledFillCache = true;
+    // TODO (Facet): we should probably completely clear the cache before starting
+    // to read it?
+    if (reader == null) {
+      reader = openReader();
+    }
+
+    if (!cache.hasRoom(reader.numDocs())) {
+      return false;
+    }
+
+    CategoryPath cp = new CategoryPath();
+    TermDocs td = reader.termDocs();
+    Term fullPathTerm = new Term(Consts.FULL);
+    String field = fullPathTerm.field(); // needed so we can later use !=
+    TermEnum terms = reader.terms(fullPathTerm);
+    // The check is done here to avoid checking it on every iteration of the
+    // below loop. A null term wlil be returned if there are no terms in the
+    // lexicon, or after the Consts.FULL term. However while the loop is
+    // executed we're safe, because we only iterate as long as there are next()
+    // terms.
+    if (terms.term() != null) {
+      do {
+        Term t = terms.term();
+        if (t.field() != field) break;
+        // Since we guarantee uniqueness of categories, each term has exactly
+        // one document. Also, since we do not allow removing categories (and
+        // hence documents), there are no deletions in the index. Therefore, it
+        // is sufficient to call next(), and then doc(), exactly once with no
+        // 'validation' checks.
+        td.seek(t);
+        td.next();
+        cp.clear();
+        cp.add(t.text(), delimiter);
+        cache.put(cp, td.doc());
+      } while (terms.next());
+    }
+
+    cacheIsComplete = true;
+    // No sense to keep the reader open - we will not need to read from it
+    // if everything is in the cache.
+    reader.close();
+    reader = null;
+    return true;
+  }
+
+  private ParentArray parentArray;
+  private synchronized ParentArray getParentArray() throws IOException {
+    if (parentArray==null) {
+      if (reader == null) {
+        reader = openReader();
+      }
+      parentArray = new ParentArray();
+      parentArray.refresh(reader);
+    }
+    return parentArray;
+  }
+  public int getParent(int ordinal) throws IOException {
+    ensureOpen();
+    // Note: the following if() just enforces that a user can never ask
+    // for the parent of a nonexistant category - even if the parent array
+    // was allocated bigger than it really needs to be.
+    if (ordinal >= getSize()) {
+      throw new ArrayIndexOutOfBoundsException();
+    }
+    return getParentArray().getArray()[ordinal];
+  }
+
+  /**
+   * Take all the categories of one or more given taxonomies, and add them to
+   * the main taxonomy (this), if they are not already there.
+   * <P>
+   * Additionally, fill a <I>mapping</I> for each of the added taxonomies,
+   * mapping its ordinals to the ordinals in the enlarged main taxonomy.
+   * These mapping are saved into an array of OrdinalMap objects given by the
+   * user, one for each of the given taxonomies (not including "this", the main
+   * taxonomy). Often the first of these will be a MemoryOrdinalMap and the
+   * others will be a DiskOrdinalMap - see discussion in {OrdinalMap}. 
+   * <P> 
+   * Note that the taxonomies to be added are given as Directory objects,
+   * not opened TaxonomyReader/TaxonomyWriter objects, so if any of them are
+   * currently managed by an open TaxonomyWriter, make sure to commit() (or
+   * close()) it first. The main taxonomy (this) is an open TaxonomyWriter,
+   * and does not need to be commit()ed before this call. 
+   */
+  public void addTaxonomies(Directory[] taxonomies, OrdinalMap[] ordinalMaps) throws IOException {
+    ensureOpen();
+    // To prevent us stepping on the rest of this class's decisions on when
+    // to open a reader, and when not, we'll be opening a new reader instead
+    // of using the existing "reader" object:
+    IndexReader mainreader = openReader();
+    TermEnum mainte = mainreader.terms(new Term(Consts.FULL));
+
+    IndexReader[] otherreaders = new IndexReader[taxonomies.length];
+    TermEnum[] othertes = new TermEnum[taxonomies.length];
+    for (int i=0; i<taxonomies.length; i++) {
+      otherreaders[i] = IndexReader.open(taxonomies[i]);
+      othertes[i] = otherreaders[i].terms(new Term(Consts.FULL));
+      // Also tell the ordinal maps their expected sizes:
+      ordinalMaps[i].setSize(otherreaders[i].numDocs());
+    }
+
+    CategoryPath cp = new CategoryPath();
+
+    // We keep a "current" cursor over the alphabetically-ordered list of
+    // categories in each taxonomy. We start the cursor on the first
+    // (alphabetically) category of each taxonomy:
+
+    String currentMain;
+    String[] currentOthers = new String[taxonomies.length];
+    currentMain = nextTE(mainte);
+    int otherTaxonomiesLeft = 0;
+    for (int i=0; i<taxonomies.length; i++) {
+      currentOthers[i] = nextTE(othertes[i]);
+      if (currentOthers[i]!=null) {
+        otherTaxonomiesLeft++;
+      }
+    }
+
+    // And then, at each step look at the first (alphabetically) of the
+    // current taxonomies.
+    // NOTE: The most efficient way we could have done this is using a
+    // PriorityQueue. But for simplicity, and assuming that usually we'll
+    // have a very small number of other taxonomies (often just 1), we use
+    // a more naive algorithm (o(ntaxonomies) instead of o(ln ntaxonomies)
+    // per step)
+
+    while (otherTaxonomiesLeft>0) {
+      String first=null;
+      for (int i=0; i<taxonomies.length; i++) {
+        if (currentOthers[i]==null) continue;
+        if (first==null || first.compareTo(currentOthers[i])>0) {
+          first = currentOthers[i];
+        }
+      }
+      int comp = 0;
+      if (currentMain==null || (comp = currentMain.compareTo(first))>0) {
+        // If 'first' is before currentMain, or currentMain is null,
+        // then 'first' is a new category and we need to add it to the
+        // main taxonomy. Then for all taxonomies with this 'first'
+        // category, we need to add the new category number to their
+        // map, and move to the next category in all of them.
+        cp.clear();
+        cp.add(first, delimiter);
+        // We can call internalAddCategory() instead of addCategory()
+        // because we know the category hasn't been seen yet.
+        int newordinal = internalAddCategory(cp, cp.length());
+        // TODO (Facet): we already had this term in our hands before, in nextTE...
+        Term t = new Term(Consts.FULL, first);
+        for (int i=0; i<taxonomies.length; i++) {
+          if (first.equals(currentOthers[i])) {
+            // remember the remapping of this ordinal. Note how
+            // this requires reading a posting list from the index -
+            // but since we do this in lexical order of terms, just
+            // like Lucene's merge works, we hope there are few seeks.
+            // TODO (Facet): is there a quicker way? E.g., not specifying the
+            // next term by name every time?
+            TermDocs td = otherreaders[i].termDocs(t);
+            td.next(); // TODO (Facet): check?
+            int origordinal = td.doc();
+            ordinalMaps[i].addMapping(origordinal, newordinal);
+            // and move to the next category in the i'th taxonomy 
+            currentOthers[i] = nextTE(othertes[i]);
+            if (currentOthers[i]==null) {
+              otherTaxonomiesLeft--;
+            }
+          }
+        }
+      } else if (comp==0) {
+        // 'first' and currentMain are the same, so both the main and some
+        // other taxonomies need to be moved, but a category doesn't need
+        // to be added because it already existed in the main taxonomy.
+
+        // TODO (Facet): Again, is there a quicker way?
+        Term t = new Term(Consts.FULL, first);
+        TermDocs td = mainreader.termDocs(t);
+        td.next(); // TODO (Facet): check?
+        int newordinal = td.doc();
+
+        currentMain = nextTE(mainte);
+        for (int i=0; i<taxonomies.length; i++) {
+          if (first.equals(currentOthers[i])) {
+            // TODO (Facet): again, is there a quicker way?
+            td = otherreaders[i].termDocs(t);
+            td.next(); // TODO (Facet): check?
+            int origordinal = td.doc();
+            ordinalMaps[i].addMapping(origordinal, newordinal);
+
+            // and move to the next category 
+            currentOthers[i] = nextTE(othertes[i]);
+            if (currentOthers[i]==null) {
+              otherTaxonomiesLeft--;
+            }
+          }
+        }
+      } else /* comp > 0 */ {
+        // The currentMain doesn't appear in any of the other taxonomies -
+        // we don't need to do anything, just continue to the next one
+        currentMain = nextTE(mainte);
+      }
+    }
+
+    // Close all the readers we've opened, and also tell the ordinal maps
+    // we're done adding to them
+    mainreader.close();
+    for (int i=0; i<taxonomies.length; i++) {
+      otherreaders[i].close();
+      // We never actually added a mapping for the root ordinal - let's do
+      // it now, just so that the map is complete (every ordinal between 0
+      // and size-1 is remapped)
+      ordinalMaps[i].addMapping(0, 0);
+      ordinalMaps[i].addDone();
+    }
+  }
+
+  /**
+   * Mapping from old ordinal to new ordinals, used when merging indexes 
+   * wit separate taxonomies.
+   * <p> 
+   * addToTaxonomies() merges one or more taxonomies into the given taxonomy
+   * (this). An OrdinalMap is filled for each of the added taxonomies,
+   * containing the new ordinal (in the merged taxonomy) of each of the
+   * categories in the old taxonomy.
+   * <P>  
+   * There exist two implementations of OrdinalMap: MemoryOrdinalMap and
+   * DiskOrdinalMap. As their names suggest, the former keeps the map in
+   * memory and the latter in a temporary disk file. Because these maps will
+   * later be needed one by one (to remap the counting lists), not all at the
+   * same time, it is recommended to put the first taxonomy's map in memory,
+   * and all the rest on disk (later to be automatically read into memory one
+   * by one, when needed).
+   */
+  public static interface OrdinalMap {
+    /**
+     * Set the size of the map. This MUST be called before addMapping().
+     * It is assumed (but not verified) that addMapping() will then be
+     * called exactly 'size' times, with different origOrdinals between 0
+     * and size-1.  
+     */
+    public void setSize(int size) throws IOException;
+    public void addMapping(int origOrdinal, int newOrdinal) throws IOException;
+    /**
+     * Call addDone() to say that all addMapping() have been done.
+     * In some implementations this might free some resources. 
+     */
+    public void addDone() throws IOException;
+    /**
+     * Return the map from the taxonomy's original (consecutive) ordinals
+     * to the new taxonomy's ordinals. If the map has to be read from disk
+     * and ordered appropriately, it is done when getMap() is called.
+     * getMap() should only be called once, and only when the map is actually
+     * needed. Calling it will also free all resources that the map might
+     * be holding (such as temporary disk space), other than the returned int[].
+     */
+    public int[] getMap() throws IOException;
+  }
+
+  /**
+   * {@link OrdinalMap} maintained in memory
+   */
+  public static final class MemoryOrdinalMap implements OrdinalMap {
+    int[] map;
+    public void setSize(int taxonomySize) {
+      map = new int[taxonomySize];
+    }
+    public void addMapping(int origOrdinal, int newOrdinal) {
+      map[origOrdinal] = newOrdinal;
+    }
+    public void addDone() { /* nothing to do */ }
+    public int[] getMap() {
+      return map;
+    }
+  }
+
+  /**
+   * {@link OrdinalMap} maintained on file system
+   */
+  public static final class DiskOrdinalMap implements OrdinalMap {
+    File tmpfile;
+    DataOutputStream out;
+
+    public DiskOrdinalMap(File tmpfile) throws FileNotFoundException {
+      this.tmpfile = tmpfile;
+      out = new DataOutputStream(new BufferedOutputStream(
+          new FileOutputStream(tmpfile)));
+    }
+
+    public void addMapping(int origOrdinal, int newOrdinal) throws IOException {
+      out.writeInt(origOrdinal);
+      out.writeInt(newOrdinal);
+    }
+
+    public void setSize(int taxonomySize) throws IOException {
+      out.writeInt(taxonomySize);
+    }
+
+    public void addDone() throws IOException {
+      if (out!=null) {
+        out.close();
+        out = null;
+      }
+    }
+
+    int[] map = null;
+
+    public int[] getMap() throws IOException {
+      if (map!=null) {
+        return map;
+      }
+      addDone(); // in case this wasn't previously called
+      DataInputStream in = new DataInputStream(new BufferedInputStream(
+          new FileInputStream(tmpfile)));
+      map = new int[in.readInt()];
+      // NOTE: The current code assumes here that the map is complete,
+      // i.e., every ordinal gets one and exactly one value. Otherwise,
+      // we may run into an EOF here, or vice versa, not read everything.
+      for (int i=0; i<map.length; i++) {
+        int origordinal = in.readInt();
+        int newordinal = in.readInt();
+        map[origordinal] = newordinal;
+      }
+      in.close();
+      // Delete the temporary file, which is no longer needed.
+      if (!tmpfile.delete()) {
+        tmpfile.deleteOnExit();
+      }
+      return map;
+    }
+  }
+
+  private static final String nextTE(TermEnum te) throws IOException {
+    if (te.next()) {
+      Term t = te.term();
+      // If our enumeration reached a different field, we're done. Note
+      // how we're allowed compare string references, rather than the
+      // actual string's contents.
+      if (t.field()==Consts.FULL) {
+        return t.text();
+      }
+      return null;
+    } 
+    return null;
+  }
+
+  /**
+   * Rollback changes to the taxonomy writer and closes the instance. Following
+   * this method the instance becomes unusable (calling any of its API methods
+   * will yield an {@link AlreadyClosedException}).
+   */
+  public void rollback() throws IOException {
+    ensureOpen();
+    indexWriter.rollback();
+    // since IndexWriter.rollback() closes the IW instance, we should close too.
+    doClose();
+  }
+  
+}