You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2011/12/15 06:11:30 UTC

svn commit: r1214623 [3/17] - in /lucene/dev/branches/solrcloud: ./ dev-tools/idea/lucene/contrib/ lucene/ lucene/contrib/ lucene/contrib/demo/src/java/org/apache/lucene/demo/ lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight...

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Thu Dec 15 05:11:14 2011
@@ -23,123 +23,61 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.LockObtainFailedException;
-import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.MapBackedSet;
+import org.apache.lucene.util.IOUtils;
 
 /** 
  * An IndexReader which reads indexes with multiple segments.
  */
-class DirectoryReader extends IndexReader implements Cloneable {
-  protected Directory directory;
-  protected boolean readOnly;
-
-  IndexWriter writer;
-
-  private IndexDeletionPolicy deletionPolicy;
-  private Lock writeLock;
+final class DirectoryReader extends BaseMultiReader<SegmentReader> {
+  protected final Directory directory;
+  private final IndexWriter writer;
   private final SegmentInfos segmentInfos;
-  private boolean stale;
   private final int termInfosIndexDivisor;
-
-  private boolean rollbackHasChanges;
-
-  private SegmentReader[] subReaders;
-  private ReaderContext topLevelReaderContext;
-  private int[] starts;                           // 1st docno for each segment
-  private int maxDoc = 0;
-  private int numDocs = -1;
-  private boolean hasDeletions = false;
-
-  // Max version in index as of when we opened; this can be
-  // > our current segmentInfos version in case we were
-  // opened on a past IndexCommit:
-  private long maxIndexVersion;
-
   private final boolean applyAllDeletes;
-
-//  static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly,
-//      final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-//    return open(directory, deletionPolicy, commit, readOnly, termInfosIndexDivisor, null);
-//  }
-  
-  static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly,
-                          final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
-      @Override
-      protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
-        SegmentInfos infos = new SegmentInfos();
-        infos.read(directory, segmentFileName);
-        return new DirectoryReader(directory, infos, deletionPolicy, readOnly, termInfosIndexDivisor);
-      }
-    }.run(commit);
-  }
-
-  /** Construct reading the named set of readers. */
-//  DirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor) throws IOException {
-//    this(directory, sis, deletionPolicy, readOnly, termInfosIndexDivisor, null);
-//  }
   
-  /** Construct reading the named set of readers. */
-  DirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor) throws IOException {
+  DirectoryReader(SegmentReader[] readers, Directory directory, IndexWriter writer,
+    SegmentInfos sis, int termInfosIndexDivisor, boolean applyAllDeletes) throws IOException {
+    super(readers);
     this.directory = directory;
-    this.readOnly = readOnly;
+    this.writer = writer;
     this.segmentInfos = sis;
-    this.deletionPolicy = deletionPolicy;
     this.termInfosIndexDivisor = termInfosIndexDivisor;
-    readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
-    applyAllDeletes = false;
-
-    // To reduce the chance of hitting FileNotFound
-    // (and having to retry), we open segments in
-    // reverse because IndexWriter merges & deletes
-    // the newest segments first.
+    this.applyAllDeletes = applyAllDeletes;
+  }
 
-    SegmentReader[] readers = new SegmentReader[sis.size()];
-    for (int i = sis.size()-1; i >= 0; i--) {
-      boolean success = false;
-      try {
-        readers[i] = SegmentReader.get(readOnly, sis.info(i), termInfosIndexDivisor, IOContext.READ);
-        readers[i].readerFinishedListeners = readerFinishedListeners;
-        success = true;
-      } finally {
-        if (!success) {
-          // Close all readers we had opened:
-          for(i++;i<sis.size();i++) {
-            try {
-              readers[i].close();
-            } catch (Throwable ignore) {
-              // keep going - we want to clean up as much as possible
-            }
+  static IndexReader open(final Directory directory, final IndexCommit commit,
+                          final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
+    return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
+      @Override
+      protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
+        SegmentInfos sis = new SegmentInfos();
+        sis.read(directory, segmentFileName);
+        final SegmentReader[] readers = new SegmentReader[sis.size()];
+        for (int i = sis.size()-1; i >= 0; i--) {
+          IOException prior = null;
+          boolean success = false;
+          try {
+            readers[i] = SegmentReader.get(sis.info(i), termInfosIndexDivisor, IOContext.READ);
+            success = true;
+          } catch(IOException ex) {
+            prior = ex;
+          } finally {
+            if (!success)
+              IOUtils.closeWhileHandlingException(prior, readers);
           }
         }
+        return new DirectoryReader(readers, directory, null, sis, termInfosIndexDivisor, false);
       }
-    }
-
-    initialize(readers);
+    }.run(commit);
   }
 
   // Used by near real-time search
-  DirectoryReader(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes) throws IOException {
-    this.directory = writer.getDirectory();
-    this.readOnly = true;
-    this.applyAllDeletes = applyAllDeletes;       // saved for reopen
-
-    this.termInfosIndexDivisor = writer.getConfig().getReaderTermsIndexDivisor();
-    readerFinishedListeners = writer.getReaderFinishedListeners();
-
+  static DirectoryReader open(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes) throws IOException {
     // IndexWriter synchronizes externally before calling
     // us, which ensures infos will not change; so there's
     // no need to process segments in reverse order
@@ -148,16 +86,16 @@ class DirectoryReader extends IndexReade
     List<SegmentReader> readers = new ArrayList<SegmentReader>();
     final Directory dir = writer.getDirectory();
 
-    segmentInfos = (SegmentInfos) infos.clone();
+    final SegmentInfos segmentInfos = (SegmentInfos) infos.clone();
     int infosUpto = 0;
     for (int i=0;i<numSegments;i++) {
+      IOException prior = null;
       boolean success = false;
       try {
         final SegmentInfo info = infos.info(i);
         assert info.dir == dir;
         final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, IOContext.READ);
         if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) {
-          reader.readerFinishedListeners = readerFinishedListeners;
           readers.add(reader);
           infosUpto++;
         } else {
@@ -165,38 +103,23 @@ class DirectoryReader extends IndexReade
           segmentInfos.remove(infosUpto);
         }
         success = true;
+      } catch(IOException ex) {
+        prior = ex;
       } finally {
-        if (!success) {
-          // Close all readers we had opened:
-          for(SegmentReader reader : readers) {
-            try {
-              reader.close();
-            } catch (Throwable ignore) {
-              // keep going - we want to clean up as much as possible
-            }
-          }
-        }
+        if (!success)
+          IOUtils.closeWhileHandlingException(prior, readers);
       }
     }
-
-    this.writer = writer;
-
-    initialize(readers.toArray(new SegmentReader[readers.size()]));
+    return new DirectoryReader(readers.toArray(new SegmentReader[readers.size()]),
+      dir, writer, segmentInfos, writer.getConfig().getReaderTermsIndexDivisor(), applyAllDeletes);
   }
 
   /** This constructor is only used for {@link #doOpenIfChanged()} */
-  DirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders,
-                  boolean readOnly, boolean doClone, int termInfosIndexDivisor, Collection<ReaderFinishedListener> readerFinishedListeners) throws IOException {
-    this.directory = directory;
-    this.readOnly = readOnly;
-    this.segmentInfos = infos;
-    this.termInfosIndexDivisor = termInfosIndexDivisor;
-    this.readerFinishedListeners = readerFinishedListeners;
-    applyAllDeletes = false;
-
+  static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, SegmentReader[] oldReaders,
+    boolean doClone, int termInfosIndexDivisor) throws IOException {
     // we put the old SegmentReaders in a map, that allows us
     // to lookup a reader using its segment name
-    Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
+    final Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
 
     if (oldReaders != null) {
       // create a Map SegmentName->SegmentReader
@@ -223,6 +146,7 @@ class DirectoryReader extends IndexReade
       }
 
       boolean success = false;
+      IOException prior = null;
       try {
         SegmentReader newReader;
         if (newReaders[i] == null || infos.info(i).getUseCompoundFile() != newReaders[i].getSegmentInfo().getUseCompoundFile()) {
@@ -231,25 +155,25 @@ class DirectoryReader extends IndexReade
           assert !doClone;
 
           // this is a new reader; in case we hit an exception we can close it safely
-          newReader = SegmentReader.get(readOnly, infos.info(i), termInfosIndexDivisor, IOContext.READ);
-          newReader.readerFinishedListeners = readerFinishedListeners;
+          newReader = SegmentReader.get(infos.info(i), termInfosIndexDivisor, IOContext.READ);
           readerShared[i] = false;
           newReaders[i] = newReader;
         } else {
-          newReader = newReaders[i].reopenSegment(infos.info(i), doClone, readOnly);
+          newReader = newReaders[i].reopenSegment(infos.info(i), doClone);
           if (newReader == null) {
             // this reader will be shared between the old and the new one,
             // so we must incRef it
             readerShared[i] = true;
             newReaders[i].incRef();
           } else {
-            assert newReader.readerFinishedListeners == readerFinishedListeners;
             readerShared[i] = false;
             // Steal ref returned to us by reopenSegment:
             newReaders[i] = newReader;
           }
         }
         success = true;
+      } catch (IOException ex) {
+        prior = ex;
       } finally {
         if (!success) {
           for (i++; i < infos.size(); i++) {
@@ -264,26 +188,24 @@ class DirectoryReader extends IndexReade
                   // closing we must decRef it
                   newReaders[i].decRef();
                 }
-              } catch (IOException ignore) {
-                // keep going - we want to clean up as much as possible
+              } catch (IOException ex) {
+                if (prior == null) prior = ex;
               }
             }
           }
         }
