You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2010/07/21 13:21:58 UTC

svn commit: r966183 [2/2] - in /lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index: DocumentsWriter.java IndexWriter.java

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=966183&r1=966182&r2=966183&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java Wed Jul 21 11:21:58 2010
@@ -240,7 +240,7 @@ public class IndexWriter implements Clos
    * printed to infoStream, if set (see {@link
    * #setInfoStream}).
    */
-  public final static int MAX_TERM_LENGTH = DocumentsWriter.MAX_TERM_LENGTH_UTF8;
+  public final static int MAX_TERM_LENGTH = DocumentsWriterRAMAllocator.MAX_TERM_LENGTH_UTF8;
 
   // The normal read buffer size defaults to 1024, but
   // increasing this during merging seems to yield
@@ -271,10 +271,12 @@ public class IndexWriter implements Clos
   volatile SegmentInfos pendingCommit;            // set when a commit is pending (after prepareCommit() & before commit())
   volatile long pendingCommitChangeCount;
 
-  private SegmentInfos segmentInfos = new SegmentInfos();       // the segments
+  // nocommit - private
+  SegmentInfos segmentInfos = new SegmentInfos();       // the segments
 
   private DocumentsWriter docWriter;
-  private IndexFileDeleter deleter;
+  //nocommit - private
+  IndexFileDeleter deleter;
 
   private Set<SegmentInfo> segmentsToOptimize = new HashSet<SegmentInfo>();           // used by optimize to note those needing optimization
 
@@ -289,8 +291,8 @@ public class IndexWriter implements Clos
   // Holds all SegmentInfo instances currently involved in
   // merges
   private HashSet<SegmentInfo> mergingSegments = new HashSet<SegmentInfo>();
-
-  private MergePolicy mergePolicy;
+  // nocommit - private
+  MergePolicy mergePolicy;
   // TODO 4.0: this should be made final once the setter is removed
   private /*final*/MergeScheduler mergeScheduler;
   private LinkedList<MergePolicy.OneMerge> pendingMerges = new LinkedList<MergePolicy.OneMerge>();
@@ -733,113 +735,6 @@ public class IndexWriter implements Clos
       throw new IllegalArgumentException("this method can only be called when the merge policy is the default LogMergePolicy");
   }
 
-  /** <p>Get the current setting of whether newly flushed
-   *  segments will use the compound file format.  Note that
-   *  this just returns the value previously set with
-   *  setUseCompoundFile(boolean), or the default value
-   *  (true).  You cannot use this to query the status of
-   *  previously flushed segments.</p>
-   *
-   *  <p>Note that this method is a convenience method: it
-   *  just calls mergePolicy.getUseCompoundFile as long as
-   *  mergePolicy is an instance of {@link LogMergePolicy}.
-   *  Otherwise an IllegalArgumentException is thrown.</p>
-   *
-   *  @see #setUseCompoundFile(boolean)
-   *  @deprecated use {@link LogMergePolicy#getUseCompoundDocStore()} and
-   *  {@link LogMergePolicy#getUseCompoundFile()} directly.
-   */
-  public boolean getUseCompoundFile() {
-    return getLogMergePolicy().getUseCompoundFile();
-  }
-
-  /**
-   * <p>
-   * Setting to turn on usage of a compound file. When on, multiple files for
-   * each segment are merged into a single file when a new segment is flushed.
-   * </p>
-   * 
-   * <p>
-   * Note that this method is a convenience method: it just calls
-   * mergePolicy.setUseCompoundFile as long as mergePolicy is an instance of
-   * {@link LogMergePolicy}. Otherwise an IllegalArgumentException is thrown.
-   * </p>
-   * 
-   * @deprecated use {@link LogMergePolicy#setUseCompoundDocStore(boolean)} and
-   *             {@link LogMergePolicy#setUseCompoundFile(boolean)} directly.
-   *             Note that this method set the given value on both, therefore
-   *             you should consider doing the same.
-   */
-  public void setUseCompoundFile(boolean value) {
-    getLogMergePolicy().setUseCompoundFile(value);
-    getLogMergePolicy().setUseCompoundDocStore(value);
-  }
-
-  /** Expert: Set the Similarity implementation used by this IndexWriter.
-   *
-   * @see Similarity#setDefault(Similarity)
-   * @deprecated use {@link IndexWriterConfig#setSimilarity(Similarity)} instead
-   */
-  public void setSimilarity(Similarity similarity) {
-    ensureOpen();
-    this.similarity = similarity;
-    docWriter.setSimilarity(similarity);
-    // Required so config.getSimilarity returns the right value. But this will
-    // go away together with the method in 4.0.
-    config.setSimilarity(similarity);
-  }
-
-  /** Expert: Return the Similarity implementation used by this IndexWriter.
-   *
-   * <p>This defaults to the current value of {@link Similarity#getDefault()}.
-   * @deprecated use {@link IndexWriterConfig#getSimilarity()} instead
-   */
-  public Similarity getSimilarity() {
-    ensureOpen();
-    return similarity;
-  }
-
-  /** Expert: Set the interval between indexed terms.  Large values cause less
-   * memory to be used by IndexReader, but slow random-access to terms.  Small
-   * values cause more memory to be used by an IndexReader, and speed
-   * random-access to terms.
-   *
-   * This parameter determines the amount of computation required per query
-   * term, regardless of the number of documents that contain that term.  In
-   * particular, it is the maximum number of other terms that must be
-   * scanned before a term is located and its frequency and position information
-   * may be processed.  In a large index with user-entered query terms, query
-   * processing time is likely to be dominated not by term lookup but rather
-   * by the processing of frequency and positional data.  In a small index
-   * or when many uncommon query terms are generated (e.g., by wildcard
-   * queries) term lookup may become a dominant cost.
-   *
-   * In particular, <code>numUniqueTerms/interval</code> terms are read into
-   * memory by an IndexReader, and, on average, <code>interval/2</code> terms
-   * must be scanned for each random term access.
-   *
-   * @see #DEFAULT_TERM_INDEX_INTERVAL
-   * @deprecated use {@link IndexWriterConfig#setTermIndexInterval(int)}
-   */
-  public void setTermIndexInterval(int interval) {
-    ensureOpen();
-    this.termIndexInterval = interval;
-    // Required so config.getTermIndexInterval returns the right value. But this
-    // will go away together with the method in 4.0.
-    config.setTermIndexInterval(interval);
-  }
-
-  /** Expert: Return the interval between indexed terms.
-   *
-   * @see #setTermIndexInterval(int)
-   * @deprecated use {@link IndexWriterConfig#getTermIndexInterval()}
-   */
-  public int getTermIndexInterval() {
-    // We pass false because this method is called by SegmentMerger while we are in the process of closing
-    ensureOpen(false);
-    return termIndexInterval;
-  }
-
   /**
    * Constructs an IndexWriter for the index in <code>d</code>.
    * Text will be analyzed with <code>a</code>.  If <code>create</code>
@@ -1028,7 +923,6 @@ public class IndexWriter implements Clos
     directory = d;
     analyzer = conf.getAnalyzer();
     setMessageID(defaultInfoStream);
-    maxFieldLength = conf.getMaxFieldLength();
     termIndexInterval = conf.getTermIndexInterval();
     writeLockTimeout = conf.getWriteLockTimeout();
     similarity = conf.getSimilarity();
@@ -1102,9 +996,10 @@ public class IndexWriter implements Clos
 
       setRollbackSegmentInfos(segmentInfos);
 
-      docWriter = new DocumentsWriter(directory, this, conf.getIndexingChain(), conf.getMaxThreadStates());
-      docWriter.setInfoStream(infoStream);
-      docWriter.setMaxFieldLength(maxFieldLength);
+      docWriter = new DocumentsWriter(directory, this, conf);
+      // nocommit
+      //docWriter.setInfoStream(infoStream);
+      //docWriter.setMaxFieldLength(maxFieldLength);
 
       // Default deleter (for backwards compatibility) is
       // KeepOnlyLastCommitDeleter:
@@ -1167,149 +1062,6 @@ public class IndexWriter implements Clos
     return config;
   }
   
-  /**
-   * Expert: set the merge policy used by this writer.
-   * 
-   * @deprecated use {@link IndexWriterConfig#setMergePolicy(MergePolicy)} instead.
-   */
-  public void setMergePolicy(MergePolicy mp) {
-    ensureOpen();
-    if (mp == null)
-      throw new NullPointerException("MergePolicy must be non-null");
-
-    if (mergePolicy != mp)
-      mergePolicy.close();
-    mergePolicy = mp;
-    mergePolicy.setIndexWriter(this);
-    pushMaxBufferedDocs();
-    if (infoStream != null)
-      message("setMergePolicy " + mp);
-    // Required so config.getMergePolicy returns the right value. But this will
-    // go away together with the method in 4.0.
-    config.setMergePolicy(mp);
-  }
-
-  /**
-   * Expert: returns the current MergePolicy in use by this writer.
-   * @see #setMergePolicy
-   * 
-   * @deprecated use {@link IndexWriterConfig#getMergePolicy()} instead
-   */
-  public MergePolicy getMergePolicy() {
-    ensureOpen();
-    return mergePolicy;
-  }
-
-  /**
-   * Expert: set the merge scheduler used by this writer.
-   * @deprecated use {@link IndexWriterConfig#setMergeScheduler(MergeScheduler)} instead
-   */
-  synchronized public void setMergeScheduler(MergeScheduler mergeScheduler) throws CorruptIndexException, IOException {
-    ensureOpen();
-    if (mergeScheduler == null)
-      throw new NullPointerException("MergeScheduler must be non-null");
-
-    if (this.mergeScheduler != mergeScheduler) {
-      finishMerges(true);
-      this.mergeScheduler.close();
-    }
-    this.mergeScheduler = mergeScheduler;
-    if (infoStream != null)
-      message("setMergeScheduler " + mergeScheduler);
-    // Required so config.getMergeScheduler returns the right value. But this will
-    // go away together with the method in 4.0.
-    config.setMergeScheduler(mergeScheduler);
-  }
-
-  /**
-   * Expert: returns the current MergeScheduler in use by this
-   * writer.
-   * @see #setMergeScheduler(MergeScheduler)
-   * @deprecated use {@link IndexWriterConfig#getMergeScheduler()} instead
-   */
-  public MergeScheduler getMergeScheduler() {
-    ensureOpen();
-    return mergeScheduler;
-  }
-
-  /** <p>Determines the largest segment (measured by
-   * document count) that may be merged with other segments.
-   * Small values (e.g., less than 10,000) are best for
-   * interactive indexing, as this limits the length of
-   * pauses while indexing to a few seconds.  Larger values
-   * are best for batched indexing and speedier
-   * searches.</p>
-   *
-   * <p>The default value is {@link Integer#MAX_VALUE}.</p>
-   *
-   * <p>Note that this method is a convenience method: it
-   * just calls mergePolicy.setMaxMergeDocs as long as
-   * mergePolicy is an instance of {@link LogMergePolicy}.
-   * Otherwise an IllegalArgumentException is thrown.</p>
-   *
-   * <p>The default merge policy ({@link
-   * LogByteSizeMergePolicy}) also allows you to set this
-   * limit by net size (in MB) of the segment, using {@link
-   * LogByteSizeMergePolicy#setMaxMergeMB}.</p>
-   * @deprecated use {@link LogMergePolicy#setMaxMergeDocs(int)} directly.
-   */
-  public void setMaxMergeDocs(int maxMergeDocs) {
-    getLogMergePolicy().setMaxMergeDocs(maxMergeDocs);
-  }
-
-  /**
-   * <p>Returns the largest segment (measured by document
-   * count) that may be merged with other segments.</p>
-   *
-   * <p>Note that this method is a convenience method: it
-   * just calls mergePolicy.getMaxMergeDocs as long as
-   * mergePolicy is an instance of {@link LogMergePolicy}.
-   * Otherwise an IllegalArgumentException is thrown.</p>
-   *
-   * @see #setMaxMergeDocs
-   * @deprecated use {@link LogMergePolicy#getMaxMergeDocs()} directly.
-   */
-  public int getMaxMergeDocs() {
-    return getLogMergePolicy().getMaxMergeDocs();
-  }
-
-  /**
-   * The maximum number of terms that will be indexed for a single field in a
-   * document.  This limits the amount of memory required for indexing, so that
-   * collections with very large files will not crash the indexing process by
-   * running out of memory.  This setting refers to the number of running terms,
-   * not to the number of different terms.<p/>
-   * <strong>Note:</strong> this silently truncates large documents, excluding from the
-   * index all terms that occur further in the document.  If you know your source
-   * documents are large, be sure to set this value high enough to accomodate
-   * the expected size.  If you set it to Integer.MAX_VALUE, then the only limit
-   * is your memory, but you should anticipate an OutOfMemoryError.<p/>
-   * By default, no more than {@link #DEFAULT_MAX_FIELD_LENGTH} terms
-   * will be indexed for a field.
-   * @deprecated use {@link IndexWriterConfig#setMaxFieldLength(int)} instead
-   */
-  public void setMaxFieldLength(int maxFieldLength) {
-    ensureOpen();
-    this.maxFieldLength = maxFieldLength;
-    docWriter.setMaxFieldLength(maxFieldLength);
-    if (infoStream != null)
-      message("setMaxFieldLength " + maxFieldLength);
-    // Required so config.getMaxFieldLength returns the right value. But this
-    // will go away together with the method in 4.0.
-    config.setMaxFieldLength(maxFieldLength);
-  }
-
-  /**
-   * Returns the maximum number of terms that will be
-   * indexed for a single field in a document.
-   * @see #setMaxFieldLength
-   * @deprecated use {@link IndexWriterConfig#getMaxFieldLength()} instead
-   */
-  public int getMaxFieldLength() {
-    ensureOpen();
-    return maxFieldLength;
-  }
-
   /** Determines the minimal number of documents required
    * before the buffered in-memory documents are flushed as
    * a new Segment.  Large values generally gives faster
@@ -1543,7 +1295,8 @@ public class IndexWriter implements Clos
   public void setInfoStream(PrintStream infoStream) {
     ensureOpen();
     setMessageID(infoStream);
-    docWriter.setInfoStream(infoStream);
+    // nocommit
+    //docWriter.setInfoStream(infoStream);
     deleter.setInfoStream(infoStream);
     if (infoStream != null)
       messageState();
@@ -1572,48 +1325,6 @@ public class IndexWriter implements Clos
   }
   
   /**
-   * Sets the maximum time to wait for a write lock (in milliseconds) for this instance of IndexWriter.  @see
-   * @see #setDefaultWriteLockTimeout to change the default value for all instances of IndexWriter.
-   * @deprecated use {@link IndexWriterConfig#setWriteLockTimeout(long)} instead
-   */
-  public void setWriteLockTimeout(long writeLockTimeout) {
-    ensureOpen();
-    this.writeLockTimeout = writeLockTimeout;
-    // Required so config.getWriteLockTimeout returns the right value. But this
-    // will go away together with the method in 4.0.
-    config.setWriteLockTimeout(writeLockTimeout);
-  }
-
-  /**
-   * Returns allowed timeout when acquiring the write lock.
-   * @see #setWriteLockTimeout
-   * @deprecated use {@link IndexWriterConfig#getWriteLockTimeout()}
-   */
-  public long getWriteLockTimeout() {
-    ensureOpen();
-    return writeLockTimeout;
-  }
-
-  /**
-   * Sets the default (for any instance of IndexWriter) maximum time to wait for a write lock (in
-   * milliseconds).
-   * @deprecated use {@link IndexWriterConfig#setDefaultWriteLockTimeout(long)} instead
-   */
-  public static void setDefaultWriteLockTimeout(long writeLockTimeout) {
-    IndexWriterConfig.setDefaultWriteLockTimeout(writeLockTimeout);
-  }
-
-  /**
-   * Returns default write lock timeout for newly
-   * instantiated IndexWriters.
-   * @see #setDefaultWriteLockTimeout
-   * @deprecated use {@link IndexWriterConfig#getDefaultWriteLockTimeout()} instead
-   */
-  public static long getDefaultWriteLockTimeout() {
-    return IndexWriterConfig.getDefaultWriteLockTimeout();
-  }
-
-  /**
    * Commits all changes to an index and closes all
    * associated files.  Note that this may be a costly
    * operation, so, try to re-use a single writer instead of
@@ -1774,8 +1485,9 @@ public class IndexWriter implements Clos
         closing = false;
         notifyAll();
         if (!closed) {
-          if (docWriter != null)
+          if (docWriter != null) {
             docWriter.resumeAllThreads();
+          }
           if (infoStream != null)
             message("hit exception while closing");
         }
@@ -1783,76 +1495,6 @@ public class IndexWriter implements Clos
     }
   }
 
-  /** Tells the docWriter to close its currently open shared
-   *  doc stores (stored fields & vectors files).
-   *  Return value specifices whether new doc store files are compound or not.
-   */
-  private synchronized boolean flushDocStores() throws IOException {
-
-    boolean useCompoundDocStore = false;
-
-    String docStoreSegment;
-
-    boolean success = false;
-    try {
-      docStoreSegment = docWriter.closeDocStore();
-      success = true;
-    } finally {
-      if (!success && infoStream != null) {
-        message("hit exception closing doc store segment");
-      }
-    }
-
-    useCompoundDocStore = mergePolicy.useCompoundDocStore(segmentInfos);
-      
-    if (useCompoundDocStore && docStoreSegment != null && docWriter.closedFiles().size() != 0) {
-      // Now build compound doc store file
-
-      if (infoStream != null) {
-        message("create compound file " + IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION));
-      }
-
-      success = false;
-
-      final int numSegments = segmentInfos.size();
-      final String compoundFileName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION);
-
-      try {
-        CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
-        for (final String file :  docWriter.closedFiles() ) {
-          cfsWriter.addFile(file);
-        }
-      
-        // Perform the merge
-        cfsWriter.close();
-        success = true;
-
-      } finally {
-        if (!success) {
-          if (infoStream != null)
-            message("hit exception building compound file doc store for segment " + docStoreSegment);
-          deleter.deleteFile(compoundFileName);
-          docWriter.abort();
-        }
-      }
-
-      for(int i=0;i<numSegments;i++) {
-        SegmentInfo si = segmentInfos.info(i);
-        if (si.getDocStoreOffset() != -1 &&
-            si.getDocStoreSegment().equals(docStoreSegment))
-          si.setDocStoreIsCompoundFile(true);
-      }
-
-      checkpoint();
-
-      // In case the files we just merged into a CFS were
-      // not previously checkpointed:
-      deleter.deleteNewFiles(docWriter.closedFiles());
-    }
-
-    return useCompoundDocStore;
-  }
-
   /** Returns the Directory used by this index. */
   public Directory getDirectory() {     
     // Pass false because the flush during closing calls getDirectory
@@ -1913,22 +1555,6 @@ public class IndexWriter implements Clos
   }
 
   /**
-   * The maximum number of terms that will be indexed for a single field in a
-   * document.  This limits the amount of memory required for indexing, so that
-   * collections with very large files will not crash the indexing process by
-   * running out of memory.<p/>
-   * Note that this effectively truncates large documents, excluding from the
-   * index terms that occur further in the document.  If you know your source
-   * documents are large, be sure to set this value high enough to accommodate
-   * the expected size.  If you set it to Integer.MAX_VALUE, then the only limit
-   * is your memory, but you should anticipate an OutOfMemoryError.<p/>
-   * By default, no more than 10,000 terms will be indexed for a field.
-   *
-   * @see MaxFieldLength
-   */
-  private int maxFieldLength;
-
-  /**
    * Adds a document to this index.  If the document contains more than
    * {@link #setMaxFieldLength(int)} terms for a given field, the remainder are
    * discarded.
@@ -1972,8 +1598,8 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void addDocument(Document doc) throws CorruptIndexException, IOException {
-    addDocument(doc, analyzer);
+  public long addDocument(Document doc) throws CorruptIndexException, IOException {
+    return addDocument(doc, analyzer);
   }
 
   /**
@@ -1993,36 +1619,36 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void addDocument(Document doc, Analyzer analyzer) throws CorruptIndexException, IOException {
+  public long addDocument(Document doc, Analyzer analyzer) throws CorruptIndexException, IOException {
     ensureOpen();
-    boolean doFlush = false;
     boolean success = false;
     try {
       try {
-        doFlush = docWriter.addDocument(doc, analyzer);
+        long sequenceID = docWriter.addDocument(doc, analyzer);
         success = true;
+        return sequenceID;
       } finally {
         if (!success) {
-
-          if (infoStream != null)
+          if (infoStream != null) {
             message("hit exception adding document");
-
+          }
           synchronized (this) {
             // If docWriter has some aborted files that were
             // never incref'd, then we clean them up here
             if (docWriter != null) {
               final Collection<String> files = docWriter.abortedFiles();
-              if (files != null)
+              if (files != null) {
                 deleter.deleteNewFiles(files);
+              }
             }
           }
         }
       }
-      if (doFlush)
-        flush(true, false, false);
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "addDocument");
     }
+    
+    return -1;
   }
 
   /**
@@ -2036,15 +1662,14 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void deleteDocuments(Term term) throws CorruptIndexException, IOException {
+  public long deleteDocuments(Term term) throws CorruptIndexException, IOException {
     ensureOpen();
     try {
-      boolean doFlush = docWriter.bufferDeleteTerm(term);
-      if (doFlush)
-        flush(true, false, false);
+      return docWriter.bufferDeleteTerm(term);
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Term)");
     }
+    return -1;
   }
 
   /**
@@ -2060,15 +1685,14 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void deleteDocuments(Term... terms) throws CorruptIndexException, IOException {
+  public long deleteDocuments(Term... terms) throws CorruptIndexException, IOException {
     ensureOpen();
     try {
-      boolean doFlush = docWriter.bufferDeleteTerms(terms);
-      if (doFlush)
-        flush(true, false, false);
+      return docWriter.bufferDeleteTerms(terms);
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Term..)");
     }
+    return -1;
   }
 
   /**
@@ -2082,11 +1706,9 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void deleteDocuments(Query query) throws CorruptIndexException, IOException {
+  public long deleteDocuments(Query query) throws CorruptIndexException, IOException {
     ensureOpen();
-    boolean doFlush = docWriter.bufferDeleteQuery(query);
-    if (doFlush)
-      flush(true, false, false);
+    return docWriter.bufferDeleteQuery(query);
   }
 
   /**
@@ -2102,11 +1724,9 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void deleteDocuments(Query... queries) throws CorruptIndexException, IOException {
+  public long deleteDocuments(Query... queries) throws CorruptIndexException, IOException {
     ensureOpen();
-    boolean doFlush = docWriter.bufferDeleteQueries(queries);
-    if (doFlush)
-      flush(true, false, false);
+    return docWriter.bufferDeleteQueries(queries);
   }
 
   /**
@@ -2149,35 +1769,37 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void updateDocument(Term term, Document doc, Analyzer analyzer)
+  public long updateDocument(Term term, Document doc, Analyzer analyzer)
       throws CorruptIndexException, IOException {
     ensureOpen();
     try {
-      boolean doFlush = false;
       boolean success = false;
       try {
-        doFlush = docWriter.updateDocument(term, doc, analyzer);
+        long sequenceID = docWriter.updateDocument(term, doc, analyzer);
         success = true;
+        return sequenceID;
       } finally {
         if (!success) {
 
-          if (infoStream != null)
+          if (infoStream != null) {
             message("hit exception updating document");
+          }
 
           synchronized (this) {
             // If docWriter has some aborted files that were
             // never incref'd, then we clean them up here
             final Collection<String> files = docWriter.abortedFiles();
-            if (files != null)
+            if (files != null) {
               deleter.deleteNewFiles(files);
+            }
           }
         }
       }
-      if (doFlush)
-        flush(true, false, false);
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "updateDocument");
     }
+    
+    return -1;
   }
 
   // for test purpose
@@ -2697,7 +2319,8 @@ public class IndexWriter implements Clos
 
       // Remove any buffered docs
       docWriter.abort();
-      docWriter.setFlushedDocCount(0);
+      // nocommit
+      //docWriter.setFlushedDocCount(0);
 
       // Remove all segments
       segmentInfos.clear();
@@ -2790,7 +2413,8 @@ public class IndexWriter implements Clos
    * the index files referenced exist (correctly) in the
    * index directory.
    */
