You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/10/21 20:58:44 UTC

svn commit: r1534320 [14/39] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java Mon Oct 21 18:58:24 2013
@@ -18,19 +18,31 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.NumericFieldUpdates.UpdatesIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MutableBits;
 
 // Used by IndexWriter to hold open SegmentReaders (for
-// searching or merging), plus pending deletes,
+// searching or merging), plus pending deletes and updates,
 // for a given segment
-class ReadersAndLiveDocs {
+class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
   // Not final because we replace (clone) when we need to
   // change it and it's been shared:
   public final SegmentInfoPerCommit info;
@@ -43,8 +55,8 @@ class ReadersAndLiveDocs {
   // Set once (null, and then maybe set, and never set again):
   private SegmentReader reader;
 
-  // Holds the current shared (readable and writable
-  // liveDocs).  This is null when there are no deleted
+  // Holds the current shared (readable and writable)
+  // liveDocs.  This is null when there are no deleted
   // docs, and it's copy-on-write (cloned whenever we need
   // to change it but it's been shared to an external NRT
   // reader).
@@ -56,12 +68,22 @@ class ReadersAndLiveDocs {
 
   // True if the current liveDocs is referenced by an
   // external NRT reader:
-  private boolean shared;
+  private boolean liveDocsShared;
 
+  // Indicates whether this segment is currently being merged. While a segment
+  // is merging, all field updates are also registered in the
+  // mergingNumericUpdates map. Also, calls to writeFieldUpdates merge the 
+  // updates with mergingNumericUpdates.
+  // That way, when the segment is done merging, IndexWriter can apply the
+  // updates on the merged segment too.
+  private boolean isMerging = false;
+  
+  private final Map<String,NumericFieldUpdates> mergingNumericUpdates = new HashMap<String,NumericFieldUpdates>();
+  
   public ReadersAndLiveDocs(IndexWriter writer, SegmentInfoPerCommit info) {
     this.info = info;
     this.writer = writer;
-    shared = true;
+    liveDocsShared = true;
   }
 
   public void incRef() {
@@ -83,7 +105,7 @@ class ReadersAndLiveDocs {
   public synchronized int getPendingDeleteCount() {
     return pendingDeleteCount;
   }
-
+  
   // Call only from assert!
   public synchronized boolean verifyDocCounts() {
     int count;
@@ -102,25 +124,21 @@ class ReadersAndLiveDocs {
     return true;
   }
 
-  // Get reader for searching/deleting
-  public synchronized SegmentReader getReader(IOContext context) throws IOException {
-    //System.out.println("  livedocs=" + rld.liveDocs);
-
+  /** Returns a {@link SegmentReader}. */
+  public SegmentReader getReader(IOContext context) throws IOException {
     if (reader == null) {
       // We steal returned ref:
       reader = new SegmentReader(info, context);
       if (liveDocs == null) {
         liveDocs = reader.getLiveDocs();
       }
-      //System.out.println("ADD seg=" + rld.info + " isMerge=" + isMerge + " " + readerMap.size() + " in pool");
-      //System.out.println(Thread.currentThread().getName() + ": getReader seg=" + info.name);
     }
-
+    
     // Ref for caller
     reader.incRef();
     return reader;
   }
-
+  
   public synchronized void release(SegmentReader sr) throws IOException {
     assert info == sr.getSegmentInfo();
     sr.decRef();
@@ -130,7 +148,7 @@ class ReadersAndLiveDocs {
     assert liveDocs != null;
     assert Thread.holdsLock(writer);
     assert docID >= 0 && docID < liveDocs.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + liveDocs.length() + " seg=" + info.info.name + " docCount=" + info.info.getDocCount();
-    assert !shared;
+    assert !liveDocsShared;
     final boolean didDelete = liveDocs.get(docID);
     if (didDelete) {
       ((MutableBits) liveDocs).clear(docID);
@@ -157,18 +175,17 @@ class ReadersAndLiveDocs {
   }
 
   /**
-   * Returns a ref to a clone.  NOTE: this clone is not
-   * enrolled in the pool, so you should simply close()
-   * it when you're done (ie, do not call release()).
+   * Returns a ref to a clone. NOTE: you should decRef() the reader when you're
+   * dont (ie do not call close()).
    */
   public synchronized SegmentReader getReadOnlyClone(IOContext context) throws IOException {
     if (reader == null) {
       getReader(context).decRef();
       assert reader != null;
     }
-    shared = true;
+    liveDocsShared = true;
     if (liveDocs != null) {
-      return new SegmentReader(reader.getSegmentInfo(), reader.core, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
+      return new SegmentReader(reader.getSegmentInfo(), reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
     } else {
       assert reader.getLiveDocs() == liveDocs;
       reader.incRef();
@@ -180,7 +197,7 @@ class ReadersAndLiveDocs {
     assert Thread.holdsLock(writer);
     assert info.info.getDocCount() > 0;
     //System.out.println("initWritableLivedocs seg=" + info + " liveDocs=" + liveDocs + " shared=" + shared);
-    if (shared) {
+    if (liveDocsShared) {
       // Copy on write: this means we've cloned a
       // SegmentReader sharing the current liveDocs
       // instance; must now make a private clone so we can
@@ -192,9 +209,7 @@ class ReadersAndLiveDocs {
       } else {
         liveDocs = liveDocsFormat.newLiveDocs(liveDocs);
       }
-      shared = false;
-    } else {
-      assert liveDocs != null;
+      liveDocsShared = false;
     }
   }
 
@@ -206,7 +221,7 @@ class ReadersAndLiveDocs {
   public synchronized Bits getReadOnlyLiveDocs() {
     //System.out.println("getROLiveDocs seg=" + info);
     assert Thread.holdsLock(writer);
-    shared = true;
+    liveDocsShared = true;
     //if (liveDocs != null) {
     //System.out.println("  liveCount=" + liveDocs.count());
     //}
@@ -222,61 +237,288 @@ class ReadersAndLiveDocs {
     // deletes onto the newly merged segment, so we can
     // discard them on the sub-readers:
     pendingDeleteCount = 0;
+    dropMergingUpdates();
   }
 
-  // Commit live docs to the directory (writes new
-  // _X_N.del files); returns true if it wrote the file
-  // and false if there were no new deletes to write:
+  // Commit live docs (writes new _X_N.del files) and field updates (writes new
+  // _X_N updates files) to the directory; returns true if it wrote any file
+  // and false if there were no new deletes or updates to write:
+  // TODO (DVU_RENAME) to writeDeletesAndUpdates
   public synchronized boolean writeLiveDocs(Directory dir) throws IOException {
-    //System.out.println("rld.writeLiveDocs seg=" + info + " pendingDelCount=" + pendingDeleteCount);
-    if (pendingDeleteCount != 0) {
-      // We have new deletes
-      assert liveDocs.length() == info.info.getDocCount();
-
-      // Do this so we can delete any created files on
-      // exception; this saves all codecs from having to do
-      // it:
-      TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
-
-      // We can write directly to the actual name (vs to a
-      // .tmp & renaming it) because the file is not live
-      // until segments file is written:
-      boolean success = false;
+    assert Thread.holdsLock(writer);
+    //System.out.println("rld.writeLiveDocs seg=" + info + " pendingDelCount=" + pendingDeleteCount + " numericUpdates=" + numericUpdates);
+    if (pendingDeleteCount == 0) {
+      return false;
+    }
+    
+    // We have new deletes
+    assert liveDocs.length() == info.info.getDocCount();
+    
+    // Do this so we can delete any created files on
+    // exception; this saves all codecs from having to do
+    // it:
+    TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+    
+    // We can write directly to the actual name (vs to a
+    // .tmp & renaming it) because the file is not live
+    // until segments file is written:
+    boolean success = false;
+    try {
+      Codec codec = info.info.getCodec();
+      codec.liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
+      success = true;
+    } finally {
+      if (!success) {
+        // Advance only the nextWriteDelGen so that a 2nd
+        // attempt to write will write to a new file
+        info.advanceNextWriteDelGen();
+        
+        // Delete any partially created file(s):
+        for (String fileName : trackingDir.getCreatedFiles()) {
+          try {
+            dir.deleteFile(fileName);
+          } catch (Throwable t) {
+            // Ignore so we throw only the first exc
+          }
+        }
+      }
+    }
+    
+    // If we hit an exc in the line above (eg disk full)
+    // then info's delGen remains pointing to the previous
+    // (successfully written) del docs:
+    info.advanceDelGen();
+    info.setDelCount(info.getDelCount() + pendingDeleteCount);
+    pendingDeleteCount = 0;
+    
+    return true;
+  }
+
+  // Writes field updates (new _X_N updates files) to the directory
+  public synchronized void writeFieldUpdates(Directory dir, Map<String,NumericFieldUpdates> numericFieldUpdates) throws IOException {
+    assert Thread.holdsLock(writer);
+    //System.out.println("rld.writeFieldUpdates: seg=" + info + " numericFieldUpdates=" + numericFieldUpdates);
+    
+    assert numericFieldUpdates != null && !numericFieldUpdates.isEmpty();
+    
+    // Do this so we can delete any created files on
+    // exception; this saves all codecs from having to do
+    // it:
+    TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+    
+    FieldInfos fieldInfos = null;
+    boolean success = false;
+    try {
+      final Codec codec = info.info.getCodec();
+
+      // reader could be null e.g. for a just merged segment (from
+      // IndexWriter.commitMergedDeletes).
+      final SegmentReader reader = this.reader == null ? new SegmentReader(info, IOContext.READONCE) : this.reader;
       try {
-        info.info.getCodec().liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
-        success = true;
-      } finally {
-        if (!success) {
-          // Advance only the nextWriteDelGen so that a 2nd
-          // attempt to write will write to a new file
-          info.advanceNextWriteDelGen();
-
-          // Delete any partially created file(s):
-          for(String fileName : trackingDir.getCreatedFiles()) {
-            try {
-              dir.deleteFile(fileName);
-            } catch (Throwable t) {
-              // Ignore so we throw only the first exc
+        // clone FieldInfos so that we can update their dvGen separately from
+        // the reader's infos and write them to a new fieldInfos_gen file
+        FieldInfos.Builder builder = new FieldInfos.Builder(writer.globalFieldNumberMap);
+        // cannot use builder.add(reader.getFieldInfos()) because it does not
+        // clone FI.attributes as well FI.dvGen
+        for (FieldInfo fi : reader.getFieldInfos()) {
+          FieldInfo clone = builder.add(fi);
+          // copy the stuff FieldInfos.Builder doesn't copy
+          if (fi.attributes() != null) {
+            for (Entry<String,String> e : fi.attributes().entrySet()) {
+              clone.putAttribute(e.getKey(), e.getValue());
             }
           }
+          clone.setDocValuesGen(fi.getDocValuesGen());
+        }
+        // create new fields or update existing ones to have NumericDV type
+        for (String f : numericFieldUpdates.keySet()) {
+          builder.addOrUpdate(f, NumericDocValuesField.TYPE);
+        }
+        
+        fieldInfos = builder.finish();
+        final long nextFieldInfosGen = info.getNextFieldInfosGen();
+        final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
+        final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, IOContext.DEFAULT, segmentSuffix);
+        final DocValuesFormat docValuesFormat = codec.docValuesFormat();
+        final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
+        boolean fieldsConsumerSuccess = false;
+        try {
+//          System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: applying updates; seg=" + info + " updates=" + numericUpdates);
+          for (Entry<String,NumericFieldUpdates> e : numericFieldUpdates.entrySet()) {
+            final String field = e.getKey();
+            final NumericFieldUpdates fieldUpdates = e.getValue();
+            final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+            assert fieldInfo != null;
+
+            fieldInfo.setDocValuesGen(nextFieldInfosGen);
+            // write the numeric updates to a new gen'd docvalues file
+            fieldsConsumer.addNumericField(fieldInfo, new Iterable<Number>() {
+              final NumericDocValues currentValues = reader.getNumericDocValues(field);
+              final Bits docsWithField = reader.getDocsWithField(field);
+              final int maxDoc = reader.maxDoc();
+              final UpdatesIterator updatesIter = fieldUpdates.getUpdates();
+              @Override
+              public Iterator<Number> iterator() {
+                updatesIter.reset();
+                return new Iterator<Number>() {
+
+                  int curDoc = -1;
+                  int updateDoc = updatesIter.nextDoc();
+                  
+                  @Override
+                  public boolean hasNext() {
+                    return curDoc < maxDoc - 1;
+                  }
+
+                  @Override
+                  public Number next() {
+                    if (++curDoc >= maxDoc) {
+                      throw new NoSuchElementException("no more documents to return values for");
+                    }
+                    if (curDoc == updateDoc) { // this document has an updated value
+                      Long value = updatesIter.value(); // either null (unset value) or updated value
+                      updateDoc = updatesIter.nextDoc(); // prepare for next round
+                      return value;
+                    } else {
+                      // no update for this document
+                      assert curDoc < updateDoc;
+                      if (currentValues != null && docsWithField.get(curDoc)) {
+                        // only read the current value if the document had a value before
+                        return currentValues.get(curDoc);
+                      } else {
+                        return null;
+                      }
+                    }
+                  }
+
+                  @Override
+                  public void remove() {
+                    throw new UnsupportedOperationException("this iterator does not support removing elements");
+                  }
+                };
+              }
+            });
+          }
+          
+          codec.fieldInfosFormat().getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, IOContext.DEFAULT);
+          fieldsConsumerSuccess = true;
+        } finally {
+          if (fieldsConsumerSuccess) {
+            fieldsConsumer.close();
+          } else {
+            IOUtils.closeWhileHandlingException(fieldsConsumer);
+          }
+        }
+      } finally {
+        if (reader != this.reader) {
+//          System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: closeReader " + reader);
+          reader.close();
         }
       }
+    
+      success = true;
+    } finally {
+      if (!success) {
+        // Advance only the nextWriteDocValuesGen so that a 2nd
+        // attempt to write will write to a new file
+        info.advanceNextWriteFieldInfosGen();
+        
+        // Delete any partially created file(s):
+        for (String fileName : trackingDir.getCreatedFiles()) {
+          try {
+            dir.deleteFile(fileName);
+          } catch (Throwable t) {
+            // Ignore so we throw only the first exc
+          }
+        }
+      }
+    }
+    
+    info.advanceFieldInfosGen();
+    // copy all the updates to mergingUpdates, so they can later be applied to the merged segment
+    if (isMerging) {
+      for (Entry<String,NumericFieldUpdates> e : numericFieldUpdates.entrySet()) {
+        NumericFieldUpdates fieldUpdates = mergingNumericUpdates.get(e.getKey());
+        if (fieldUpdates == null) {
+          mergingNumericUpdates.put(e.getKey(), e.getValue());
+        } else {
+          fieldUpdates.merge(e.getValue());
+        }
+      }
+    }
+    
+    // create a new map, keeping only the gens that are in use
+    Map<Long,Set<String>> genUpdatesFiles = info.getUpdatesFiles();
+    Map<Long,Set<String>> newGenUpdatesFiles = new HashMap<Long,Set<String>>();
+    final long fieldInfosGen = info.getFieldInfosGen();
+    for (FieldInfo fi : fieldInfos) {
+      long dvGen = fi.getDocValuesGen();
+      if (dvGen != -1 && !newGenUpdatesFiles.containsKey(dvGen)) {
+        if (dvGen == fieldInfosGen) {
+          newGenUpdatesFiles.put(fieldInfosGen, trackingDir.getCreatedFiles());
+        } else {
+          newGenUpdatesFiles.put(dvGen, genUpdatesFiles.get(dvGen));
+        }
+      }
+    }
+    
+    info.setGenUpdatesFiles(newGenUpdatesFiles);
+    
+    // wrote new files, should checkpoint()
+    writer.checkpoint();
 
-      // If we hit an exc in the line above (eg disk full)
-      // then info's delGen remains pointing to the previous
-      // (successfully written) del docs:
-      info.advanceDelGen();
-      info.setDelCount(info.getDelCount() + pendingDeleteCount);
-
-      pendingDeleteCount = 0;
-      return true;
-    } else {
-      return false;
+    // if there is a reader open, reopen it to reflect the updates
+    if (reader != null) {
+      SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
+      boolean reopened = false;
+      try {
+        reader.decRef();
+        reader = newReader;
+        reopened = true;
+      } finally {
+        if (!reopened) {
+          newReader.decRef();
+        }
+      }
     }
   }
 
+  /**
+   * Returns a reader for merge. This method applies field updates if there are
+   * any and marks that this segment is currently merging.
+   */
+  synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException {
+    assert Thread.holdsLock(writer);
+    // must execute these two statements as atomic operation, otherwise we
+    // could lose updates if e.g. another thread calls writeFieldUpdates in
+    // between, or the updates are applied to the obtained reader, but then
+    // re-applied in IW.commitMergedDeletes (unnecessary work and potential
+    // bugs).
+    isMerging = true;
+    return getReader(context);
+  }
+  
+  /**
+   * Drops all merging updates. Called from IndexWriter after this segment
+   * finished merging (whether successfully or not).
+   */
+  public synchronized void dropMergingUpdates() {
+    mergingNumericUpdates.clear();
+    isMerging = false;
+  }
+  
+  /** Returns updates that came in while this segment was merging. */
+  public synchronized Map<String,NumericFieldUpdates> getMergingFieldUpdates() {
+    return mergingNumericUpdates;
+  }
+  
   @Override
   public String toString() {
-    return "ReadersAndLiveDocs(seg=" + info + " pendingDeleteCount=" + pendingDeleteCount + " shared=" + shared + ")";
+    StringBuilder sb = new StringBuilder();
+    sb.append("ReadersAndLiveDocs(seg=").append(info);
+    sb.append(" pendingDeleteCount=").append(pendingDeleteCount);
+    sb.append(" liveDocsShared=").append(liveDocsShared);
+    return sb.toString();
   }
+  
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Mon Oct 21 18:58:24 2013
@@ -26,12 +26,11 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.SegmentReader.CoreClosedListener;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
@@ -43,29 +42,24 @@ import org.apache.lucene.util.IOUtils;
  * SegmentReader is cloned or reopened */
 final class SegmentCoreReaders {
   
-  // Counts how many other reader share the core objects
+  // Counts how many other readers share the core objects
   // (freqStream, proxStream, tis, etc.) of this reader;
   // when coreRef drops to 0, these core objects may be
   // closed.  A given instance of SegmentReader may be
-  // closed, even those it shares core objects with other
+  // closed, even though it shares core objects with other
   // SegmentReaders:
   private final AtomicInteger ref = new AtomicInteger(1);
   
-  final FieldInfos fieldInfos;
-  
   final FieldsProducer fields;
-  final DocValuesProducer dvProducer;
   final DocValuesProducer normsProducer;
 
-  private final SegmentReader owner;
-  
   final StoredFieldsReader fieldsReaderOrig;
   final TermVectorsReader termVectorsReaderOrig;
   final CompoundFileDirectory cfsReader;
 
   // TODO: make a single thread local w/ a
   // Thingy class holding fieldsReader, termVectorsReader,
-  // normsProducer, dvProducer
+  // normsProducer
 
   final CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new CloseableThreadLocal<StoredFieldsReader>() {
     @Override
@@ -81,13 +75,6 @@ final class SegmentCoreReaders {
     }
   };
 
-  final CloseableThreadLocal<Map<String,Object>> docValuesLocal = new CloseableThreadLocal<Map<String,Object>>() {
-    @Override
-    protected Map<String,Object> initialValue() {
-      return new HashMap<String,Object>();
-    }
-  };
-
   final CloseableThreadLocal<Map<String,Object>> normsLocal = new CloseableThreadLocal<Map<String,Object>>() {
     @Override
     protected Map<String,Object> initialValue() {
@@ -99,7 +86,7 @@ final class SegmentCoreReaders {
       Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
   
   SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context) throws IOException {
-    
+
     final Codec codec = si.info.getCodec();
     final Directory cfsDir; // confusing name: if (cfs) its the cfsdir, otherwise its the segment's directory.
 
@@ -112,10 +99,11 @@ final class SegmentCoreReaders {
         cfsReader = null;
         cfsDir = dir;
       }
-      fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE);
 
-      final PostingsFormat format = codec.postingsFormat();
+      final FieldInfos fieldInfos = owner.fieldInfos;
+      
       final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context);
+      final PostingsFormat format = codec.postingsFormat();
       // Ask codec for its Fields
       fields = format.fieldsProducer(segmentReadState);
       assert fields != null;
@@ -123,13 +111,6 @@ final class SegmentCoreReaders {
       // TODO: since we don't write any norms file if there are no norms,
       // kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
 
-      if (fieldInfos.hasDocValues()) {
-        dvProducer = codec.docValuesFormat().fieldsProducer(segmentReadState);
-        assert dvProducer != null;
-      } else {
-        dvProducer = null;
-      }
-
       if (fieldInfos.hasNorms()) {
         normsProducer = codec.normsFormat().normsProducer(segmentReadState);
         assert normsProducer != null;
@@ -151,148 +132,25 @@ final class SegmentCoreReaders {
         decRef();
       }
     }
-    
-    // Must assign this at the end -- if we hit an
-    // exception above core, we don't want to attempt to
-    // purge the FieldCache (will hit NPE because core is
-    // not assigned yet).
-    this.owner = owner;
   }
   
-  void incRef() {
-    ref.incrementAndGet();
-  }
-
-  NumericDocValues getNumericDocValues(String field) throws IOException {
-    FieldInfo fi = fieldInfos.fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() == null) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.NUMERIC) {
-      // DocValues were not numeric
-      return null;
-    }
-
-    assert dvProducer != null;
-
-    Map<String,Object> dvFields = docValuesLocal.get();
-
-    NumericDocValues dvs = (NumericDocValues) dvFields.get(field);
-    if (dvs == null) {
-      dvs = dvProducer.getNumeric(fi);
-      dvFields.put(field, dvs);
-    }
-
-    return dvs;
-  }
-
-  BinaryDocValues getBinaryDocValues(String field) throws IOException {
-    FieldInfo fi = fieldInfos.fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() == null) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.BINARY) {
-      // DocValues were not binary
-      return null;
-    }
-
-    assert dvProducer != null;
-
-    Map<String,Object> dvFields = docValuesLocal.get();
-
-    BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
-    if (dvs == null) {
-      dvs = dvProducer.getBinary(fi);
-      dvFields.put(field, dvs);
-    }
-
-    return dvs;
-  }
-
-  SortedDocValues getSortedDocValues(String field) throws IOException {
-    FieldInfo fi = fieldInfos.fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() == null) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.SORTED) {
-      // DocValues were not sorted
-      return null;
-    }
-
-    assert dvProducer != null;
-
-    Map<String,Object> dvFields = docValuesLocal.get();
-
-    SortedDocValues dvs = (SortedDocValues) dvFields.get(field);
-    if (dvs == null) {
-      dvs = dvProducer.getSorted(fi);
-      dvFields.put(field, dvs);
-    }
-
-    return dvs;
+  int getRefCount() {
+    return ref.get();
   }
   
-  SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
-    FieldInfo fi = fieldInfos.fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() == null) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.SORTED_SET) {
-      // DocValues were not sorted
-      return null;
-    }
-
-    assert dvProducer != null;
-
-    Map<String,Object> dvFields = docValuesLocal.get();
-
-    SortedSetDocValues dvs = (SortedSetDocValues) dvFields.get(field);
-    if (dvs == null) {
-      dvs = dvProducer.getSortedSet(fi);
-      dvFields.put(field, dvs);
-    }
-
-    return dvs;
+  void incRef() {
+    ref.incrementAndGet();
   }
 
-  NumericDocValues getNormValues(String field) throws IOException {
-    FieldInfo fi = fieldInfos.fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (!fi.hasNorms()) {
-      return null;
-    }
-   
+  NumericDocValues getNormValues(FieldInfo fi) throws IOException {
     assert normsProducer != null;
 
     Map<String,Object> normFields = normsLocal.get();
 
-    NumericDocValues norms = (NumericDocValues) normFields.get(field);
+    NumericDocValues norms = (NumericDocValues) normFields.get(fi.name);
     if (norms == null) {
       norms = normsProducer.getNumeric(fi);
-      normFields.put(field, norms);
+      normFields.put(fi.name, norms);
     }
 
     return norms;
@@ -300,8 +158,9 @@ final class SegmentCoreReaders {
 
   void decRef() throws IOException {
     if (ref.decrementAndGet() == 0) {
-      IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, fields, dvProducer,
-                    termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
+//      System.err.println("--- closing core readers");
+      IOUtils.close(termVectorsLocal, fieldsReaderLocal, normsLocal, fields, termVectorsReaderOrig, fieldsReaderOrig, 
+          cfsReader, normsProducer);
       notifyCoreClosedListeners();
     }
   }
@@ -309,7 +168,9 @@ final class SegmentCoreReaders {
   private void notifyCoreClosedListeners() {
     synchronized(coreClosedListeners) {
       for (CoreClosedListener listener : coreClosedListeners) {
-        listener.onClose(owner);
+        // SegmentReader uses our instance as its
+        // coreCacheKey:
+        listener.onClose(this);
       }
     }
   }
@@ -322,8 +183,11 @@ final class SegmentCoreReaders {
     coreClosedListeners.remove(listener);
   }
 
-  @Override
-  public String toString() {
-    return "SegmentCoreReader(owner=" + owner + ")";
+  /** Returns approximate RAM bytes used */
+  public long ramBytesUsed() {
+    return ((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
+        ((fields!=null) ? fields.ramBytesUsed() : 0) + 
+        ((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) + 
+        ((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Mon Oct 21 18:58:24 2013
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import java.util.regex.Matcher;
@@ -60,8 +59,6 @@ public final class SegmentInfo {
 
   private Map<String,String> diagnostics;
   
-  private Map<String,String> attributes;
-
   // Tracks the Lucene version this segment was created with, since 3.1. Null
   // indicates an older than 3.0 index, and it's used to detect a too old index.
   // The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and
@@ -85,7 +82,7 @@ public final class SegmentInfo {
    * the codecs package.</p>
    */
   public SegmentInfo(Directory dir, String version, String name, int docCount, 
-                     boolean isCompoundFile, Codec codec, Map<String,String> diagnostics, Map<String,String> attributes) {
+                     boolean isCompoundFile, Codec codec, Map<String,String> diagnostics) {
     assert !(dir instanceof TrackingDirectoryWrapper);
     this.dir = dir;
     this.version = version;
@@ -94,7 +91,6 @@ public final class SegmentInfo {
     this.isCompoundFile = isCompoundFile;
     this.codec = codec;
     this.diagnostics = diagnostics;
-    this.attributes = attributes;
   }
 
   /**
@@ -119,7 +115,7 @@ public final class SegmentInfo {
   public void setCodec(Codec codec) {
     assert this.codec == null;
     if (codec == null) {
-      throw new IllegalArgumentException("segmentCodecs must be non-null");
+      throw new IllegalArgumentException("codec must be non-null");
     }
     this.codec = codec;
   }
@@ -170,7 +166,6 @@ public final class SegmentInfo {
    *  left off when there are no deletions).</p>
    */
   public String toString(Directory dir, int delCount) {
-
     StringBuilder s = new StringBuilder();
     s.append(name).append('(').append(version == null ? "?" : version).append(')').append(':');
     char cfs = getUseCompoundFile() ? 'c' : 'C';
@@ -260,40 +255,4 @@ public final class SegmentInfo {
     }
   }
     
-  /**
-   * Get a codec attribute value, or null if it does not exist
-   */
-  public String getAttribute(String key) {
-    if (attributes == null) {
-      return null;
-    } else {
-      return attributes.get(key);
-    }
-  }
-  
-  /**
-   * Puts a codec attribute value.
-   * <p>
-   * This is a key-value mapping for the field that the codec can use
-   * to store additional metadata, and will be available to the codec
-   * when reading the segment via {@link #getAttribute(String)}
-   * <p>
-   * If a value already exists for the field, it will be replaced with 
-   * the new value.
-   */
-  public String putAttribute(String key, String value) {
-    if (attributes == null) {
-      attributes = new HashMap<String,String>();
-    }
-    return attributes.put(key, value);
-  }
-  
-  /**
-   * Returns the internal codec attributes map.
-   *
-   * @return internal codec attributes map. May be null if no mappings exist.
-   */
-  public Map<String,String> attributes() {
-    return attributes;
-  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java Mon Oct 21 18:58:24 2013
@@ -19,7 +19,12 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.lucene.store.Directory;
 
@@ -27,9 +32,8 @@ import org.apache.lucene.store.Directory
  *  fields.
  *
  *  @lucene.experimental */
-
-public class SegmentInfoPerCommit {
-
+public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo
+  
   /** The {@link SegmentInfo} that we wrap. */
   public final SegmentInfo info;
 
@@ -44,15 +48,31 @@ public class SegmentInfoPerCommit {
   // attempt to write:
   private long nextWriteDelGen;
 
+  // Generation number of the FieldInfos (-1 if there are no updates)
+  private long fieldInfosGen;
+  
+  // Normally 1 + fieldInfosGen, unless an exception was hit on last attempt to
+  // write
+  private long nextWriteFieldInfosGen;
+
+  // Track the per-generation updates files
+  private final Map<Long,Set<String>> genUpdatesFiles = new HashMap<Long,Set<String>>();
+  
   private volatile long sizeInBytes = -1;
 
-  /** Sole constructor.
-   * @param info {@link SegmentInfo} that we wrap
-   * @param delCount number of deleted documents in this segment
-   * @param delGen deletion generation number (used to name
-             deletion files)
+  /**
+   * Sole constructor.
+   * 
+   * @param info
+   *          {@link SegmentInfo} that we wrap
+   * @param delCount
+   *          number of deleted documents in this segment
+   * @param delGen
+   *          deletion generation number (used to name deletion files)
+   * @param fieldInfosGen
+   *          FieldInfos generation number (used to name field-infos files)
    **/
-  public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen) {
+  public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen, long fieldInfosGen) {
     this.info = info;
     this.delCount = delCount;
     this.delGen = delGen;
@@ -61,8 +81,26 @@ public class SegmentInfoPerCommit {
     } else {
       nextWriteDelGen = delGen+1;
     }
+    
+    this.fieldInfosGen = fieldInfosGen;
+    if (fieldInfosGen == -1) {
+      nextWriteFieldInfosGen = 1;
+    } else {
+      nextWriteFieldInfosGen = fieldInfosGen + 1;
+    }
   }
 
+  /** Returns the per generation updates files. */
+  public Map<Long,Set<String>> getUpdatesFiles() {
+    return Collections.unmodifiableMap(genUpdatesFiles);
+  }
+  
+  /** Sets the updates file names per generation. Does not deep clone the map. */
+  public void setGenUpdatesFiles(Map<Long,Set<String>> genUpdatesFiles) {
+    this.genUpdatesFiles.clear();
+    this.genUpdatesFiles.putAll(genUpdatesFiles);
+  }
+  
   /** Called when we succeed in writing deletes */
   void advanceDelGen() {
     delGen = nextWriteDelGen;
@@ -76,6 +114,21 @@ public class SegmentInfoPerCommit {
   void advanceNextWriteDelGen() {
     nextWriteDelGen++;
   }
+  
+  /** Called when we succeed in writing a new FieldInfos generation. */
+  void advanceFieldInfosGen() {
+    fieldInfosGen = nextWriteFieldInfosGen;
+    nextWriteFieldInfosGen = fieldInfosGen + 1;
+    sizeInBytes = -1;
+  }
+  
+  /**
+   * Called if there was an exception while writing a new generation of
+   * FieldInfos, so that we don't try to write to the same file more than once.
+   */
+  void advanceNextWriteFieldInfosGen() {
+    nextWriteFieldInfosGen++;
+  }
 
   /** Returns total size in bytes of all files for this
    *  segment. */
@@ -96,9 +149,17 @@ public class SegmentInfoPerCommit {
     // Start from the wrapped info's files:
     Collection<String> files = new HashSet<String>(info.files());
 
+    // TODO we could rely on TrackingDir.getCreatedFiles() (like we do for
+    // updates) and then maybe even be able to remove LiveDocsFormat.files().
+    
     // Must separately add any live docs files:
     info.getCodec().liveDocsFormat().files(this, files);
 
+    // Must separately add any field updates files
+    for (Set<String> updateFiles : genUpdatesFiles.values()) {
+      files.addAll(updateFiles);
+    }
+    
     return files;
   }
 
@@ -115,26 +176,30 @@ public class SegmentInfoPerCommit {
     sizeInBytes =  -1;
   }
   
-  void clearDelGen() {
-    delGen = -1;
-    sizeInBytes =  -1;
-  }
-
-  /**
-   * Sets the generation number of the live docs file.
-   * @see #getDelGen()
-   */
-  public void setDelGen(long delGen) {
-    this.delGen = delGen;
-    sizeInBytes =  -1;
-  }
-
   /** Returns true if there are any deletions for the 
    * segment at this commit. */
   public boolean hasDeletions() {
     return delGen != -1;
   }
 
+  /** Returns true if there are any field updates for the segment in this commit. */
+  public boolean hasFieldUpdates() {
+    return fieldInfosGen != -1;
+  }
+  
+  /** Returns the next available generation number of the FieldInfos files. */
+  public long getNextFieldInfosGen() {
+    return nextWriteFieldInfosGen;
+  }
+  
+  /**
+   * Returns the generation number of the field infos file or -1 if there are no
+   * field updates yet.
+   */
+  public long getFieldInfosGen() {
+    return fieldInfosGen;
+  }
+  
   /**
    * Returns the next available generation number
    * of the live docs file.
@@ -165,26 +230,36 @@ public class SegmentInfoPerCommit {
 
   /** Returns a description of this segment. */
   public String toString(Directory dir, int pendingDelCount) {
-    return info.toString(dir, delCount + pendingDelCount);
-  }
-
-  @Override
-  public String toString() {
-    String s = info.toString(info.dir, delCount);
+    String s = info.toString(dir, delCount + pendingDelCount);
     if (delGen != -1) {
       s += ":delGen=" + delGen;
     }
+    if (fieldInfosGen != -1) {
+      s += ":fieldInfosGen=" + fieldInfosGen;
+    }
     return s;
   }
 
   @Override
+  public String toString() {
+    return toString(info.dir, 0);
+  }
+
+  @Override
   public SegmentInfoPerCommit clone() {
-    SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen);
+    SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen);
     // Not clear that we need to carry over nextWriteDelGen
     // (i.e. do we ever clone after a failed write and
     // before the next successful write?), but just do it to
     // be safe:
     other.nextWriteDelGen = nextWriteDelGen;
+    other.nextWriteFieldInfosGen = nextWriteFieldInfosGen;
+    
+    // deep clone
+    for (Entry<Long,Set<String>> e : genUpdatesFiles.entrySet()) {
+      other.genUpdatesFiles.put(e.getKey(), new HashSet<String>(e.getValue()));
+    }
+    
     return other;
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Mon Oct 21 18:58:24 2013
@@ -28,21 +28,22 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.ChecksumIndexOutput;
-import org.apache.lucene.store.DataOutput; // javadocs
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.NoSuchDirectoryException;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * A collection of segmentInfo objects with methods for operating on
@@ -70,7 +71,7 @@ import org.apache.lucene.util.ThreadInte
  * <ul>
  *   <li><tt>segments.gen</tt>: GenHeader, Generation, Generation
  *   <li><tt>segments_N</tt>: Header, Version, NameCounter, SegCount,
- *    &lt;SegName, SegCodec, DelGen, DeletionCount&gt;<sup>SegCount</sup>, 
+ *    &lt;SegName, SegCodec, DelGen, DeletionCount, FieldInfosGen, UpdatesFiles&gt;<sup>SegCount</sup>, 
  *    CommitUserData, Checksum
  * </ul>
  * </p>
@@ -79,9 +80,10 @@ import org.apache.lucene.util.ThreadInte
  * <ul>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>GenHeader, NameCounter, SegCount, DeletionCount --&gt; {@link DataOutput#writeInt Int32}</li>
- *   <li>Generation, Version, DelGen, Checksum --&gt; {@link DataOutput#writeLong Int64}</li>
+ *   <li>Generation, Version, DelGen, Checksum, FieldInfosGen --&gt; {@link DataOutput#writeLong Int64}</li>
  *   <li>SegName, SegCodec --&gt; {@link DataOutput#writeString String}</li>
  *   <li>CommitUserData --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
+ *   <li>UpdatesFiles --&gt; {@link DataOutput#writeStringSet(Set) Set&lt;String&gt;}</li>
  * </ul>
  * </p>
  * Field Descriptions:
@@ -104,6 +106,10 @@ import org.apache.lucene.util.ThreadInte
  *   <li>CommitUserData stores an optional user-supplied opaque
  *       Map&lt;String,String&gt; that was passed to 
  *       {@link IndexWriter#setCommitData(java.util.Map)}.</li>
+ *   <li>FieldInfosGen is the generation count of the fieldInfos file. If this is -1,
+ *       there are no updates to the fieldInfos in that segment. Anything above zero 
+ *       means there are updates to fieldInfos stored by {@link FieldInfosFormat}.</li>
+ *   <li>UpdatesFiles stores the list of files that were updated in that segment.</li>
  * </ul>
  * </p>
  * 
@@ -111,11 +117,12 @@ import org.apache.lucene.util.ThreadInte
  */
 public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCommit> {
 
-  /**
-   * The file format version for the segments_N codec header
-   */
+  /** The file format version for the segments_N codec header, up to 4.5. */
   public static final int VERSION_40 = 0;
 
+  /** The file format version for the segments_N codec header, since 4.6+. */
+  public static final int VERSION_46 = 1;
+
   /** Used for the segments.gen file only!
    * Whenever you add a new format, make it 1 smaller (negative version logic)! */
   public static final int FORMAT_SEGMENTS_GEN_CURRENT = -2;
@@ -319,14 +326,14 @@ public final class SegmentInfos implemen
         throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
       }
       // 4.0+
-      CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_40);
+      int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_46);
       version = input.readLong();
       counter = input.readInt();
       int numSegments = input.readInt();
       if (numSegments < 0) {
         throw new CorruptIndexException("invalid segment count: " + numSegments + " (resource: " + input + ")");
       }
-      for(int seg=0;seg<numSegments;seg++) {
+      for (int seg = 0; seg < numSegments; seg++) {
         String segName = input.readString();
         Codec codec = Codec.forName(input.readString());
         //System.out.println("SIS.read seg=" + seg + " codec=" + codec);
@@ -337,7 +344,25 @@ public final class SegmentInfos implemen
         if (delCount < 0 || delCount > info.getDocCount()) {
           throw new CorruptIndexException("invalid deletion count: " + delCount + " (resource: " + input + ")");
         }
-        add(new SegmentInfoPerCommit(info, delCount, delGen));
+        long fieldInfosGen = -1;
+        if (format >= VERSION_46) {
+          fieldInfosGen = input.readLong();
+        }
+        SegmentInfoPerCommit siPerCommit = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen);
+        if (format >= VERSION_46) {
+          int numGensUpdatesFiles = input.readInt();
+          final Map<Long,Set<String>> genUpdatesFiles;
+          if (numGensUpdatesFiles == 0) {
+            genUpdatesFiles = Collections.emptyMap();
+          } else {
+            genUpdatesFiles = new HashMap<Long,Set<String>>(numGensUpdatesFiles);
+            for (int i = 0; i < numGensUpdatesFiles; i++) {
+              genUpdatesFiles.put(input.readLong(), input.readStringSet());
+            }
+          }
+          siPerCommit.setGenUpdatesFiles(genUpdatesFiles);
+        }
+        add(siPerCommit);
       }
       userData = input.readStringStringMap();
 
@@ -395,7 +420,7 @@ public final class SegmentInfos implemen
 
     try {
       segnOutput = new ChecksumIndexOutput(directory.createOutput(segmentFileName, IOContext.DEFAULT));
-      CodecUtil.writeHeader(segnOutput, "segments", VERSION_40);
+      CodecUtil.writeHeader(segnOutput, "segments", VERSION_46);
       segnOutput.writeLong(version); 
       segnOutput.writeInt(counter); // write counter
       segnOutput.writeInt(size()); // write infos
@@ -405,6 +430,13 @@ public final class SegmentInfos implemen
         segnOutput.writeString(si.getCodec().getName());
         segnOutput.writeLong(siPerCommit.getDelGen());
         segnOutput.writeInt(siPerCommit.getDelCount());
+        segnOutput.writeLong(siPerCommit.getFieldInfosGen());
+        final Map<Long,Set<String>> genUpdatesFiles = siPerCommit.getUpdatesFiles();
+        segnOutput.writeInt(genUpdatesFiles.size());
+        for (Entry<Long,Set<String>> e : genUpdatesFiles.entrySet()) {
+          segnOutput.writeLong(e.getKey());
+          segnOutput.writeStringSet(e.getValue());
+        }
         assert si.dir == directory;
 
         assert siPerCommit.getDelCount() <= si.getDocCount();
@@ -804,10 +836,6 @@ public final class SegmentInfos implemen
     if (includeSegmentsFile) {
       final String segmentFileName = getSegmentsFileName();
       if (segmentFileName != null) {
-        /*
-         * TODO: if lastGen == -1 we get might get null here it seems wrong to
-         * add null to the files set
-         */
         files.add(segmentFileName);
       }
     }
@@ -819,6 +847,7 @@ public final class SegmentInfos implemen
         files.addAll(info.files());
       }
     }
+    
     return files;
   }
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Mon Oct 21 18:58:24 2013
@@ -22,14 +22,14 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.FieldInfosWriter;
-import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
 
@@ -52,12 +52,18 @@ final class SegmentMerger {
 
   // note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
   SegmentMerger(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
-                MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) {
+                MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) throws IOException {
     mergeState = new MergeState(readers, segmentInfo, infoStream, checkAbort);
     directory = dir;
     this.codec = segmentInfo.getCodec();
     this.context = context;
     this.fieldInfosBuilder = new FieldInfos.Builder(fieldNumbers);
+    mergeState.segmentInfo.setDocCount(setDocMaps());
+  }
+  
+  /** True if any merging should happen */
+  boolean shouldMerge() {
+    return mergeState.segmentInfo.getDocCount() > 0;
   }
 
   /**
@@ -67,14 +73,15 @@ final class SegmentMerger {
    * @throws IOException if there is a low-level IO error
    */
   MergeState merge() throws IOException {
+    if (!shouldMerge()) {
+      throw new IllegalStateException("Merge would result in 0 document segment");
+    }
     // NOTE: it's important to add calls to
     // checkAbort.work(...) if you make any changes to this
     // method that will spend alot of time.  The frequency
     // of this check impacts how long
     // IndexWriter.close(false) takes to actually stop the
     // threads.
-    
-    mergeState.segmentInfo.setDocCount(setDocMaps());
     mergeFieldInfos();
     setMatchingSegmentReaders();
     long t0 = 0;
@@ -135,7 +142,7 @@ final class SegmentMerger {
     
     // write the merged infos
     FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
-    fieldInfosWriter.write(directory, mergeState.segmentInfo.name, mergeState.fieldInfos, context);
+    fieldInfosWriter.write(directory, mergeState.segmentInfo.name, "", mergeState.fieldInfos, context);
 
     return mergeState;
   }
@@ -149,24 +156,32 @@ final class SegmentMerger {
         if (type != null) {
           if (type == DocValuesType.NUMERIC) {
             List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
+            List<Bits> docsWithField = new ArrayList<Bits>();
             for (AtomicReader reader : mergeState.readers) {
               NumericDocValues values = reader.getNumericDocValues(field.name);
+              Bits bits = reader.getDocsWithField(field.name);
               if (values == null) {
                 values = NumericDocValues.EMPTY;
+                bits = new Bits.MatchNoBits(reader.maxDoc());
               }
               toMerge.add(values);
+              docsWithField.add(bits);
             }
-            consumer.mergeNumericField(field, mergeState, toMerge);
+            consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
           } else if (type == DocValuesType.BINARY) {
             List<BinaryDocValues> toMerge = new ArrayList<BinaryDocValues>();
+            List<Bits> docsWithField = new ArrayList<Bits>();
             for (AtomicReader reader : mergeState.readers) {
               BinaryDocValues values = reader.getBinaryDocValues(field.name);
+              Bits bits = reader.getDocsWithField(field.name);
               if (values == null) {
                 values = BinaryDocValues.EMPTY;
+                bits = new Bits.MatchNoBits(reader.maxDoc());
               }
               toMerge.add(values);
+              docsWithField.add(bits);
             }
-            consumer.mergeBinaryField(field, mergeState, toMerge);
+            consumer.mergeBinaryField(field, mergeState, toMerge, docsWithField);
           } else if (type == DocValuesType.SORTED) {
             List<SortedDocValues> toMerge = new ArrayList<SortedDocValues>();
             for (AtomicReader reader : mergeState.readers) {
@@ -209,14 +224,16 @@ final class SegmentMerger {
       for (FieldInfo field : mergeState.fieldInfos) {
         if (field.hasNorms()) {
           List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
+          List<Bits> docsWithField = new ArrayList<Bits>();
           for (AtomicReader reader : mergeState.readers) {
             NumericDocValues norms = reader.getNormValues(field.name);
             if (norms == null) {
               norms = NumericDocValues.EMPTY;
             }
             toMerge.add(norms);
+            docsWithField.add(new Bits.MatchAllBits(reader.maxDoc()));
           }
-          consumer.mergeNumericField(field, mergeState, toMerge);
+          consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
         }
       }
       success = true;
@@ -357,19 +374,10 @@ final class SegmentMerger {
       docBase += maxDoc;
     }
 
-    final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
-    boolean success = false;
-    try {
-      consumer.merge(mergeState,
-                     new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
-                                     slices.toArray(ReaderSlice.EMPTY_ARRAY)));
-      success = true;
-    } finally {
-      if (success) {
-        IOUtils.close(consumer);
-      } else {
-        IOUtils.closeWhileHandlingException(consumer);
-      }
-    }
+    Fields mergedFields = new MappedMultiFields(mergeState, 
+                                                new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
+                                                                slices.toArray(ReaderSlice.EMPTY_ARRAY)));
+
+    codec.postingsFormat().fieldsConsumer(segmentWriteState).write(mergedFields);
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Mon Oct 21 18:58:24 2013
@@ -18,13 +18,26 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-
-import org.apache.lucene.store.Directory;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
-import org.apache.lucene.search.FieldCache; // javadocs
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.store.CompoundFileDirectory;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CloseableThreadLocal;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RefCount;
 
 /**
  * IndexReader implementation over a single segment. 
@@ -35,6 +48,18 @@ import org.apache.lucene.util.Bits;
  */
 public final class SegmentReader extends AtomicReader {
 
+  private static final class DocValuesRefCount extends RefCount<DocValuesProducer> {
+
+    public DocValuesRefCount(DocValuesProducer object) {
+      super(object);
+    }
+    
+    @Override
+    protected void release() throws IOException {
+      object.close();
+    }
+  }
+  
   private final SegmentInfoPerCommit si;
   private final Bits liveDocs;
 
@@ -45,6 +70,25 @@ public final class SegmentReader extends
 
   final SegmentCoreReaders core;
 
+  final CloseableThreadLocal<Map<String,Object>> docValuesLocal = new CloseableThreadLocal<Map<String,Object>>() {
+    @Override
+    protected Map<String,Object> initialValue() {
+      return new HashMap<String,Object>();
+    }
+  };
+
+  final CloseableThreadLocal<Map<String,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
+    @Override
+    protected Map<String,Bits> initialValue() {
+      return new HashMap<String,Bits>();
+    }
+  };
+
+  final Map<String,DocValuesProducer> dvProducers = new HashMap<String,DocValuesProducer>();
+  final Map<Long,RefCount<DocValuesProducer>> genDVProducers = new HashMap<Long,RefCount<DocValuesProducer>>();
+
+  final FieldInfos fieldInfos;
+  
   /**
    * Constructs a new SegmentReader with a new core.
    * @throws CorruptIndexException if the index is corrupt
@@ -53,17 +97,51 @@ public final class SegmentReader extends
   // TODO: why is this public?
   public SegmentReader(SegmentInfoPerCommit si, IOContext context) throws IOException {
     this.si = si;
+    // TODO if the segment uses CFS, we may open the CFS file twice: once for
+    // reading the FieldInfos (if they are not gen'd) and second time by
+    // SegmentCoreReaders. We can open the CFS here and pass to SCR, but then it
+    // results in less readable code (resource not closed where it was opened).
+    // Best if we could somehow read FieldInfos in SCR but not keep it there, but
+    // constructors don't allow returning two things...
+    fieldInfos = readFieldInfos(si);
     core = new SegmentCoreReaders(this, si.info.dir, si, context);
+
     boolean success = false;
+    final Codec codec = si.info.getCodec();
     try {
       if (si.hasDeletions()) {
         // NOTE: the bitvector is stored using the regular directory, not cfs
-        liveDocs = si.info.getCodec().liveDocsFormat().readLiveDocs(directory(), si, new IOContext(IOContext.READ, true));
+        liveDocs = codec.liveDocsFormat().readLiveDocs(directory(), si, IOContext.READONCE);
       } else {
         assert si.getDelCount() == 0;
         liveDocs = null;
       }
       numDocs = si.info.getDocCount() - si.getDelCount();
+      
+      if (fieldInfos.hasDocValues()) {
+        final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
+        final DocValuesFormat dvFormat = codec.docValuesFormat();
+        // initialize the per generation numericDVProducers and put the correct
+        // DVProducer for each field
+        final Map<Long,List<FieldInfo>> genInfos = getGenInfos();
+        
+//        System.out.println("[" + Thread.currentThread().getName() + "] SR.init: new reader: " + si + "; gens=" + genInfos.keySet());
+
+        for (Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
+          Long gen = e.getKey();
+          List<FieldInfo> infos = e.getValue();
+          RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
+          if (dvp == null) {
+            dvp = newDocValuesProducer(si, context, dir, dvFormat, gen, infos);
+            assert dvp != null;
+            genDVProducers.put(gen, dvp);
+          }
+          for (FieldInfo fi : infos) {
+            dvProducers.put(fi.name, dvp.get());
+          }
+        }
+      }
+
       success = true;
     } finally {
       // With lock-less commits, it's entirely possible (and
@@ -72,7 +150,7 @@ public final class SegmentReader extends
       // of things that were opened so that we don't have to
       // wait for a GC to do so.
       if (!success) {
-        core.decRef();
+        doClose();
       }
     }
   }
@@ -80,9 +158,9 @@ public final class SegmentReader extends
   /** Create new SegmentReader sharing core from a previous
    *  SegmentReader and loading new live docs from a new
    *  deletes file.  Used by openIfChanged. */
-  SegmentReader(SegmentInfoPerCommit si, SegmentCoreReaders core, IOContext context) throws IOException {
-    this(si, core,
-         si.info.getCodec().liveDocsFormat().readLiveDocs(si.info.dir, si, context),
+  SegmentReader(SegmentInfoPerCommit si, SegmentReader sr) throws IOException {
+    this(si, sr,
+         si.info.getCodec().liveDocsFormat().readLiveDocs(si.info.dir, si, IOContext.READONCE),
          si.info.getDocCount() - si.getDelCount());
   }
 
@@ -90,15 +168,124 @@ public final class SegmentReader extends
    *  SegmentReader and using the provided in-memory
    *  liveDocs.  Used by IndexWriter to provide a new NRT
    *  reader */
-  SegmentReader(SegmentInfoPerCommit si, SegmentCoreReaders core, Bits liveDocs, int numDocs) {
+  SegmentReader(SegmentInfoPerCommit si, SegmentReader sr, Bits liveDocs, int numDocs) throws IOException {
     this.si = si;
-    this.core = core;
+    this.liveDocs = liveDocs;
+    this.numDocs = numDocs;
+    this.core = sr.core;
     core.incRef();
+    
+//    System.out.println("[" + Thread.currentThread().getName() + "] SR.init: sharing reader: " + sr + " for gens=" + sr.genDVProducers.keySet());
+    
+    // increment refCount of DocValuesProducers that are used by this reader
+    boolean success = false;
+    try {
+      final Codec codec = si.info.getCodec();
+      if (si.getFieldInfosGen() == -1) {
+        fieldInfos = sr.fieldInfos;
+      } else {
+        fieldInfos = readFieldInfos(si);
+      }
+      
+      if (fieldInfos.hasDocValues()) {
+        final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
+        
+        final DocValuesFormat dvFormat = codec.docValuesFormat();
+        final Map<Long,List<FieldInfo>> genInfos = getGenInfos();
+        
+        for (Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
+          Long gen = e.getKey();
+          List<FieldInfo> infos = e.getValue();
+          RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
+          if (dvp == null) {
+            // check if this DVP gen is used by the given reader
+            dvp = sr.genDVProducers.get(gen);
+            if (dvp != null) {
+              // gen used by given reader, incRef its DVP
+              dvp.incRef();
+//              System.out.println("[" + Thread.currentThread().getName() + "] SR.init: sharing DVP for gen=" + gen + " refCount=" + dvp.getRefCount());
+            } else {
+              // this gen is not used by given reader, initialize a new one
+              dvp = newDocValuesProducer(si, IOContext.READ, dir, dvFormat, gen, infos);
+//              System.out.println("[" + Thread.currentThread().getName() + "] SR.init: new DVP for gen=" + gen + " refCount=" + dvp.getRefCount());
+            }
+            assert dvp != null;
+            genDVProducers.put(gen, dvp);
+          }
+          for (FieldInfo fi : infos) {
+            dvProducers.put(fi.name, dvp.get());
+          }
+        }
+      }
+      success = true;
+    } finally {
+      if (!success) {
+        doClose();
+      }
+    }
+  }
 
-    assert liveDocs != null;
-    this.liveDocs = liveDocs;
+  /**
+   * Reads the most recent {@link FieldInfos} of the given segment info.
+   * 
+   * @lucene.internal
+   */
+  static FieldInfos readFieldInfos(SegmentInfoPerCommit info) throws IOException {
+    final Directory dir;
+    final boolean closeDir;
+    if (info.getFieldInfosGen() == -1 && info.info.getUseCompoundFile()) {
+      // no fieldInfos gen and segment uses a compound file
+      dir = new CompoundFileDirectory(info.info.dir,
+          IndexFileNames.segmentFileName(info.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
+          IOContext.READONCE,
+          false);
+      closeDir = true;
+    } else {
+      // gen'd FIS are read outside CFS, or the segment doesn't use a compound file
+      dir = info.info.dir;
+      closeDir = false;
+    }
+    
+    try {
+      final String segmentSuffix = info.getFieldInfosGen() == -1 ? "" : Long.toString(info.getFieldInfosGen(), Character.MAX_RADIX);
+      return info.info.getCodec().fieldInfosFormat().getFieldInfosReader().read(dir, info.info.name, segmentSuffix, IOContext.READONCE);
+    } finally {
+      if (closeDir) {
+        dir.close();
+      }
+    }
+  }
+  
+  // returns a gen->List<FieldInfo> mapping. Fields without DV updates have gen=-1
+  private Map<Long,List<FieldInfo>> getGenInfos() {
+    final Map<Long,List<FieldInfo>> genInfos = new HashMap<Long,List<FieldInfo>>();
+    for (FieldInfo fi : fieldInfos) {
+      if (fi.getDocValuesType() == null) {
+        continue;
+      }
+      long gen = fi.getDocValuesGen();
+      List<FieldInfo> infos = genInfos.get(gen);
+      if (infos == null) {
+        infos = new ArrayList<FieldInfo>();
+        genInfos.put(gen, infos);
+      }
+      infos.add(fi);
+    }
+    return genInfos;
+  }
 
-    this.numDocs = numDocs;
+  private RefCount<DocValuesProducer> newDocValuesProducer(SegmentInfoPerCommit si, IOContext context, Directory dir,
+      DocValuesFormat dvFormat, Long gen, List<FieldInfo> infos) throws IOException {
+    Directory dvDir = dir;
+    String segmentSuffix = "";
+    if (gen.longValue() != -1) {
+      dvDir = si.info.dir; // gen'd files are written outside CFS, so use SegInfo directory
+      segmentSuffix = Long.toString(gen.longValue(), Character.MAX_RADIX);
+    }
+
+    // set SegmentReadState to list only the fields that are relevant to that gen
+    SegmentReadState srs = new SegmentReadState(dvDir, si.info, new FieldInfos(infos.toArray(new FieldInfo[infos.size()])), context, segmentSuffix);
+    return new DocValuesRefCount(dvFormat.fieldsProducer(srs));
   }
 
   @Override
@@ -110,13 +297,31 @@ public final class SegmentReader extends
   @Override
   protected void doClose() throws IOException {
     //System.out.println("SR.close seg=" + si);
-    core.decRef();
+    try {
+      core.decRef();
+    } finally {
+      Throwable t = null;
+      for (RefCount<DocValuesProducer> dvp : genDVProducers.values()) {
+        try {
+          dvp.decRef();
+        } catch (Throwable th) {
+          if (t != null) {
+            t = th;
+          }
+        }
+      }
+      genDVProducers.clear();
+      dvProducers.clear();
+      docValuesLocal.close();
+      docsWithFieldLocal.close();
+      IOUtils.reThrow(t);
+    }
   }
 
   @Override
   public FieldInfos getFieldInfos() {
     ensureOpen();
-    return core.fieldInfos;
+    return fieldInfos;
   }
 
   /** Expert: retrieve thread-private {@link
@@ -209,6 +414,9 @@ public final class SegmentReader extends
   // same entry in the FieldCache.  See LUCENE-1579.
   @Override
   public Object getCoreCacheKey() {
+    // NOTE: if this ever changes, be sure to fix
+    // SegmentCoreReader.notifyCoreClosedListeners to match!
+    // Today it passes "this" as its coreCacheKey:
     return core;
   }
 
@@ -217,34 +425,151 @@ public final class SegmentReader extends
     return this;
   }
 
+  // returns the FieldInfo that corresponds to the given field and type, or
+  // null if the field does not exist, or not indexed as the requested
+  // DovDocValuesType.
+  private FieldInfo getDVField(String field, DocValuesType type) {
+    FieldInfo fi = fieldInfos.fieldInfo(field);
+    if (fi == null) {
+      // Field does not exist
+      return null;
+    }
+    if (fi.getDocValuesType() == null) {
+      // Field was not indexed with doc values
+      return null;
+    }
+    if (fi.getDocValuesType() != type) {
+      // Field DocValues are different than requested type
+      return null;
+    }
+
+    return fi;
+  }
+  
   @Override
   public NumericDocValues getNumericDocValues(String field) throws IOException {
     ensureOpen();
-    return core.getNumericDocValues(field);
+    FieldInfo fi = getDVField(field, DocValuesType.NUMERIC);
+    if (fi == null) {
+      return null;
+    }
+
+    DocValuesProducer dvProducer = dvProducers.get(field);
+    assert dvProducer != null;
+
+    Map<String,Object> dvFields = docValuesLocal.get();
+
+    NumericDocValues dvs = (NumericDocValues) dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getNumeric(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
+  }
+
+  @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    ensureOpen();
+    FieldInfo fi = fieldInfos.fieldInfo(field);
+    if (fi == null) {
+      // Field does not exist
+      return null;
+    }
+    if (fi.getDocValuesType() == null) {
+      // Field was not indexed with doc values
+      return null;
+    }
+
+    DocValuesProducer dvProducer = dvProducers.get(field);
+    assert dvProducer != null;
+
+    Map<String,Bits> dvFields = docsWithFieldLocal.get();
+
+    Bits dvs = dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getDocsWithField(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
   }
 
   @Override
   public BinaryDocValues getBinaryDocValues(String field) throws IOException {
     ensureOpen();
-    return core.getBinaryDocValues(field);
+    FieldInfo fi = getDVField(field, DocValuesType.BINARY);
+    if (fi == null) {
+      return null;
+    }
+
+    DocValuesProducer dvProducer = dvProducers.get(field);
+    assert dvProducer != null;
+
+    Map<String,Object> dvFields = docValuesLocal.get();
+
+    BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getBinary(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
   }
 
   @Override
   public SortedDocValues getSortedDocValues(String field) throws IOException {
     ensureOpen();
-    return core.getSortedDocValues(field);
+    FieldInfo fi = getDVField(field, DocValuesType.SORTED);
+    if (fi == null) {
+      return null;
+    }
+
+    DocValuesProducer dvProducer = dvProducers.get(field);
+    assert dvProducer != null;
+
+    Map<String,Object> dvFields = docValuesLocal.get();
+
+    SortedDocValues dvs = (SortedDocValues) dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getSorted(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
   }
 
   @Override
   public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
     ensureOpen();
-    return core.getSortedSetDocValues(field);
+    FieldInfo fi = getDVField(field, DocValuesType.SORTED_SET);
+    if (fi == null) {
+      return null;
+    }
+
+    DocValuesProducer dvProducer = dvProducers.get(field);
+    assert dvProducer != null;
+
+    Map<String,Object> dvFields = docValuesLocal.get();
+
+    SortedSetDocValues dvs = (SortedSetDocValues) dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getSortedSet(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
   }
 
   @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     ensureOpen();
-    return core.getNormValues(field);
+    FieldInfo fi = fieldInfos.fieldInfo(field);
+    if (fi == null || !fi.hasNorms()) {
+      // Field does not exist or does not index norms
+      return null;
+    }
+    return core.getNormValues(fi);
   }
 
   /**
@@ -261,9 +586,9 @@ public final class SegmentReader extends
    * @lucene.experimental
    */
   public static interface CoreClosedListener {
-    /** Invoked when the shared core of the provided {@link
+    /** Invoked when the shared core of the original {@code
      *  SegmentReader} has closed. */
-    public void onClose(SegmentReader owner);
+    public void onClose(Object ownerCoreCacheKey);
   }
   
   /** Expert: adds a CoreClosedListener to this reader's shared core */
@@ -277,4 +602,22 @@ public final class SegmentReader extends
     ensureOpen();
     core.removeCoreClosedListener(listener);
   }
+  
+  private long dvRamBytesUsed() {
+    long ramBytesUsed = 0;
+    for (RefCount<DocValuesProducer> dvp : genDVProducers.values()) {
+      ramBytesUsed += dvp.get().ramBytesUsed();
+    }
+    return ramBytesUsed;
+  }
+
+  /** Returns approximate RAM Bytes used */
+  public long ramBytesUsed() {
+    ensureOpen();
+    long ramBytesUsed = dvRamBytesUsed();
+    if (core != null) {
+      ramBytesUsed += core.ramBytesUsed();
+    }
+    return ramBytesUsed;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java Mon Oct 21 18:58:24 2013
@@ -74,12 +74,23 @@ public class SegmentWriteState {
   /** Sole constructor. */
   public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
       BufferedDeletes segDeletes, IOContext context) {
+    this(infoStream, directory, segmentInfo, fieldInfos, segDeletes, context, "");
+  }
+
+  /**
+   * Constructor which takes segment suffix.
+   * 
+   * @see #SegmentWriteState(InfoStream, Directory, SegmentInfo, FieldInfos,
+   *      BufferedDeletes, IOContext)
+   */
+  public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
+      BufferedDeletes segDeletes, IOContext context, String segmentSuffix) {
     this.infoStream = infoStream;
     this.segDeletes = segDeletes;
     this.directory = directory;
     this.segmentInfo = segmentInfo;
     this.fieldInfos = fieldInfos;
-    segmentSuffix = "";
+    this.segmentSuffix = segmentSuffix;
     this.context = context;
   }
   

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Mon Oct 21 18:58:24 2013
@@ -64,9 +64,7 @@ public final class SlowCompositeReaderWr
     }
   }
 
-  /** Sole constructor, wrapping the provided {@link
-   *  CompositeReader}. */
-  public SlowCompositeReaderWrapper(CompositeReader reader) throws IOException {
+  private SlowCompositeReaderWrapper(CompositeReader reader) throws IOException {
     super();
     in = reader;
     fields = MultiFields.getFields(in);
@@ -92,6 +90,12 @@ public final class SlowCompositeReaderWr
   }
 
   @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    ensureOpen();
+    return MultiDocValues.getDocsWithField(in, field);
+  }
+
+  @Override
   public BinaryDocValues getBinaryDocValues(String field) throws IOException {
     ensureOpen();
     return MultiDocValues.getBinaryValues(in, field);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java Mon Oct 21 18:58:24 2013
@@ -37,12 +37,13 @@ public abstract class SortedDocValues ex
    * Returns the ordinal for the specified docID.
    * @param  docID document ID to lookup
    * @return ordinal for the document: this is dense, starts at 0, then
-   *         increments by 1 for the next value in sorted order. 
+   *         increments by 1 for the next value in sorted order. Note that
+   *         missing values are indicated by -1.
    */
   public abstract int getOrd(int docID);
 
   /** Retrieves the value for the specified ordinal.
-   * @param ord ordinal to lookup
+   * @param ord ordinal to lookup (must be &gt;= 0 and &lt {@link #getValueCount()})
    * @param result will be populated with the ordinal's value
    * @see #getOrd(int) 
    */
@@ -59,7 +60,7 @@ public abstract class SortedDocValues ex
   public void get(int docID, BytesRef result) {
     int ord = getOrd(docID);
     if (ord == -1) {
-      result.bytes = MISSING;
+      result.bytes = BytesRef.EMPTY_BYTES;
       result.length = 0;
       result.offset = 0;
     } else {
@@ -67,23 +68,23 @@ public abstract class SortedDocValues ex
     }
   }
 
-  /** An empty SortedDocValues which returns {@link #MISSING} for every document */
+  /** An empty SortedDocValues which returns {@link BytesRef#EMPTY_BYTES} for every document */
   public static final SortedDocValues EMPTY = new SortedDocValues() {
     @Override
     public int getOrd(int docID) {
-      return 0;
+      return -1;
     }
 
     @Override
     public void lookupOrd(int ord, BytesRef result) {
-      result.bytes = MISSING;
+      result.bytes = BytesRef.EMPTY_BYTES;
       result.offset = 0;
       result.length = 0;
     }
 
     @Override
     public int getValueCount() {
-      return 1;
+      return 0;
     }
   };
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -125,11 +124,6 @@ class SortedDocValuesTermsEnum extends T
   }
 
   @Override
-  public Comparator<BytesRef> getComparator() {
-    return BytesRef.getUTF8SortedAsUnicodeComparator();
-  }
-
-  @Override
   public void seekExact(BytesRef term, TermState state) throws IOException {
     assert state != null && state instanceof OrdTermState;
     this.seekExact(((OrdTermState)state).ord);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java Mon Oct 21 18:58:24 2013
@@ -30,19 +30,19 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
 import org.apache.lucene.util.packed.PackedInts;
 
 /** Buffers up pending byte[] per doc, deref and sorting via
  *  int ord, then flushes when segment flushes. */
 class SortedDocValuesWriter extends DocValuesWriter {
   final BytesRefHash hash;
-  private AppendingPackedLongBuffer pending;
+  private AppendingDeltaPackedLongBuffer pending;
   private final Counter iwBytesUsed;
   private long bytesUsed; // this currently only tracks differences in 'pending'
   private final FieldInfo fieldInfo;
 
-  private static final BytesRef EMPTY = new BytesRef(BytesRef.EMPTY_BYTES);
+  private static final int EMPTY_ORD = -1;
 
   public SortedDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     this.fieldInfo = fieldInfo;
@@ -52,7 +52,7 @@ class SortedDocValuesWriter extends DocV
             new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
             BytesRefHash.DEFAULT_CAPACITY,
             new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
-    pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
+    pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -70,7 +70,7 @@ class SortedDocValuesWriter extends DocV
 
     // Fill in any holes:
     while(pending.size() < docID) {
-      addOneValue(EMPTY);
+      pending.add(EMPTY_ORD);
     }
 
     addOneValue(value);
@@ -79,8 +79,9 @@ class SortedDocValuesWriter extends DocV
   @Override
   public void finish(int maxDoc) {
     while(pending.size() < maxDoc) {
-      addOneValue(EMPTY);
+      pending.add(EMPTY_ORD);
     }
+    updateBytesUsed();
   }
 
   private void addOneValue(BytesRef value) {
@@ -177,7 +178,7 @@ class SortedDocValuesWriter extends DocV
   
   // iterates over the ords for each doc we have in ram
   private class OrdsIterator implements Iterator<Number> {
-    final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
     final int ordMap[];
     final int maxDoc;
     int docUpto;
@@ -200,8 +201,7 @@ class SortedDocValuesWriter extends DocV
       }
       int ord = (int) iter.next();
       docUpto++;
-      // TODO: make reusable Number
-      return ordMap[ord];
+      return ord == -1 ? ord : ordMap[ord];
     }
 
     @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -125,11 +124,6 @@ class SortedSetDocValuesTermsEnum extend
   }
 
   @Override
-  public Comparator<BytesRef> getComparator() {
-    return BytesRef.getUTF8SortedAsUnicodeComparator();
-  }
-
-  @Override
   public void seekExact(BytesRef term, TermState state) throws IOException {
     assert state != null && state instanceof OrdTermState;
     this.seekExact(((OrdTermState)state).ord);