+        // throw the first exception
+        if (prior != null) throw prior;
       }
     }    
-    
-    // initialize the readers to calculate maxDoc before we try to reuse the old normsCache
-    initialize(newReaders);
+    return new DirectoryReader(newReaders, directory, writer, 
+        infos, termInfosIndexDivisor, false);
   }
 
   /** {@inheritDoc} */
   @Override
   public String toString() {
     final StringBuilder buffer = new StringBuilder();
-    if (hasChanges) {
-      buffer.append("*");
-    }
     buffer.append(getClass().getSimpleName());
     buffer.append('(');
     final String segmentsFile = segmentInfos.getCurrentSegmentFileName();
@@ -301,105 +223,46 @@ class DirectoryReader extends IndexReade
     return buffer.toString();
   }
 
-  private void initialize(SegmentReader[] subReaders) throws IOException {
-    this.subReaders = subReaders;
-    starts = new int[subReaders.length + 1];    // build starts array
-    final AtomicReaderContext[] subReaderCtx = new AtomicReaderContext[subReaders.length];
-    topLevelReaderContext = new CompositeReaderContext(this, subReaderCtx, subReaderCtx);
-    final List<Fields> subFields = new ArrayList<Fields>();
-    
-    for (int i = 0; i < subReaders.length; i++) {
-      starts[i] = maxDoc;
-      subReaderCtx[i] = new AtomicReaderContext(topLevelReaderContext, subReaders[i], i, maxDoc, i, maxDoc);
-      maxDoc += subReaders[i].maxDoc();      // compute maxDocs
-
-      if (subReaders[i].hasDeletions()) {
-        hasDeletions = true;
-      }
-      
-      final Fields f = subReaders[i].fields();
-      if (f != null) {
-        subFields.add(f);
-      }
-    }
-    starts[subReaders.length] = maxDoc;
-
-    if (!readOnly) {
-      maxIndexVersion = SegmentInfos.readCurrentVersion(directory);
-    }
-  }
-
-  @Override
-  public Bits getLiveDocs() {
-    throw new UnsupportedOperationException("please use MultiFields.getLiveDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Bits liveDocs");
-  }
-
   @Override
   public final synchronized Object clone() {
     try {
-      return clone(readOnly); // Preserve current readOnly
+      DirectoryReader newReader = doOpenIfChanged((SegmentInfos) segmentInfos.clone(), true, writer);
+      return newReader;
     } catch (Exception ex) {
       throw new RuntimeException(ex);
     }
   }
 
   @Override
-  public final synchronized IndexReader clone(boolean openReadOnly) throws CorruptIndexException, IOException {
-    // doOpenIfChanged calls ensureOpen
-    DirectoryReader newReader = doOpenIfChanged((SegmentInfos) segmentInfos.clone(), true, openReadOnly);
-
-    if (this != newReader) {
-      newReader.deletionPolicy = deletionPolicy;
-    }
-    newReader.writer = writer;
-    // If we're cloning a non-readOnly reader, move the
-    // writeLock (if there is one) to the new reader:
-    if (!openReadOnly && writeLock != null) {
-      // In near real-time search, reader is always readonly
-      assert writer == null;
-      newReader.writeLock = writeLock;
-      newReader.hasChanges = hasChanges;
-      newReader.hasDeletions = hasDeletions;
-      writeLock = null;
-      hasChanges = false;
-    }
-    assert newReader.readerFinishedListeners != null;
-
-    return newReader;
-  }
-
-  @Override
   protected final IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