-  private synchronized void checkpoint() throws IOException {
+  // nocommit - private
+  synchronized void checkpoint() throws IOException {
     changeCount++;
     deleter.checkpoint(segmentInfos, false);
   }
@@ -2841,7 +2465,7 @@ public class IndexWriter implements Clos
    * starting index (see {@link #optimize()} for details).
    *
    * <p>
-   * <b>NOTE:</b> this method only copies the segments of the incomning indexes
+   * <b>NOTE:</b> this method only copies the segments of the incoming indexes
    * and does not merge them. Therefore deleted documents are not removed and
    * the new segments are not merged with the existing ones. Also, the segments 
    * are copied as-is, meaning they are not converted to CFS if they aren't, 
@@ -2925,7 +2549,8 @@ public class IndexWriter implements Clos
         ensureOpen();
         segmentInfos.addAll(infos);
         // Notify DocumentsWriter that the flushed count just increased
-        docWriter.updateFlushedDocCount(docCount);
+        // nocommit
+        //docWriter.updateFlushedDocCount(docCount);
 
         checkpoint();
       }
@@ -2977,11 +2602,12 @@ public class IndexWriter implements Clos
         checkpoint();
         
         // Notify DocumentsWriter that the flushed count just increased
-        docWriter.updateFlushedDocCount(docCount);
+        // nocommit
+        //docWriter.updateFlushedDocCount(docCount);
       }
       
       // Now create the compound file if needed
-      if (mergePolicy instanceof LogMergePolicy && getUseCompoundFile()) {
+      if (mergePolicy instanceof LogMergePolicy && getLogMergePolicy().getUseCompoundFile()) {
 
         List<String> files = null;
 
@@ -3211,183 +2837,17 @@ public class IndexWriter implements Clos
   // synchronized, ie, merges should be allowed to commit
   // even while a flush is happening
   private synchronized final boolean doFlush(boolean flushDocStores, boolean flushDeletes) throws CorruptIndexException, IOException {
-    try {
-      try {
-        return doFlushInternal(flushDocStores, flushDeletes);
-      } finally {
-        docWriter.balanceRAM();
-      }
-    } finally {
-      docWriter.clearFlushPending();
-    }
-  }
-
-  // TODO: this method should not have to be entirely
-  // synchronized, ie, merges should be allowed to commit
-  // even while a flush is happening
-  private synchronized final boolean doFlushInternal(boolean flushDocStores, boolean flushDeletes) throws CorruptIndexException, IOException {
-
-    if (hitOOM) {
-      throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush");
-    }
-
-    ensureOpen(false);
-
-    assert testPoint("startDoFlush");
-
-    doBeforeFlush();
-    
-    flushCount++;
-
-    // If we are flushing because too many deletes
-    // accumulated, then we should apply the deletes to free
-    // RAM:
-    flushDeletes |= docWriter.doApplyDeletes();
-
-    // Make sure no threads are actively adding a document.
-    // Returns true if docWriter is currently aborting, in
-    // which case we skip flushing this segment
-    if (infoStream != null) {
-      message("flush: now pause all indexing threads");
-    }
-    if (docWriter.pauseAllThreads()) {
-      docWriter.resumeAllThreads();
-      return false;
-    }
-
-    try {
-
-      SegmentInfo newSegment = null;
-
-      final int numDocs = docWriter.getNumDocsInRAM();
-
-      // Always flush docs if there are any
-      boolean flushDocs = numDocs > 0;
-
-      String docStoreSegment = docWriter.getDocStoreSegment();
-
-      assert docStoreSegment != null || numDocs == 0: "dss=" + docStoreSegment + " numDocs=" + numDocs;
-
-      if (docStoreSegment == null)
-        flushDocStores = false;
-
-      int docStoreOffset = docWriter.getDocStoreOffset();
-
-      boolean docStoreIsCompoundFile = false;
-
-      if (infoStream != null) {
-        message("  flush: segment=" + docWriter.getSegment() +
-                " docStoreSegment=" + docWriter.getDocStoreSegment() +
-                " docStoreOffset=" + docStoreOffset +
-                " flushDocs=" + flushDocs +
-                " flushDeletes=" + flushDeletes +
-                " flushDocStores=" + flushDocStores +
-                " numDocs=" + numDocs +
-                " numBufDelTerms=" + docWriter.getNumBufferedDeleteTerms());
-        message("  index before flush " + segString());
-      }
-
-      // Check if the doc stores must be separately flushed
-      // because other segments, besides the one we are about
-      // to flush, reference it
-      if (flushDocStores && (!flushDocs || !docWriter.getSegment().equals(docWriter.getDocStoreSegment()))) {
-        // We must separately flush the doc store
-        if (infoStream != null)
-          message("  flush shared docStore segment " + docStoreSegment);
-      
-        docStoreIsCompoundFile = flushDocStores();
-        flushDocStores = false;
-      }
-
-      String segment = docWriter.getSegment();
-
-      // If we are flushing docs, segment must not be null:
-      assert segment != null || !flushDocs;
-
-      if (flushDocs) {
-
-        boolean success = false;
-        final int flushedDocCount;
-
-        try {
-          flushedDocCount = docWriter.flush(flushDocStores);
-          success = true;
-        } finally {
-          if (!success) {
-            if (infoStream != null)
-              message("hit exception flushing segment " + segment);
-            deleter.refresh(segment);
-          }
-        }
-        
-        if (0 == docStoreOffset && flushDocStores) {
-          // This means we are flushing private doc stores
-          // with this segment, so it will not be shared
-          // with other segments
-          assert docStoreSegment != null;
-          assert docStoreSegment.equals(segment);
-          docStoreOffset = -1;
-          docStoreIsCompoundFile = false;
-          docStoreSegment = null;
-        }
-
-        // Create new SegmentInfo, but do not add to our
-        // segmentInfos until deletes are flushed
-        // successfully.
-        newSegment = new SegmentInfo(segment,
-                                     flushedDocCount,
-                                     directory, false, docStoreOffset,
-                                     docStoreSegment, docStoreIsCompoundFile,
-                                     docWriter.hasProx(),    
-                                     docWriter.getCodec());
-
-        setDiagnostics(newSegment, "flush");
-      }
-
-      docWriter.pushDeletes();
-
-      if (flushDocs) {
-        segmentInfos.add(newSegment);
-        checkpoint();
-      }
-
-      if (flushDocs && mergePolicy.useCompoundFile(segmentInfos, newSegment)) {
-        // Now build compound file
-        boolean success = false;
-        try {
-          docWriter.createCompoundFile(segment);
-          success = true;
-        } finally {
-          if (!success) {
-            if (infoStream != null)
-              message("hit exception creating compound file for newly flushed segment " + segment);
-            deleter.deleteFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
-          }
-        }
-
-        newSegment.setUseCompoundFile(true);
-        checkpoint();
-      }
-
-      if (flushDeletes) {
-        applyDeletes();
-      }
-      
-      if (flushDocs)
-        checkpoint();
-
-      doAfterFlush();
-
-      return flushDocs;
-
-    } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "doFlush");
-      // never hit
-      return false;
-    } finally {
-      docWriter.clearFlushPending();
-      docWriter.resumeAllThreads();
-    }
+    return docWriter.flushAllThreads(flushDocStores, flushDeletes);
+    // nocommit
+//    try {
+//      try {
+//        return doFlushInternal(flushDocStores, flushDeletes);
+//      } finally {
+//        docWriter.balanceRAM();
+//      }
+//    } finally {
+//      docWriter.clearFlushPending();
+//    }
   }
 
   /** Expert:  Return the total size of all index files currently cached in memory.
@@ -3535,7 +2995,8 @@ public class IndexWriter implements Clos
     final int start = ensureContiguousMerge(merge);
 
     commitMergedDeletes(merge, mergedReader);
-    docWriter.remapDeletes(segmentInfos, merger.getDocMaps(), merger.getDelCounts(), merge, mergedDocCount);
+    // nocommit
+    //docWriter.remapDeletes(segmentInfos, merger.getDocMaps(), merger.getDelCounts(), merge, mergedDocCount);
       
     setMergeDocStoreIsCompoundFile(merge);
     merge.info.setHasProx(merger.hasProx());
@@ -3749,7 +3210,8 @@ public class IndexWriter implements Clos
 
     boolean mergeDocStores = false;
     boolean doFlushDocStore = false;
-    final String currentDocStoreSegment = docWriter.getDocStoreSegment();
+    // nocommit
+    //final String currentDocStoreSegment = docWriter.getDocStoreSegment();
 
     // Test each segment to be merged: check if we need to
     // flush/merge doc stores
@@ -3793,9 +3255,10 @@ public class IndexWriter implements Clos
 
       // If the segment is referencing the current "live"
       // doc store outputs then we must merge
-      if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment)) {
-        doFlushDocStore = true;
-      }
+      // nocommit
+//      if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment)) {
+//        doFlushDocStore = true;
+//      }
     }
 
     final int docStoreOffset;
@@ -3854,12 +3317,13 @@ public class IndexWriter implements Clos
     // CFS:
     mergingSegments.add(merge.info);
   }
-
-  private void setDiagnostics(SegmentInfo info, String source) {
+  
+  // nocommit - private
+  static void setDiagnostics(SegmentInfo info, String source) {
     setDiagnostics(info, source, null);
   }
-
-  private void setDiagnostics(SegmentInfo info, String source, Map<String,String> details) {
+  
+  private static void setDiagnostics(SegmentInfo info, String source, Map<String,String> details) {
     Map<String,String> diagnostics = new HashMap<String,String>();
     diagnostics.put("source", source);
     diagnostics.put("lucene.version", Constants.LUCENE_VERSION);
@@ -3964,7 +3428,7 @@ public class IndexWriter implements Clos
         // commit merged deletes
         SegmentReader reader = merge.readers[i] = readerPool.get(info, merge.mergeDocStores,
                                                                  MERGE_READ_BUFFER_SIZE,
-                                                                 -1);
+                                                                 -config.getReaderTermsIndexDivisor());
 
         // We clone the segment readers because other
         // deletes may come in while we're merging so we
@@ -4003,11 +3467,12 @@ public class IndexWriter implements Clos
           // readers will attempt to open an IndexInput
           // on files that have still-open IndexOutputs
           // against them:
-          if (dss.contains(docWriter.getDocStoreSegment())) {
-            if (infoStream != null)
-              message("now flush at mergeMiddle");
-            doFlush(true, false);
-          }
+          // nocommit
+//          if (dss.contains(docWriter.getDocStoreSegment())) {
+//            if (infoStream != null)
+//              message("now flush at mergeMiddle");
+//            doFlush(true, false);
+//          }
         }
 
         for(int i=0;i<numSegments;i++) {
@@ -4227,12 +3692,16 @@ public class IndexWriter implements Clos
 
   // For test purposes.
   final synchronized int getBufferedDeleteTermsSize() {
-    return docWriter.getBufferedDeleteTerms().size();
+    // nocommit
+    return 0;
+    //return docWriter.getBufferedDeleteTerms().size();
   }
 
   // For test purposes.
   final synchronized int getNumBufferedDeleteTerms() {
-    return docWriter.getNumBufferedDeleteTerms();
+    // nocommit
+    return 0;
+    //return docWriter.getNumBufferedDeleteTerms();
   }
 
   // utility routines for tests