-    // Preserve current readOnly
-    return doOpenIfChanged(readOnly, null);
-  }
-
-  @Override
-  protected final IndexReader doOpenIfChanged(boolean openReadOnly) throws CorruptIndexException, IOException {
-    return doOpenIfChanged(openReadOnly, null);
+    return doOpenIfChanged(null);
   }
 
   @Override
   protected final IndexReader doOpenIfChanged(final IndexCommit commit) throws CorruptIndexException, IOException {
-    return doOpenIfChanged(true, commit);
+    ensureOpen();
+
+    // If we were obtained by writer.getReader(), re-ask the
+    // writer to get a new reader.
+    if (writer != null) {
+      return doOpenFromWriter(commit);
+    } else {
+      return doOpenNoWriter(commit);
+    }
   }
 
   @Override
   protected final IndexReader doOpenIfChanged(IndexWriter writer, boolean applyAllDeletes) throws CorruptIndexException, IOException {
+    ensureOpen();
     if (writer == this.writer && applyAllDeletes == this.applyAllDeletes) {
-      return doOpenIfChanged();
-    } else {    
+      return doOpenFromWriter(null);
+    } else {
+      // fail by calling supers impl throwing UOE
       return super.doOpenIfChanged(writer, applyAllDeletes);
     }
   }
 
-  private final IndexReader doOpenFromWriter(boolean openReadOnly, IndexCommit commit) throws CorruptIndexException, IOException {
-    assert readOnly;
-
-    if (!openReadOnly) {
-      throw new IllegalArgumentException("a reader obtained from IndexWriter.getReader() can only be reopened with openReadOnly=true (got false)");
-    }
-
+  private final IndexReader doOpenFromWriter(IndexCommit commit) throws CorruptIndexException, IOException {
     if (commit != null) {
       throw new IllegalArgumentException("a reader obtained from IndexWriter.getReader() cannot currently accept a commit");
     }
@@ -416,60 +279,21 @@ class DirectoryReader extends IndexReade
       return null;
     }
 
-    reader.readerFinishedListeners = readerFinishedListeners;
     return reader;
   }
 
-  private IndexReader doOpenIfChanged(final boolean openReadOnly, IndexCommit commit) throws CorruptIndexException, IOException {
-    ensureOpen();
-
-    assert commit == null || openReadOnly;
-
-    // If we were obtained by writer.getReader(), re-ask the
-    // writer to get a new reader.
-    if (writer != null) {
-      return doOpenFromWriter(openReadOnly, commit);
-    } else {
-      return doOpenNoWriter(openReadOnly, commit);
-    }
-  }
-
-  private synchronized IndexReader doOpenNoWriter(final boolean openReadOnly, IndexCommit commit) throws CorruptIndexException, IOException {
+  private synchronized IndexReader doOpenNoWriter(IndexCommit commit) throws CorruptIndexException, IOException {
 
     if (commit == null) {
-      if (hasChanges) {
-        // We have changes, which means we are not readOnly:
-        assert readOnly == false;
-        // and we hold the write lock:
-        assert writeLock != null;
-        // so no other writer holds the write lock, which
-        // means no changes could have been done to the index:
-        assert isCurrent();
-
-        if (openReadOnly) {
-          return clone(openReadOnly);
-        } else {
-          return null;
-        }
-      } else if (isCurrent()) {
-        if (openReadOnly != readOnly) {
-          // Just fallback to clone
-          return clone(openReadOnly);
-        } else {
-          return null;
-        }
+      if (isCurrent()) {
+        return null;
       }
     } else {
       if (directory != commit.getDirectory()) {
         throw new IOException("the specified commit does not match the specified Directory");
       }
       if (segmentInfos != null && commit.getSegmentsFileName().equals(segmentInfos.getCurrentSegmentFileName())) {
-        if (readOnly != openReadOnly) {
-          // Just fallback to clone
-          return clone(openReadOnly);
-        } else {
-          return null;
-        }
+        return null;
       }
     }
 
@@ -478,13 +302,13 @@ class DirectoryReader extends IndexReade
       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
         final SegmentInfos infos = new SegmentInfos();
         infos.read(directory, segmentFileName);
-        return doOpenIfChanged(infos, false, openReadOnly);
+        return doOpenIfChanged(infos, false, null);
       }
     }.run(commit);
   }
 
-  private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
-    return new DirectoryReader(directory, infos, subReaders, openReadOnly, doClone, termInfosIndexDivisor, readerFinishedListeners);
+  private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone, IndexWriter writer) throws CorruptIndexException, IOException {
+    return DirectoryReader.open(directory, writer, infos, subReaders, doClone, termInfosIndexDivisor);
   }
 
   /** Version number when this IndexReader was opened. */
@@ -495,294 +319,11 @@ class DirectoryReader extends IndexReade
   }
 
   @Override
-  public Fields getTermVectors(int docID) throws IOException {
-    ensureOpen();
-    int i = readerIndex(docID);        // find segment num
-    return subReaders[i].getTermVectors(docID - starts[i]); // dispatch to segment
-  }
-
-  @Override
-  public int numDocs() {
-    // Don't call ensureOpen() here (it could affect performance)
-
-    // NOTE: multiple threads may wind up init'ing
-    // numDocs... but that's harmless
-    if (numDocs == -1) {        // check cache
-      int n = 0;                // cache miss--recompute
-      for (int i = 0; i < subReaders.length; i++)
-        n += subReaders[i].numDocs();      // sum from readers
-      numDocs = n;
-    }
-    return numDocs;
-  }
-
-  @Override
-  public int maxDoc() {
-    // Don't call ensureOpen() here (it could affect performance)
-    return maxDoc;
-  }
-
-  @Override
-  public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
-    ensureOpen();
-    int i = readerIndex(docID);                          // find segment num
-    subReaders[i].document(docID - starts[i], visitor);    // dispatch to segment reader
-  }
-
-  @Override
-  public boolean hasDeletions() {
-    ensureOpen();
-    return hasDeletions;
-  }
-
-  @Override
-  protected void doDelete(int n) throws CorruptIndexException, IOException {
-    numDocs = -1;                             // invalidate cache
-    int i = readerIndex(n);                   // find segment num
-    subReaders[i].deleteDocument(n - starts[i]);      // dispatch to segment reader
-    hasDeletions = true;
-  }
-
-  @Override
-  protected void doUndeleteAll() throws CorruptIndexException, IOException {
-    for (int i = 0; i < subReaders.length; i++)
-      subReaders[i].undeleteAll();
-
-    hasDeletions = false;
-    numDocs = -1;                                 // invalidate cache
-  }
-
-  private int readerIndex(int n) {    // find reader for doc n:
-    return readerIndex(n, this.starts, this.subReaders.length);
-  }
-  
-  final static int readerIndex(int n, int[] starts, int numSubReaders) {    // find reader for doc n:
-    int lo = 0;                                      // search starts array
-    int hi = numSubReaders - 1;                  // for first element less
-
-    while (hi >= lo) {
-      int mid = (lo + hi) >>> 1;
-      int midValue = starts[mid];
-      if (n < midValue)
-        hi = mid - 1;
-      else if (n > midValue)
-        lo = mid + 1;
-      else {                                      // found a match
-        while (mid+1 < numSubReaders && starts[mid+1] == midValue) {
-          mid++;                                  // scan to last match
-        }
-        return mid;
-      }
-    }
-    return hi;
-  }
-
-  @Override
-  public boolean hasNorms(String field) throws IOException {
-    ensureOpen();
-    for (int i = 0; i < subReaders.length; i++) {
-      if (subReaders[i].hasNorms(field)) return true;
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized byte[] norms(String field) throws IOException {
-    ensureOpen();
-    throw new UnsupportedOperationException("please use MultiNorms.norms, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level norms");
-  }
-
-  @Override
-  protected void doSetNorm(int n, String field, byte value)
-    throws CorruptIndexException, IOException {
-    int i = readerIndex(n);                           // find segment num
-    subReaders[i].setNorm(n-starts[i], field, value); // dispatch
-  }
-
-  @Override
-  public int docFreq(Term t) throws IOException {
-    ensureOpen();
-    int total = 0;          // sum freqs in segments
-    for (int i = 0; i < subReaders.length; i++)
-      total += subReaders[i].docFreq(t);
-    return total;
-  }
-
-  @Override
-  public int docFreq(String field, BytesRef term) throws IOException {
-    ensureOpen();
-    int total = 0;          // sum freqs in segments
-    for (int i = 0; i < subReaders.length; i++) {
-      total += subReaders[i].docFreq(field, term);
-    }
-    return total;
-  }
-
-  @Override
-  public Fields fields() throws IOException {
-    throw new UnsupportedOperationException("please use MultiFields.getFields, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
-  }
-
-  /**
-   * Tries to acquire the WriteLock on this directory. this method is only valid if this IndexReader is directory
-   * owner.
-   *
-   * @throws StaleReaderException  if the index has changed since this reader was opened
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws org.apache.lucene.store.LockObtainFailedException
-   *                               if another writer has this index open (<code>write.lock</code> could not be
-   *                               obtained)
-   * @throws IOException           if there is a low-level IO error
-   */
-  @Override
-  protected void acquireWriteLock() throws StaleReaderException, CorruptIndexException, LockObtainFailedException, IOException {
-
-    if (readOnly) {
-      // NOTE: we should not reach this code w/ the core
-      // IndexReader classes; however, an external subclass
-      // of IndexReader could reach this.
-      throw new UnsupportedOperationException("This IndexReader cannot make any changes to the index (it was opened with readOnly = true)");
-    }
-
-    if (segmentInfos != null) {
-      ensureOpen();
-      if (stale)
-        throw new StaleReaderException("IndexReader out of date and no longer valid for delete, undelete, or setNorm operations");
-
-      if (writeLock == null) {
-        Lock writeLock = directory.makeLock(IndexWriter.WRITE_LOCK_NAME);
-        if (!writeLock.obtain(IndexWriterConfig.WRITE_LOCK_TIMEOUT)) // obtain write lock
-          throw new LockObtainFailedException("Index locked for write: " + writeLock);
-        this.writeLock = writeLock;
-
-        // we have to check whether index has changed since this reader was opened.
-        // if so, this reader is no longer valid for deletion
-        if (SegmentInfos.readCurrentVersion(directory) > maxIndexVersion) {
-          stale = true;
-          this.writeLock.release();
-          this.writeLock = null;
-          throw new StaleReaderException("IndexReader out of date and no longer valid for delete, undelete, or setNorm operations");
-        }
-      }
-    }
-  }
-
-  /**
-   * Commit changes resulting from delete, undeleteAll, or setNorm operations
-   * <p/>
-   * If an exception is hit, then either no changes or all changes will have been committed to the index (transactional
-   * semantics).
-   *
-   * @throws IOException if there is a low-level IO error
-   */
-  @Override
-  protected void doCommit(Map<String,String> commitUserData) throws IOException {
-    // poll subreaders for changes
-    for (int i = 0; !hasChanges && i < subReaders.length; i++) {
-      hasChanges |= subReaders[i].hasChanges;
-    }
-    
-    if (hasChanges) {
-      segmentInfos.setUserData(commitUserData);
-      // Default deleter (for backwards compatibility) is
-      // KeepOnlyLastCommitDeleter:
-      // TODO: Decide what to do with InfoStream here? Use default or keep NO_OUTPUT?
-      IndexFileDeleter deleter = new IndexFileDeleter(directory,
-                                                      deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
-                                                      segmentInfos, InfoStream.NO_OUTPUT, null);
-      segmentInfos.updateGeneration(deleter.getLastSegmentInfos());
-      segmentInfos.changed();
-
-      // Checkpoint the state we are about to change, in
-      // case we have to roll back:
-      startCommit();
-
-      final List<SegmentInfo> rollbackSegments = segmentInfos.createBackupSegmentInfos(false);
-
-      boolean success = false;
-      try {
-        for (int i = 0; i < subReaders.length; i++)
-          subReaders[i].commit();
-
-        // Remove segments that contain only 100% deleted
-        // docs:
-        segmentInfos.pruneDeletedSegments();
-
-        // Sync all files we just wrote
-        directory.sync(segmentInfos.files(directory, false));
-        segmentInfos.commit(directory, segmentInfos.codecFormat());
-        success = true;
-      } finally {
-
-        if (!success) {
-
-          // Rollback changes that were made to
-          // SegmentInfos but failed to get [fully]
-          // committed.  This way this reader instance
-          // remains consistent (matched to what's
-          // actually in the index):
-          rollbackCommit();
-
-          // Recompute deletable files & remove them (so
-          // partially written .del files, etc, are
-          // removed):
-          deleter.refresh();
-
-          // Restore all SegmentInfos (in case we pruned some)
-          segmentInfos.rollbackSegmentInfos(rollbackSegments);
-        }
-      }
-
-      // Have the deleter remove any now unreferenced
-      // files due to this commit:
-      deleter.checkpoint(segmentInfos, true);
-      deleter.close();
-
-      maxIndexVersion = segmentInfos.getVersion();
-
-      if (writeLock != null) {
-        writeLock.release();  // release write lock
-        writeLock = null;
-      }
-    }
-    hasChanges = false;
-  }
-
-  void startCommit() {
-    rollbackHasChanges = hasChanges;
-    for (int i = 0; i < subReaders.length; i++) {
-      subReaders[i].startCommit();
-    }
-  }
-
-  void rollbackCommit() {
-    hasChanges = rollbackHasChanges;
-    for (int i = 0; i < subReaders.length; i++) {
-      subReaders[i].rollbackCommit();
-    }
-  }
-
-  @Override
-  public long getUniqueTermCount() throws IOException {
-    return -1;
-  }
-
-  @Override
   public Map<String,String> getCommitUserData() {
     ensureOpen();
     return segmentInfos.getUserData();
   }
 
-  /**
-   * Check whether this IndexReader is still using the current (i.e., most recently committed) version of the index.  If
-   * a writer has committed any changes to the index since this reader was opened, this will return <code>false</code>,
-   * in which case you must open a new IndexReader in order
-   * to see the changes.  Use {@link IndexWriter#commit} to
-   * commit changes to the index.
-   *
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException           if there is a low-level IO error
-   */
   @Override
   public boolean isCurrent() throws CorruptIndexException, IOException {
     ensureOpen();
@@ -816,33 +357,6 @@ class DirectoryReader extends IndexReade
     if (ioe != null) throw ioe;
   }
 
-  @Override
-  public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
-    ensureOpen();
-    return getFieldNames(fieldNames, this.subReaders);
-  }
-  
-  static Collection<String> getFieldNames (IndexReader.FieldOption fieldNames, IndexReader[] subReaders) {
-    // maintain a unique set of field names
-    Set<String> fieldSet = new HashSet<String>();
-    for (IndexReader reader : subReaders) {
-      Collection<String> names = reader.getFieldNames(fieldNames);
-      fieldSet.addAll(names);
-    }
-    return fieldSet;
-  }
-  
-  @Override
-  public ReaderContext getTopReaderContext() {
-    ensureOpen();
-    return topLevelReaderContext;
-  }
-  
-  @Override
-  public IndexReader[] getSequentialSubReaders() {
-    return subReaders;
-  }
-
   /** Returns the directory this index resides in. */
   @Override
   public Directory directory() {
@@ -985,9 +499,4 @@ class DirectoryReader extends IndexReade
       throw new UnsupportedOperationException("This IndexCommit does not support deletions");
     }
   }
-
-  @Override
-  public PerDocValues perDocValues() throws IOException {
-    throw new UnsupportedOperationException("please use MultiPerDocValues#getPerDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
-  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Thu Dec 15 05:11:14 2011
@@ -26,11 +26,11 @@ import java.util.Map;
 
 import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.DocValuesConsumer;
+import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.values.PerDocFieldValues;
+import org.apache.lucene.index.DocValues;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils;
@@ -224,7 +224,7 @@ final class DocFieldProcessor extends Do
         // needs to be more "pluggable" such that if I want
         // to have a new "thing" my Fields can do, I can
         // easily add it
-        FieldInfo fi = fieldInfos.addOrUpdate(fieldName, field.fieldType(), false, field.docValuesType());
+        FieldInfo fi = fieldInfos.addOrUpdate(fieldName, field.fieldType());
 
         fp = new DocFieldProcessorPerField(this, fi);
         fp.next = fieldHash[hashPos];
@@ -235,7 +235,7 @@ final class DocFieldProcessor extends Do
           rehash();
         }
       } else {
-        fieldInfos.addOrUpdate(fp.fieldInfo.name, field.fieldType(), false, field.docValuesType());
+        fieldInfos.addOrUpdate(fp.fieldInfo.name, field.fieldType());
       }
 
       if (thisFieldGen != fp.lastGen) {
@@ -259,9 +259,9 @@ final class DocFieldProcessor extends Do
       if (field.fieldType().stored()) {
         fieldsWriter.addField(field, fp.fieldInfo);
       }
-      final PerDocFieldValues docValues = field.docValues();
-      if (docValues != null) {
-        docValuesConsumer(docState, fp.fieldInfo).add(docState.docID, docValues);
+      final DocValue docValue = field.docValue();
+      if (docValue != null) {
+        docValuesConsumer(field.docValueType(), docState, fp.fieldInfo).add(docState.docID, docValue);
       }
     }
 
@@ -277,8 +277,8 @@ final class DocFieldProcessor extends Do
       perField.consumer.processFields(perField.fields, perField.fieldCount);
     }
 
-    if (docState.maxTermPrefix != null && docState.infoStream != null) {
-      docState.infoStream.println("WARNING: document contains at least one immense term (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped.  Please correct the analyzer to not produce such terms.  The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");
+    if (docState.maxTermPrefix != null && docState.infoStream.isEnabled("IW")) {
+      docState.infoStream.message("IW", "WARNING: document contains at least one immense term (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped.  Please correct the analyzer to not produce such terms.  The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");
       docState.maxTermPrefix = null;
     }
   }
@@ -310,12 +310,12 @@ final class DocFieldProcessor extends Do
   final private Map<String, DocValuesConsumerAndDocID> docValues = new HashMap<String, DocValuesConsumerAndDocID>();
   final private Map<Integer, PerDocConsumer> perDocConsumers = new HashMap<Integer, PerDocConsumer>();
 
-  DocValuesConsumer docValuesConsumer(DocState docState, FieldInfo fieldInfo) 
+  DocValuesConsumer docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo) 
       throws IOException {
     DocValuesConsumerAndDocID docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
     if (docValuesConsumerAndDocID != null) {
       if (docState.docID == docValuesConsumerAndDocID.docID) {
-        throw new IllegalArgumentException("IndexDocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
+        throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
       }
       assert docValuesConsumerAndDocID.docID < docState.docID;
       docValuesConsumerAndDocID.docID = docState.docID;
@@ -329,17 +329,9 @@ final class DocFieldProcessor extends Do
       perDocConsumer = dvFormat.docsConsumer(perDocWriteState);
       perDocConsumers.put(0, perDocConsumer);
     }
-    boolean success = false;
-    DocValuesConsumer docValuesConsumer = null;
-    try {
-      docValuesConsumer = perDocConsumer.addValuesField(fieldInfo);
-      fieldInfo.commitDocValues();
-      success = true;
-    } finally {
-      if (!success) {
-        fieldInfo.revertUncommitted();
-      }
-    }
+
+    DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
+    fieldInfo.setDocValuesType(valueType);
 
     docValuesConsumerAndDocID = new DocValuesConsumerAndDocID(docValuesConsumer);
     docValuesConsumerAndDocID.docID = docState.docID;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocInverter.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocInverter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocInverter.java Thu Dec 15 05:11:14 2011
@@ -56,9 +56,6 @@ final class DocInverter extends DocField
     }
   }
 
-  // Used to read a string value for a field
-  final ReusableStringReader stringReader = new ReusableStringReader();
-
   public DocInverter(DocumentsWriterPerThread.DocState docState, InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) {
     this.docState = docState;
     this.consumer = consumer;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Thu Dec 15 05:11:14 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.io.PrintStream;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -52,7 +51,7 @@ import org.apache.lucene.util.InfoStream
  * are processing the document).
  *
  * Other consumers, eg {@link FreqProxTermsWriter} and
- * {@link NormsWriter}, buffer bytes in RAM and flush only
+ * {@link NormsConsumer}, buffer bytes in RAM and flush only
  * when a new segment is produced.
 
  * Once we have used our allowed RAM buffer, or the number
@@ -216,7 +215,9 @@ final class DocumentsWriter {
     }
 
     try {
-      infoStream.message("DW", "abort");
+      if (infoStream.isEnabled("DW")) {
+        infoStream.message("DW", "abort");
+      }
 
       final Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
       while (threadsIterator.hasNext()) {
@@ -298,8 +299,10 @@ final class DocumentsWriter {
           maybeMerge |= doFlush(flushingDWPT);
         }
   
-        if (infoStream.isEnabled("DW") && flushControl.anyStalledThreads()) {
-          infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
+        if (infoStream.isEnabled("DW")) {
+          if (flushControl.anyStalledThreads()) {
+            infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
+          }
         }
         
         flushControl.waitIfStalled(); // block if stalled

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Thu Dec 15 05:11:14 2011
@@ -21,7 +21,6 @@ import static org.apache.lucene.util.Byt
 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
 import java.io.IOException;
-import java.io.PrintStream;
 import java.text.NumberFormat;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -79,7 +78,7 @@ public class DocumentsWriterPerThread {
 
       final InvertedDocConsumer  termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
                                                            new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
-      final NormsWriter normsWriter = new NormsWriter();
+      final NormsConsumer normsWriter = new NormsConsumer(documentsWriterPerThread);
       final DocInverter docInverter = new DocInverter(documentsWriterPerThread.docState, termsHash, normsWriter);
       return new DocFieldProcessor(documentsWriterPerThread, docInverter);
     }
@@ -88,14 +87,15 @@ public class DocumentsWriterPerThread {
   static class DocState {
     final DocumentsWriterPerThread docWriter;
     Analyzer analyzer;
-    PrintStream infoStream;
+    InfoStream infoStream;
     SimilarityProvider similarityProvider;
     int docID;
     Iterable<? extends IndexableField> doc;
     String maxTermPrefix;
 
-    DocState(DocumentsWriterPerThread docWriter) {
+    DocState(DocumentsWriterPerThread docWriter, InfoStream infoStream) {
       this.docWriter = docWriter;
+      this.infoStream = infoStream;
     }
 
     // Only called by asserts
@@ -131,7 +131,9 @@ public class DocumentsWriterPerThread {
   void abort() throws IOException {
     hasAborted = aborting = true;
     try {
-      infoStream.message("DWPT", "now abort");
+      if (infoStream.isEnabled("DWPT")) {
+        infoStream.message("DWPT", "now abort");
+      }
       try {
         consumer.abort();
       } catch (Throwable t) {
@@ -144,7 +146,9 @@ public class DocumentsWriterPerThread {
 
     } finally {
       aborting = false;
-      infoStream.message("DWPT", "done abort");
+      if (infoStream.isEnabled("DWPT")) {
+        infoStream.message("DWPT", "done abort");
+      }
     }
   }
   private final static boolean INFO_VERBOSE = false;
@@ -181,7 +185,7 @@ public class DocumentsWriterPerThread {
     this.writer = parent.indexWriter;
     this.infoStream = parent.infoStream;
     this.codec = parent.codec;
-    this.docState = new DocState(this);
+    this.docState = new DocState(this, infoStream);
     this.docState.similarityProvider = parent.indexWriter.getConfig()
         .getSimilarityProvider();
     bytesUsed = Counter.newCounter();
@@ -244,7 +248,6 @@ public class DocumentsWriterPerThread {
           // mark document as deleted
           deleteDocID(docState.docID);
           numDocsInRAM++;
-          fieldInfos.revertUncommitted();
         } else {
           abort();
         }
@@ -308,7 +311,6 @@ public class DocumentsWriterPerThread {
               // Incr here because finishDocument will not
               // be called (because an exc is being thrown):
               numDocsInRAM++;
-              fieldInfos.revertUncommitted();
             } else {
               abort();
             }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfo.java Thu Dec 15 05:11:14 2011
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-import org.apache.lucene.index.values.ValueType;
+import org.apache.lucene.index.DocValues;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -25,7 +25,7 @@ public final class FieldInfo {
   public final int number;
 
   public boolean isIndexed;
-  ValueType docValues;
+  private DocValues.Type docValues;
 
 
   // true if term vector for this field should be stored
@@ -57,7 +57,7 @@ public final class FieldInfo {
    */
   public FieldInfo(String name, boolean isIndexed, int number, boolean storeTermVector, 
             boolean storePositionWithTermVector,  boolean storeOffsetWithTermVector, 
-            boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValues) {
+            boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
     this.name = name;
     this.isIndexed = isIndexed;
     this.number = number;
@@ -118,13 +118,14 @@ public final class FieldInfo {
     }
     assert this.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !this.storePayloads;
   }
-  void setDocValues(ValueType v) {
+
+  void setDocValuesType(DocValues.Type v) {
     if (docValues == null) {
       docValues = v;
     }
   }
   
-  public void resetDocValues(ValueType v) {
+  public void resetDocValuesType(DocValues.Type v) {
     if (docValues != null) {
       docValues = v;
     }
@@ -134,42 +135,13 @@ public final class FieldInfo {
     return docValues != null;
   }
 
-  public ValueType getDocValues() {
+  public DocValues.Type getDocValuesType() {
     return docValues;
   }
-  
-  private boolean vectorsCommitted;
-  private boolean docValuesCommitted;
- 
-  /**
-   * Reverts all uncommitted changes on this {@link FieldInfo}
-   * @see #commitVectors()
-   */
-  void revertUncommitted() {
-    if (storeTermVector && !vectorsCommitted) {
-      storeOffsetWithTermVector = false;
-      storePositionWithTermVector = false;
-      storeTermVector = false;  
-    }
-    
-    if (docValues != null && !docValuesCommitted) {
-      docValues = null;
-    }
-  }
 
-  /**
-   * Commits term vector modifications. Changes to term-vectors must be
-   * explicitly committed once the necessary files are created. If those changes
-   * are not committed subsequent {@link #revertUncommitted()} will reset the
-   * all term-vector flags before the next document.
-   */
-  void commitVectors() {
-    assert storeTermVector;
-    vectorsCommitted = true;
-  }
-  
-  void commitDocValues() {
-    assert hasDocValues();
-    docValuesCommitted = true;
+  public void setStoreTermVectors(boolean withPositions, boolean withOffsets) {
+    storeTermVector = true;
+    storePositionWithTermVector |= withPositions;
+    storeOffsetWithTermVector |= withOffsets;
   }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfos.java Thu Dec 15 05:11:14 2011
@@ -25,7 +25,7 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.lucene.index.FieldInfo.IndexOptions;
-import org.apache.lucene.index.values.ValueType;
+import org.apache.lucene.index.DocValues;
 
 /** Access to the Field Info file that describes document fields and whether or
  *  not they are indexed. Each segment has a separate Field Info file. Objects
@@ -299,20 +299,30 @@ public final class FieldInfos implements
    */
   synchronized public FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
                        boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
-                       boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValues) {
+                       boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
     return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, storePositionWithTermVector,
                                storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
   }
 
-  synchronized public FieldInfo addOrUpdate(String name, IndexableFieldType fieldType, boolean scorePayloads, ValueType docValues) {
-    return addOrUpdateInternal(name, -1, fieldType.indexed(), fieldType.storeTermVectors(),
-        fieldType.storeTermVectorPositions(), fieldType.storeTermVectorOffsets(), fieldType.omitNorms(), scorePayloads,
-        fieldType.indexOptions(), docValues);
+  // NOTE: this method does not carry over termVector
+  // booleans nor docValuesType; the indexer chain
+  // (TermVectorsConsumerPerField, DocFieldProcessor) must
+  // set these fields when they succeed in consuming
+  // the document:
+  public FieldInfo addOrUpdate(String name, IndexableFieldType fieldType) {
+    // TODO: really, indexer shouldn't even call this
+    // method (it's only called from DocFieldProcessor);
+    // rather, each component in the chain should update
+    // what it "owns".  EG fieldType.indexOptions() should
+    // be updated by maybe FreqProxTermsWriterPerField:
+    return addOrUpdateInternal(name, -1, fieldType.indexed(), false, false, false,
+                               fieldType.omitNorms(), false,
+                               fieldType.indexOptions(), null);
   }
 
   synchronized private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
       boolean storeTermVector, boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
-      boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValues) {
+      boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
     if (globalFieldNumbers == null) {
       throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos");
     }
@@ -322,7 +332,7 @@ public final class FieldInfos implements
       fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
     } else {
       fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions);
-      fi.setDocValues(docValues);
+      fi.setDocValuesType(docValues);
     }
     version++;
     return fi;
@@ -333,7 +343,7 @@ public final class FieldInfos implements
     return addOrUpdateInternal(fi.name, fi.number, fi.isIndexed, fi.storeTermVector,
                fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
                fi.omitNorms, fi.storePayloads,
-               fi.indexOptions, fi.docValues);
+               fi.indexOptions, fi.getDocValuesType());
   }
   
   /*
@@ -341,7 +351,7 @@ public final class FieldInfos implements
    */
   private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
                                 boolean storeTermVector, boolean storePositionWithTermVector, 
-                                boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValuesType) {
+                                boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValuesType) {
     // don't check modifiable here since we use that to initially build up FIs
     if (globalFieldNumbers != null) {
       globalFieldNumbers.setIfNotSet(fieldNumber, name);
@@ -407,7 +417,7 @@ public final class FieldInfos implements
 
   public boolean hasNorms() {
     for (FieldInfo fi : this) {
-      if (!fi.omitNorms) {
+      if (fi.isIndexed && !fi.omitNorms) {
         return true;
       }
     }
@@ -429,16 +439,6 @@ public final class FieldInfos implements
     return version;
   }
   
-  /**
-   * Reverts all uncommitted changes 
-   * @see FieldInfo#revertUncommitted()
-   */
-  void revertUncommitted() {
-    for (FieldInfo fieldInfo : this) {
-      fieldInfo.revertUncommitted();
-    }
-  }
-  
   final FieldInfos asReadOnly() {
     if (isReadOnly()) {
       return this;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java Thu Dec 15 05:11:14 2011
@@ -17,17 +17,14 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.MapBackedSet;
 
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
 import java.util.Comparator;
-import java.util.concurrent.ConcurrentHashMap;
 
 /**  A <code>FilterIndexReader</code> contains another IndexReader, which it
  * uses as its basic source of data, possibly transforming the data along the
@@ -277,15 +274,12 @@ public class FilterIndexReader extends I
 
   /**
    * <p>Construct a FilterIndexReader based on the specified base reader.
-   * Directory locking for delete, undeleteAll, and setNorm operations is
-   * left to the base reader.</p>
    * <p>Note that base reader is closed if this FilterIndexReader is closed.</p>
    * @param in specified base reader.
    */
   public FilterIndexReader(IndexReader in) {
     super();
     this.in = in;
-    readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
   }
 
   @Override
@@ -332,9 +326,6 @@ public class FilterIndexReader extends I
   }
 
   @Override
-  protected void doUndeleteAll() throws CorruptIndexException, IOException {in.undeleteAll();}
-
-  @Override
   public boolean hasNorms(String field) throws IOException {
     ensureOpen();
     return in.hasNorms(field);
@@ -347,29 +338,10 @@ public class FilterIndexReader extends I
   }
 
   @Override
-  protected void doSetNorm(int d, String f, byte b) throws CorruptIndexException, IOException {
-    in.setNorm(d, f, b);
-  }
-
-  @Override
-  public int docFreq(Term t) throws IOException {
-    ensureOpen();
-    return in.docFreq(t);
-  }
-
-  @Override
   public int docFreq(String field, BytesRef t) throws IOException {
     ensureOpen();
     return in.docFreq(field, t);
   }
-
-  @Override
-  protected void doDelete(int n) throws  CorruptIndexException, IOException { in.deleteDocument(n); }
-  
-  @Override
-  protected void doCommit(Map<String,String> commitUserData) throws IOException {
-    in.commit(commitUserData);
-  }
   
   @Override
   protected void doClose() throws IOException {
@@ -416,15 +388,24 @@ public class FilterIndexReader extends I
     return in.fields();
   }
 
-  /** If the subclass of FilteredIndexReader modifies the
-   *  contents of the FieldCache, you must override this
-   *  method to provide a different key */
+  /** {@inheritDoc}
+   * <p>If the subclass of FilteredIndexReader modifies the
+   *  contents (but not liveDocs) of the index, you must override this
+   *  method to provide a different key. */
   @Override
   public Object getCoreCacheKey() {
     return in.getCoreCacheKey();
   }
 
-  /** {@inheritDoc} */
+  /** {@inheritDoc}
+   * <p>If the subclass of FilteredIndexReader modifies the
+   *  liveDocs, you must override this
+   *  method to provide a different key. */
+  @Override
+  public Object getCombinedCoreAndDeletesKey() {
+    return in.getCombinedCoreAndDeletesKey();
+  }
+
   @Override
   public String toString() {
     final StringBuilder buffer = new StringBuilder("FilterReader(");
@@ -434,20 +415,18 @@ public class FilterIndexReader extends I
   }
 
   @Override
-  public void addReaderFinishedListener(ReaderFinishedListener listener) {
-    super.addReaderFinishedListener(listener);
-    in.addReaderFinishedListener(listener);
+  public DocValues docValues(String field) throws IOException {
+    ensureOpen();
+    return in.docValues(field);
   }
 
   @Override
-  public void removeReaderFinishedListener(ReaderFinishedListener listener) {
-    super.removeReaderFinishedListener(listener);
-    in.removeReaderFinishedListener(listener);
+  public IndexCommit getIndexCommit() throws IOException {
+    return in.getIndexCommit();
   }
 
   @Override
-  public PerDocValues perDocValues() throws IOException {
-    ensureOpen();
-    return in.perDocValues();
-  }
+  public int getTermInfosIndexDivisor() {
+    return in.getTermInfosIndexDivisor();
+  }  
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FlushPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FlushPolicy.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FlushPolicy.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FlushPolicy.java Thu Dec 15 05:11:14 2011
@@ -20,7 +20,6 @@ import java.util.Iterator;
 
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.SetOnce;
 
 /**
@@ -129,7 +128,9 @@ public abstract class FlushPolicy {
   }
   
   private boolean assertMessage(String s) {
-    writer.get().infoStream.message("FP", s);
+    if (writer.get().infoStream.isEnabled("FP")) {
+      writer.get().infoStream.message("FP", s);
+    }
     return true;
   }
 

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java Thu Dec 15 05:11:14 2011
@@ -501,8 +501,10 @@ final class IndexFileDeleter {
   void incRef(String fileName) throws IOException {
     assert locked();
     RefCount rc = getRefCount(fileName);
-    if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
-      infoStream.message("IFD", "  IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
+    if (infoStream.isEnabled("IFD")) {
+      if (VERBOSE_REF_COUNTS) {
+        infoStream.message("IFD", "  IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
+      }
     }
     rc.IncRef();
   }
@@ -517,8 +519,10 @@ final class IndexFileDeleter {
   void decRef(String fileName) throws IOException {
     assert locked();
     RefCount rc = getRefCount(fileName);
-    if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
-      infoStream.message("IFD", "  DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
+    if (infoStream.isEnabled("IFD")) {
+      if (VERBOSE_REF_COUNTS) {
+        infoStream.message("IFD", "  DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
+      }
     }
     if (0 == rc.DecRef()) {
       // This file is no longer referenced by any past

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1214623&r1=1214622&r2=1214623&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNames.java Thu Dec 15 05:11:14 2011
@@ -17,8 +17,6 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.util.regex.Pattern;
-
 import org.apache.lucene.index.codecs.Codec;  // for javadocs
 
 // TODO: put all files under codec and remove all the static extensions here
@@ -49,9 +47,6 @@ public final class IndexFileNames {
   
   /** Name of the generation reference file name */
   public static final String SEGMENTS_GEN = "segments." +  GEN_EXTENSION;
-  
-  /** Extension of norms file */
-  public static final String NORMS_EXTENSION = "nrm";
 
   /** Extension of compound file */
   public static final String COMPOUND_FILE_EXTENSION = "cfs";
@@ -65,9 +60,6 @@ public final class IndexFileNames {
   /** Extension of deletes */
   public static final String DELETES_EXTENSION = "del";
 
-  /** Extension of separate norms */
-  public static final String SEPARATE_NORMS_EXTENSION = "s";
-
   /**
    * This array contains all filename extensions used by
    * Lucene's index files, with one exception, namely the
@@ -80,14 +72,9 @@ public final class IndexFileNames {
     COMPOUND_FILE_ENTRIES_EXTENSION,
     DELETES_EXTENSION,
     GEN_EXTENSION,
-    NORMS_EXTENSION,
     COMPOUND_FILE_STORE_EXTENSION,
   };
 
-  public static final String[] NON_STORE_INDEX_EXTENSIONS = new String[] {
-    NORMS_EXTENSION
-  };
-
   /**
    * Computes the full file name from base, extension and generation. If the
    * generation is -1, the file name is null. If it's 0, the file name is
@@ -188,17 +175,5 @@ public final class IndexFileNames {
       filename = filename.substring(0, idx);
     }
     return filename;
-  }
-  
-  /**
-   * Returns true if the given filename ends with the separate norms file
-   * pattern: {@code SEPARATE_NORMS_EXTENSION + "[0-9]+"}.
-   */
-  public static boolean isSeparateNormsFile(String filename) {
-    int idx = filename.lastIndexOf('.');
-    if (idx == -1) return false;
-    String ext = filename.substring(idx + 1);
-    return Pattern.matches(SEPARATE_NORMS_EXTENSION + "[0-9]+", ext);
-  }
-  
+  }  
 }