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 2014/09/16 21:00:52 UTC

svn commit: r1625356 [2/4] - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/ lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/ lucene/codecs/src/java/org/apache/lucen...

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java Tue Sep 16 19:00:49 2014
@@ -28,6 +28,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.Accountable;
 
 /**
  * plain-text norms format.
@@ -81,9 +82,19 @@ public class SimpleTextNormsFormat exten
     }
     
     @Override
+    public Iterable<? extends Accountable> getChildResources() {
+      return impl.getChildResources();
+    }
+
+    @Override
     public void checkIntegrity() throws IOException {
       impl.checkIntegrity();
     }
+
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "(" + impl + ")";
+    }
   }
   
   /**

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java Tue Sep 16 19:00:49 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.simplet
 
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.util.Collections;
 
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.index.FieldInfo;
@@ -32,6 +33,7 @@ import org.apache.lucene.store.ChecksumI
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -213,5 +215,15 @@ public class SimpleTextStoredFieldsReade
   }
 
   @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
+
+  @Override
   public void checkIntegrity() throws IOException {}
 }

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Tue Sep 16 19:00:49 2014
@@ -39,6 +39,7 @@ import org.apache.lucene.store.ChecksumI
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -563,5 +564,15 @@ public class SimpleTextTermVectorsReader
   }
 
   @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
+  
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
+
+  @Override
   public void checkIntegrity() throws IOException {}
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java Tue Sep 16 19:00:49 2014
@@ -18,8 +18,10 @@ package org.apache.lucene.codecs.blocktr
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.List;
 import java.util.TreeMap;
 
 import org.apache.lucene.codecs.CodecUtil;
@@ -35,6 +37,8 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -302,13 +306,19 @@ public class BlockTreeTermsReader extend
 
   @Override
   public long ramBytesUsed() {
-    long sizeInByes = BASE_RAM_BYTES_USED
-        + ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0)
-        + fields.size() * 2L * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+    long sizeInBytes = postingsReader.ramBytesUsed();
     for(FieldReader reader : fields.values()) {
-      sizeInByes += reader.ramBytesUsed();
+      sizeInBytes += reader.ramBytesUsed();
     }
-    return sizeInByes;
+    return sizeInBytes;
+  }
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    resources.addAll(Accountables.namedAccountables("field", fields));
+    resources.add(Accountables.namedAccountable("delegate", postingsReader));
+    return Collections.unmodifiableList(resources);
   }
 
   @Override
@@ -321,4 +331,9 @@ public class BlockTreeTermsReader extend
       postingsReader.checkIntegrity();
     }
   }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java Tue Sep 16 19:00:49 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.blocktr
 
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.Collections;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
@@ -27,6 +28,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
@@ -186,4 +188,18 @@ public final class FieldReader extends T
   public long ramBytesUsed() {
     return BASE_RAM_BYTES_USED + ((index!=null)? index.ramBytesUsed() : 0);
   }
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    if (index == null) {
+      return Collections.emptyList();
+    } else {
+      return Collections.singleton(Accountables.namedAccountable("term index", index));
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "BlockTreeTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java Tue Sep 16 19:00:49 2014
@@ -20,12 +20,16 @@ package org.apache.lucene.codecs.compres
 import static org.apache.lucene.util.BitUtil.zigZagDecode;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.PackedInts;
@@ -184,4 +188,27 @@ public final class CompressingStoredFiel
     return res;
   }
 
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    
+    long docBaseDeltaBytes = RamUsageEstimator.shallowSizeOf(docBasesDeltas);
+    for (PackedInts.Reader r : docBasesDeltas) {
+      docBaseDeltaBytes += r.ramBytesUsed();
+    }
+    resources.add(Accountables.namedAccountable("doc base deltas", docBaseDeltaBytes));
+    
+    long startPointerDeltaBytes = RamUsageEstimator.shallowSizeOf(startPointersDeltas);
+    for (PackedInts.Reader r : startPointersDeltas) {
+      startPointerDeltaBytes += r.ramBytesUsed();
+    }
+    resources.add(Accountables.namedAccountable("start pointer deltas", startPointerDeltaBytes));
+    
+    return resources;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(blocks=" + docBases.length + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java Tue Sep 16 19:00:49 2014
@@ -38,6 +38,7 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
+import java.util.Collections;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.StoredFieldsReader;
@@ -55,6 +56,8 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -528,6 +531,11 @@ public final class CompressingStoredFiel
   public long ramBytesUsed() {
     return indexReader.ramBytesUsed();
   }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.singleton(Accountables.namedAccountable("stored field index", indexReader));
+  }
 
   @Override
   public void checkIntegrity() throws IOException {
@@ -536,4 +544,8 @@ public final class CompressingStoredFiel
     }
   }
 
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(mode=" + compressionMode + ",chunksize=" + chunkSize + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java Tue Sep 16 19:00:49 2014
@@ -33,6 +33,7 @@ import static org.apache.lucene.codecs.c
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
@@ -54,6 +55,8 @@ import org.apache.lucene.store.ChecksumI
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -1081,10 +1084,19 @@ public final class CompressingTermVector
   }
   
   @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.singleton(Accountables.namedAccountable("term vector index", indexReader));
+  }
+  
+  @Override
   public void checkIntegrity() throws IOException {
     if (version >= VERSION_CHECKSUM) {
       CodecUtil.checksumEntireFile(vectorsStream);
     }
   }
 
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(mode=" + compressionMode + ",chunksize=" + chunkSize + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java Tue Sep 16 19:00:49 2014
@@ -40,6 +40,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -247,6 +248,11 @@ class Lucene3xFields extends FieldsProdu
     public boolean hasPayloads() {
       return fieldInfo.hasPayloads();
     }
+
+    @Override
+    public Iterable<? extends Accountable> getChildResources() {
+      return Collections.emptyList();
+    }
   }
 
   private class PreTermsEnum extends TermsEnum {
@@ -1097,4 +1103,18 @@ class Lucene3xFields extends FieldsProdu
 
   @Override
   public void checkIntegrity() throws IOException {}
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    if (tis == null) {
+      return Collections.emptyList();
+    } else {
+      return Collections.singleton(Accountables.namedAccountable("terms", tis));
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "Lucene3xFields(positions=" + (freqStream != null) + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xNormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xNormsProducer.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xNormsProducer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xNormsProducer.java Tue Sep 16 19:00:49 2014
@@ -34,6 +34,8 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.Version;
@@ -159,10 +161,11 @@ class Lucene3xNormsProducer extends Norm
 
   // holds a file+offset pointing to a norms, and lazy-loads it
   // to a singleton NumericDocValues instance
-  private class NormsDocValues {
+  private class NormsDocValues implements Accountable {
     private final IndexInput file;
     private final long offset;
     private NumericDocValues instance;
+    private final AtomicLong bytesUsed = new AtomicLong(-1);
     
     public NormsDocValues(IndexInput normInput, long normSeek) {
       this.file = normInput;
@@ -182,7 +185,9 @@ class Lucene3xNormsProducer extends Norm
           openFiles.remove(file);
           file.close();
         }
-        ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
+        long ram = RamUsageEstimator.sizeOf(bytes);
+        ramBytesUsed.addAndGet(ram);
+        bytesUsed.addAndGet(ram);
         instance = new NumericDocValues() {
           @Override
           public long get(int docID) {
@@ -191,7 +196,28 @@ class Lucene3xNormsProducer extends Norm
         };
       }
       return instance;
-    }    
+    }
+
+    @Override
+    public long ramBytesUsed() {
+      long v = bytesUsed.get();
+      return Math.max(v, 0);
+    }
+
+    @Override
+    public Iterable<? extends Accountable> getChildResources() {
+      long v = bytesUsed.get();
+      if (v < 0) {
+        return Collections.emptyList();
+      } else {
+        return Collections.singleton(Accountables.namedAccountable("byte array", v));
+      }
+    }
+
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "(active=" + (bytesUsed.get() >= 0) + ")";
+    }
   }
 
   @Override
@@ -208,4 +234,14 @@ class Lucene3xNormsProducer extends Norm
   
   @Override
   public void checkIntegrity() throws IOException {}
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Accountables.namedAccountables("field", norms);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + norms.size() + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xStoredFieldsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xStoredFieldsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xStoredFieldsReader.java Tue Sep 16 19:00:49 2014
@@ -33,11 +33,13 @@ import org.apache.lucene.store.CompoundF
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.Closeable;
 import java.nio.charset.StandardCharsets;
+import java.util.Collections;
 
 /**
  * Class responsible for access to stored document fields.
@@ -308,4 +310,14 @@ final class Lucene3xStoredFieldsReader e
   
   @Override
   public void checkIntegrity() throws IOException {}
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(docStoreOffset=" + docStoreOffset + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java Tue Sep 16 19:00:49 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene3
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -42,6 +43,7 @@ import org.apache.lucene.store.CompoundF
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -747,5 +749,15 @@ class Lucene3xTermVectorsReader extends 
   
   @Override
   public void checkIntegrity() throws IOException {}
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(docStoreOffset=" + docStoreOffset + ")";
+  }
 }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReader.java Tue Sep 16 19:00:49 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene3
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Comparator;
 
 import org.apache.lucene.index.CorruptIndexException;
@@ -28,6 +29,8 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CloseableThreadLocal;
 import org.apache.lucene.util.DoubleBarrelLRUCache;
@@ -41,7 +44,7 @@ import org.apache.lucene.util.IOUtils;
  * @lucene.experimental
  */
 @Deprecated
-class TermInfosReader implements Closeable {
+class TermInfosReader implements Accountable,Closeable {
   private final Directory directory;
   private final String segment;
   private final FieldInfos fieldInfos;
@@ -359,7 +362,22 @@ class TermInfosReader implements Closeab
     return getThreadResources().termEnum.clone();
   }
 
-  long ramBytesUsed() {
+  @Override
+  public long ramBytesUsed() {
     return index == null ? 0 : index.ramBytesUsed();
   }
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    if (index == null) {
+      return Collections.emptyList();
+    } else {
+      return Collections.singleton(Accountables.namedAccountable("term index", index));
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "TermInfosReader(interval=" + totalIndexInterval + ",size=" + size + ")"; 
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReaderIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReaderIndex.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReaderIndex.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReaderIndex.java Tue Sep 16 19:00:49 2014
@@ -19,10 +19,13 @@ package org.apache.lucene.codecs.lucene3
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.MathUtil;
@@ -41,7 +44,7 @@ import org.apache.lucene.util.packed.Pac
  * @deprecated Only for reading existing 3.x indexes
  */
 @Deprecated
-class TermInfosReaderIndex {
+class TermInfosReaderIndex implements Accountable {
 
   private static final int MAX_PAGE_BITS = 18; // 256 KB block
   private Term[] fields;
@@ -52,6 +55,7 @@ class TermInfosReaderIndex {
   private final int indexSize;
   private final int skipInterval;
   private final long ramBytesUsed;
+  private final long dataBytesUsed;
 
   /**
    * Loads the segment information at segment load time.
@@ -118,7 +122,8 @@ class TermInfosReaderIndex {
     long ramBytesUsed = RamUsageEstimator.shallowSizeOf(fields);
     ramBytesUsed += RamUsageEstimator.shallowSizeOf(dataInput);
     ramBytesUsed += fields.length * RamUsageEstimator.shallowSizeOfInstance(Term.class);
-    ramBytesUsed += dataPagedBytes.ramBytesUsed();
+    dataBytesUsed = dataPagedBytes.ramBytesUsed();
+    ramBytesUsed += dataBytesUsed;
     ramBytesUsed += indexToDataOffset.ramBytesUsed();
     this.ramBytesUsed = ramBytesUsed;
   }
@@ -264,8 +269,21 @@ class TermInfosReaderIndex {
     return term.field().compareTo(fields[input.readVInt()].field());
   }
 
-  long ramBytesUsed() {
+  @Override
+  public long ramBytesUsed() {
     return ramBytesUsed;
   }
 
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    resources.add(Accountables.namedAccountable("addresses", indexToDataOffset));
+    resources.add(Accountables.namedAccountable("term bytes", dataBytesUsed));
+    return Collections.unmodifiableList(resources);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(indexterms=" + indexSize + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Tue Sep 16 19:00:49 2014
@@ -37,6 +37,8 @@ import org.apache.lucene.index.SortedSet
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -57,12 +59,11 @@ final class Lucene40DocValuesReader exte
   private static final String segmentSuffix = "dv";
 
   // ram instances we have already loaded
-  private final Map<Integer,NumericDocValues> numericInstances =
-      new HashMap<>();
-  private final Map<Integer,BinaryDocValues> binaryInstances =
-      new HashMap<>();
-  private final Map<Integer,SortedDocValues> sortedInstances =
-      new HashMap<>();
+  private final Map<String,NumericDocValues> numericInstances = new HashMap<>();
+  private final Map<String,BinaryDocValues> binaryInstances = new HashMap<>();
+  private final Map<String,SortedDocValues> sortedInstances = new HashMap<>();
+  
+  private final Map<String,Accountable> instanceInfo = new HashMap<>();
 
   private final AtomicLong ramBytesUsed;
 
@@ -75,7 +76,7 @@ final class Lucene40DocValuesReader exte
 
   @Override
   public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericDocValues instance = numericInstances.get(field.number);
+    NumericDocValues instance = numericInstances.get(field.name);
     if (instance == null) {
       String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
       IndexInput input = dir.openInput(fileName, state.context);
@@ -115,7 +116,7 @@ final class Lucene40DocValuesReader exte
           IOUtils.closeWhileHandlingException(input);
         }
       }
-      numericInstances.put(field.number, instance);
+      numericInstances.put(field.name, instance);
     }
     return instance;
   }
@@ -131,7 +132,9 @@ final class Lucene40DocValuesReader exte
       for (int i = 0; i < values.length; i++) {
         values[i] = input.readLong();
       }
-      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
+      long bytesUsed = RamUsageEstimator.sizeOf(values);
+      instanceInfo.put(field.name, Accountables.namedAccountable("long array", bytesUsed));
+      ramBytesUsed.addAndGet(bytesUsed);
       return new NumericDocValues() {
         @Override
         public long get(int docID) {
@@ -142,6 +145,7 @@ final class Lucene40DocValuesReader exte
       final long minValue = input.readLong();
       final long defaultValue = input.readLong();
       final PackedInts.Reader reader = PackedInts.getReader(input);
+      instanceInfo.put(field.name, reader);
       ramBytesUsed.addAndGet(reader.ramBytesUsed());
       return new NumericDocValues() {
         @Override
@@ -170,7 +174,9 @@ final class Lucene40DocValuesReader exte
     int maxDoc = state.segmentInfo.getDocCount();
     final byte values[] = new byte[maxDoc];
     input.readBytes(values, 0, values.length);
-    ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
+    long bytesUsed = RamUsageEstimator.sizeOf(values);
+    instanceInfo.put(field.name, Accountables.namedAccountable("byte array", bytesUsed));
+    ramBytesUsed.addAndGet(bytesUsed);
     return new NumericDocValues() {
       @Override
       public long get(int docID) {
@@ -192,7 +198,9 @@ final class Lucene40DocValuesReader exte
     for (int i = 0; i < values.length; i++) {
       values[i] = input.readShort();
     }
-    ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
+    long bytesUsed = RamUsageEstimator.sizeOf(values);
+    instanceInfo.put(field.name, Accountables.namedAccountable("short array", bytesUsed));
+    ramBytesUsed.addAndGet(bytesUsed);
     return new NumericDocValues() {
       @Override
       public long get(int docID) {
@@ -214,7 +222,9 @@ final class Lucene40DocValuesReader exte
     for (int i = 0; i < values.length; i++) {
       values[i] = input.readInt();
     }
-    ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
+    long bytesUsed = RamUsageEstimator.sizeOf(values);
+    instanceInfo.put(field.name, Accountables.namedAccountable("int array", bytesUsed));
+    ramBytesUsed.addAndGet(bytesUsed);
     return new NumericDocValues() {
       @Override
       public long get(int docID) {
@@ -236,7 +246,9 @@ final class Lucene40DocValuesReader exte
     for (int i = 0; i < values.length; i++) {
       values[i] = input.readLong();
     }
-    ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
+    long bytesUsed = RamUsageEstimator.sizeOf(values);
+    instanceInfo.put(field.name, Accountables.namedAccountable("long array", bytesUsed));
+    ramBytesUsed.addAndGet(bytesUsed);
     return new NumericDocValues() {
       @Override
       public long get(int docID) {
@@ -258,7 +270,9 @@ final class Lucene40DocValuesReader exte
     for (int i = 0; i < values.length; i++) {
       values[i] = input.readInt();
     }
-    ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
+    long bytesUsed = RamUsageEstimator.sizeOf(values);
+    instanceInfo.put(field.name, Accountables.namedAccountable("float array", bytesUsed));
+    ramBytesUsed.addAndGet(bytesUsed);
     return new NumericDocValues() {
       @Override
       public long get(int docID) {
@@ -280,7 +294,9 @@ final class Lucene40DocValuesReader exte
     for (int i = 0; i < values.length; i++) {
       values[i] = input.readLong();
     }
-    ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
+    long bytesUsed = RamUsageEstimator.sizeOf(values);
+    instanceInfo.put(field.name, Accountables.namedAccountable("double array", bytesUsed));
+    ramBytesUsed.addAndGet(bytesUsed);
     return new NumericDocValues() {
       @Override
       public long get(int docID) {
@@ -291,7 +307,7 @@ final class Lucene40DocValuesReader exte
 
   @Override
   public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
-    BinaryDocValues instance = binaryInstances.get(field.number);
+    BinaryDocValues instance = binaryInstances.get(field.name);
     if (instance == null) {
       switch(LegacyDocValuesType.valueOf(field.getAttribute(legacyKey))) {
         case BYTES_FIXED_STRAIGHT:
@@ -309,7 +325,7 @@ final class Lucene40DocValuesReader exte
         default:
           throw new AssertionError();
       }
-      binaryInstances.put(field.number, instance);
+      binaryInstances.put(field.name, instance);
     }
     return instance;
   }
@@ -329,6 +345,7 @@ final class Lucene40DocValuesReader exte
       CodecUtil.checkEOF(input);
       success = true;
       ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
+      instanceInfo.put(field.name, bytesReader);
       return new BinaryDocValues() {
 
         @Override
@@ -370,7 +387,9 @@ final class Lucene40DocValuesReader exte
       CodecUtil.checkEOF(data);
       CodecUtil.checkEOF(index);
       success = true;
-      ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
+      long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
+      ramBytesUsed.addAndGet(bytesUsed);
+      instanceInfo.put(field.name, Accountables.namedAccountable("variable straight", bytesUsed));
       return new BinaryDocValues() {
         @Override
         public BytesRef get(int docID) {
@@ -414,7 +433,9 @@ final class Lucene40DocValuesReader exte
       final PackedInts.Reader reader = PackedInts.getReader(index);
       CodecUtil.checkEOF(data);
       CodecUtil.checkEOF(index);
-      ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
+      long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
+      ramBytesUsed.addAndGet(bytesUsed);
+      instanceInfo.put(field.name, Accountables.namedAccountable("fixed deref", bytesUsed));
       success = true;
       return new BinaryDocValues() {
         @Override
@@ -457,7 +478,9 @@ final class Lucene40DocValuesReader exte
       final PackedInts.Reader reader = PackedInts.getReader(index);
       CodecUtil.checkEOF(data);
       CodecUtil.checkEOF(index);
-      ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
+      long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
+      ramBytesUsed.addAndGet(bytesUsed);
+      instanceInfo.put(field.name, Accountables.namedAccountable("variable deref", bytesUsed));
       success = true;
       return new BinaryDocValues() {
         
@@ -490,7 +513,7 @@ final class Lucene40DocValuesReader exte
 
   @Override
   public synchronized SortedDocValues getSorted(FieldInfo field) throws IOException {
-    SortedDocValues instance = sortedInstances.get(field.number);
+    SortedDocValues instance = sortedInstances.get(field.name);
     if (instance == null) {
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
       String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "idx");
@@ -520,7 +543,7 @@ final class Lucene40DocValuesReader exte
           IOUtils.closeWhileHandlingException(data, index);
         }
       }
-      sortedInstances.put(field.number, instance);
+      sortedInstances.put(field.name, instance);
     }
     return instance;
   }
@@ -540,7 +563,9 @@ final class Lucene40DocValuesReader exte
     bytes.copy(data, fixedLength * (long) valueCount);
     final PagedBytes.Reader bytesReader = bytes.freeze(true);
     final PackedInts.Reader reader = PackedInts.getReader(index);
-    ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
+    long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
+    ramBytesUsed.addAndGet(bytesUsed);
+    instanceInfo.put(field.name, Accountables.namedAccountable("fixed sorted", bytesUsed));
 
     return correctBuggyOrds(new SortedDocValues() {
       @Override
@@ -578,7 +603,9 @@ final class Lucene40DocValuesReader exte
     final PackedInts.Reader ordsReader = PackedInts.getReader(index);
 
     final int valueCount = addressReader.size() - 1;
-    ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed());
+    long bytesUsed = bytesReader.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed();
+    ramBytesUsed.addAndGet(bytesUsed);
+    instanceInfo.put(field.name, Accountables.namedAccountable("var sorted", bytesUsed));
 
     return correctBuggyOrds(new SortedDocValues() {
       @Override
@@ -654,8 +681,18 @@ final class Lucene40DocValuesReader exte
   public long ramBytesUsed() {
     return ramBytesUsed.get();
   }
+  
+  @Override
+  public synchronized Iterable<? extends Accountable> getChildResources() {
+    return Accountables.namedAccountables("field", instanceInfo);
+  }
 
   @Override
   public void checkIntegrity() throws IOException {
   }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java Tue Sep 16 19:00:49 2014
@@ -23,6 +23,7 @@ import org.apache.lucene.codecs.NormsPro
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.util.Accountable;
 
 /**
  * Reads 4.0/4.1 norms.
@@ -51,9 +52,19 @@ class Lucene40NormsReader extends NormsP
   public long ramBytesUsed() {
     return impl.ramBytesUsed();
   }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return impl.getChildResources();
+  }
 
   @Override
   public void checkIntegrity() throws IOException {
     impl.checkIntegrity();
   }
+  
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(" + impl + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java Tue Sep 16 19:00:49 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
@@ -35,6 +36,7 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -1166,8 +1168,17 @@ public class Lucene40PostingsReader exte
   public long ramBytesUsed() {
     return BASE_RAM_BYTES_USED;
   }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
 
   @Override
   public void checkIntegrity() throws IOException {}
 
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(positions=" + (proxIn != null) + ")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java Tue Sep 16 19:00:49 2014
@@ -31,11 +31,13 @@ import org.apache.lucene.store.AlreadyCl
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.Closeable;
 import java.nio.charset.StandardCharsets;
+import java.util.Collections;
 
 /**
  * Class responsible for access to stored document fields.
@@ -258,7 +260,17 @@ public final class Lucene40StoredFieldsR
   public long ramBytesUsed() {
     return RAM_BYTES_USED;
   }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
 
   @Override
   public void checkIntegrity() throws IOException {}
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Tue Sep 16 19:00:49 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene4
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -27,6 +28,7 @@ import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
@@ -39,6 +41,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -122,8 +125,12 @@ public class Lucene40TermVectorsReader e
       assert HEADER_LENGTH_INDEX == tvx.getFilePointer();
       assert HEADER_LENGTH_DOCS == tvd.getFilePointer();
       assert HEADER_LENGTH_FIELDS == tvf.getFilePointer();
-      assert tvxVersion == tvdVersion;
-      assert tvxVersion == tvfVersion;
+      if (tvxVersion != tvdVersion) {
+        throw new CorruptIndexException("version mismatch: tvx=" + tvxVersion + " != tvd=" + tvdVersion + " (resource=" + tvd + ")");
+      }
+      if (tvxVersion != tvfVersion) {
+        throw new CorruptIndexException("version mismatch: tvx=" + tvxVersion + " != tvf=" + tvfVersion + " (resource=" + tvf + ")");
+      }
 
       numTotalDocs = (int) (tvx.length()-HEADER_LENGTH_INDEX >> 4);
 
@@ -718,8 +725,18 @@ public class Lucene40TermVectorsReader e
   public long ramBytesUsed() {
     return 0;
   }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
 
   @Override
   public void checkIntegrity() throws IOException {}
+  
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
 }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java Tue Sep 16 19:00:49 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
@@ -252,5 +253,9 @@ final class ForUtil implements Accountab
     }
     return PackedInts.bitsRequired(or);
   }
-
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java Tue Sep 16 19:00:49 2014
@@ -24,6 +24,7 @@ import static org.apache.lucene.codecs.l
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
@@ -39,6 +40,7 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -1580,6 +1582,11 @@ public final class Lucene41PostingsReade
   public long ramBytesUsed() {
     return BASE_RAM_BYTES_USED  + forUtil.ramBytesUsed();
   }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
 
   @Override
   public void checkIntegrity() throws IOException {
@@ -1595,4 +1602,9 @@ public final class Lucene41PostingsReade
       }
     }
   }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(positions=" + (posIn != null) + ",payloads=" + (payIn != null) +")";
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java Tue Sep 16 19:00:49 2014
@@ -37,7 +37,10 @@ import static org.apache.lucene.codecs.l
 import java.io.Closeable; // javadocs
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -61,6 +64,8 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -72,21 +77,22 @@ import org.apache.lucene.util.packed.Mon
 
 /** reader for {@link Lucene410DocValuesFormat} */
 class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable {
-  private final Map<Integer,NumericEntry> numerics;
-  private final Map<Integer,BinaryEntry> binaries;
-  private final Map<Integer,SortedSetEntry> sortedSets;
-  private final Map<Integer,SortedSetEntry> sortedNumerics;
-  private final Map<Integer,NumericEntry> ords;
-  private final Map<Integer,NumericEntry> ordIndexes;
+  private final Map<String,NumericEntry> numerics;
+  private final Map<String,BinaryEntry> binaries;
+  private final Map<String,SortedSetEntry> sortedSets;
+  private final Map<String,SortedSetEntry> sortedNumerics;
+  private final Map<String,NumericEntry> ords;
+  private final Map<String,NumericEntry> ordIndexes;
+  private final int numFields;
   private final AtomicLong ramBytesUsed;
   private final IndexInput data;
   private final int maxDoc;
   private final int version;
 
   // memory-resident structures
-  private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
-  private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
-  private final Map<Integer,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
+  private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
+  private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
+  private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
   
   /** expert: instantiates a new reader */
   Lucene410DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
@@ -105,7 +111,7 @@ class Lucene410DocValuesProducer extends
       binaries = new HashMap<>();
       sortedSets = new HashMap<>();
       sortedNumerics = new HashMap<>();
-      readFields(in, state.fieldInfos);
+      numFields = readFields(in, state.fieldInfos);
 
       CodecUtil.checkFooter(in);
       success = true;
@@ -144,108 +150,110 @@ class Lucene410DocValuesProducer extends
     ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
   }
 
-  private void readSortedField(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
+  private void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
     // sorted = binary + numeric
-    if (meta.readVInt() != fieldNumber) {
-      throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     if (meta.readByte() != Lucene410DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     BinaryEntry b = readBinaryEntry(meta);
-    binaries.put(fieldNumber, b);
+    binaries.put(info.name, b);
     
-    if (meta.readVInt() != fieldNumber) {
-      throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     NumericEntry n = readNumericEntry(meta);
-    ords.put(fieldNumber, n);
+    ords.put(info.name, n);
   }
 
-  private void readSortedSetFieldWithAddresses(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
+  private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
     // sortedset = binary + numeric (addresses) + ordIndex
-    if (meta.readVInt() != fieldNumber) {
-      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     if (meta.readByte() != Lucene410DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     BinaryEntry b = readBinaryEntry(meta);
-    binaries.put(fieldNumber, b);
+    binaries.put(info.name, b);
 
-    if (meta.readVInt() != fieldNumber) {
-      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     NumericEntry n1 = readNumericEntry(meta);
-    ords.put(fieldNumber, n1);
+    ords.put(info.name, n1);
 
-    if (meta.readVInt() != fieldNumber) {
-      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
     }
     NumericEntry n2 = readNumericEntry(meta);
-    ordIndexes.put(fieldNumber, n2);
+    ordIndexes.put(info.name, n2);
   }
 
-  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+  private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    int numFields = 0;
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
-      if (infos.fieldInfo(fieldNumber) == null) {
-        // trickier to validate more: because we re-use for norms, because we use multiple entries
-        // for "composite" types like sortedset, etc.
+      numFields++;
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
         throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
       }
       byte type = meta.readByte();
       if (type == Lucene410DocValuesFormat.NUMERIC) {
-        numerics.put(fieldNumber, readNumericEntry(meta));
+        numerics.put(info.name, readNumericEntry(meta));
       } else if (type == Lucene410DocValuesFormat.BINARY) {
         BinaryEntry b = readBinaryEntry(meta);
-        binaries.put(fieldNumber, b);
+        binaries.put(info.name, b);
       } else if (type == Lucene410DocValuesFormat.SORTED) {
-        readSortedField(fieldNumber, meta, infos);
+        readSortedField(info, meta);
       } else if (type == Lucene410DocValuesFormat.SORTED_SET) {
         SortedSetEntry ss = readSortedSetEntry(meta);
-        sortedSets.put(fieldNumber, ss);
+        sortedSets.put(info.name, ss);
         if (ss.format == SORTED_WITH_ADDRESSES) {
-          readSortedSetFieldWithAddresses(fieldNumber, meta, infos);
+          readSortedSetFieldWithAddresses(info, meta);
         } else if (ss.format == SORTED_SINGLE_VALUED) {
           if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
           }
           if (meta.readByte() != Lucene410DocValuesFormat.SORTED) {
-            throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
           }
-          readSortedField(fieldNumber, meta, infos);
+          readSortedField(info, meta);
         } else {
           throw new AssertionError();
         }
       } else if (type == Lucene410DocValuesFormat.SORTED_NUMERIC) {
         SortedSetEntry ss = readSortedSetEntry(meta);
-        sortedNumerics.put(fieldNumber, ss);
+        sortedNumerics.put(info.name, ss);
         if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
         }
         if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
         }
-        numerics.put(fieldNumber, readNumericEntry(meta));
+        numerics.put(info.name, readNumericEntry(meta));
         if (ss.format == SORTED_WITH_ADDRESSES) {
           if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
           }
           if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
           }
           NumericEntry ordIndex = readNumericEntry(meta);
-          ordIndexes.put(fieldNumber, ordIndex);
+          ordIndexes.put(info.name, ordIndex);
         } else if (ss.format != SORTED_SINGLE_VALUED) {
           throw new AssertionError();
         }
@@ -254,6 +262,7 @@ class Lucene410DocValuesProducer extends
       }
       fieldNumber = meta.readVInt();
     }
+    return numFields;
   }
   
   static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
@@ -333,7 +342,7 @@ class Lucene410DocValuesProducer extends
 
   @Override
   public NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericEntry entry = numerics.get(field.number);
+    NumericEntry entry = numerics.get(field.name);
     return getNumeric(entry);
   }
   
@@ -343,10 +352,24 @@ class Lucene410DocValuesProducer extends
   }
   
   @Override
+  public synchronized Iterable<? extends Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
+    resources.addAll(Accountables.namedAccountables("ord index field", ordIndexInstances));
+    resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
+    return Collections.unmodifiableList(resources);
+  }
+  
+  @Override
   public void checkIntegrity() throws IOException {
     CodecUtil.checksumEntireFile(data);
   }
 
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + numFields + ")";
+  }
+
   LongValues getNumeric(NumericEntry entry) throws IOException {
     RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
     switch (entry.format) {
@@ -385,7 +408,7 @@ class Lucene410DocValuesProducer extends
 
   @Override
   public BinaryDocValues getBinary(FieldInfo field) throws IOException {
-    BinaryEntry bytes = binaries.get(field.number);
+    BinaryEntry bytes = binaries.get(field.name);
     switch(bytes.format) {
       case BINARY_FIXED_UNCOMPRESSED:
         return getFixedBinary(field, bytes);
@@ -421,11 +444,11 @@ class Lucene410DocValuesProducer extends
   
   /** returns an address instance for variable-length binary values. */
   private synchronized MonotonicBlockPackedReader getAddressInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    MonotonicBlockPackedReader addresses = addressInstances.get(field.number);
+    MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
     if (addresses == null) {
       data.seek(bytes.addressesOffset);
       addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
-      addressInstances.put(field.number, addresses);
+      addressInstances.put(field.name, addresses);
       ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
     return addresses;
@@ -458,12 +481,12 @@ class Lucene410DocValuesProducer extends
   
   /** returns an address instance for prefix-compressed binary values. */
   private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    MonotonicBlockPackedReader addresses = addressInstances.get(field.number);
+    MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
     if (addresses == null) {
       data.seek(bytes.addressesOffset);
       final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
       addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      addressInstances.put(field.number, addresses);
+      addressInstances.put(field.name, addresses);
       ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
     return addresses;
@@ -471,7 +494,7 @@ class Lucene410DocValuesProducer extends
   
   /** returns a reverse lookup instance for prefix-compressed binary values. */
   private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    ReverseTermsIndex index = reverseIndexInstances.get(field.number);
+    ReverseTermsIndex index = reverseIndexInstances.get(field.name);
     if (index == null) {
       index = new ReverseTermsIndex();
       data.seek(bytes.reverseIndexOffset);
@@ -481,8 +504,8 @@ class Lucene410DocValuesProducer extends
       PagedBytes pagedBytes = new PagedBytes(15);
       pagedBytes.copy(data, dataSize);
       index.terms = pagedBytes.freeze(true);
-      reverseIndexInstances.put(field.number, index);
-      ramBytesUsed.addAndGet(index.termAddresses.ramBytesUsed() + index.terms.ramBytesUsed());
+      reverseIndexInstances.put(field.name, index);
+      ramBytesUsed.addAndGet(index.ramBytesUsed());
     }
     return index;
   }
@@ -497,9 +520,9 @@ class Lucene410DocValuesProducer extends
 
   @Override
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
-    final int valueCount = (int) binaries.get(field.number).count;
+    final int valueCount = (int) binaries.get(field.name).count;
     final BinaryDocValues binary = getBinary(field);
-    NumericEntry entry = ords.get(field.number);
+    NumericEntry entry = ords.get(field.name);
     final LongValues ordinals = getNumeric(entry);
     return new SortedDocValues() {
 
@@ -540,11 +563,11 @@ class Lucene410DocValuesProducer extends
   
   /** returns an address instance for sortedset ordinal lists */
   private synchronized MonotonicBlockPackedReader getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
-    MonotonicBlockPackedReader instance = ordIndexInstances.get(field.number);
+    MonotonicBlockPackedReader instance = ordIndexInstances.get(field.name);
     if (instance == null) {
       data.seek(entry.offset);
       instance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
-      ordIndexInstances.put(field.number, instance);
+      ordIndexInstances.put(field.name, instance);
       ramBytesUsed.addAndGet(instance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
     return instance;
@@ -552,14 +575,14 @@ class Lucene410DocValuesProducer extends
   
   @Override
   public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
-    SortedSetEntry ss = sortedNumerics.get(field.number);
-    NumericEntry numericEntry = numerics.get(field.number);
+    SortedSetEntry ss = sortedNumerics.get(field.name);
+    NumericEntry numericEntry = numerics.get(field.name);
     final LongValues values = getNumeric(numericEntry);
     if (ss.format == SORTED_SINGLE_VALUED) {
       final Bits docsWithField = getMissingBits(numericEntry.missingOffset);
       return DocValues.singleton(values, docsWithField);
     } else if (ss.format == SORTED_WITH_ADDRESSES) {
-      final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.number));
+      final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
       
       return new SortedNumericDocValues() {
         long startOffset;
@@ -588,7 +611,7 @@ class Lucene410DocValuesProducer extends
 
   @Override
   public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
-    SortedSetEntry ss = sortedSets.get(field.number);
+    SortedSetEntry ss = sortedSets.get(field.name);
     if (ss.format == SORTED_SINGLE_VALUED) {
       final SortedDocValues values = getSorted(field);
       return DocValues.singleton(values);
@@ -596,12 +619,12 @@ class Lucene410DocValuesProducer extends
       throw new AssertionError();
     }
 
-    final long valueCount = binaries.get(field.number).count;
+    final long valueCount = binaries.get(field.name).count;
     // we keep the byte[]s and list of ords on disk, these could be large
     final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
-    final LongValues ordinals = getNumeric(ords.get(field.number));
+    final LongValues ordinals = getNumeric(ords.get(field.name));
     // but the addresses to the ord stream are in RAM
-    final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.number));
+    final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
     
     return new RandomAccessOrds() {
       long startOffset;
@@ -699,10 +722,10 @@ class Lucene410DocValuesProducer extends
       case SORTED:
         return DocValues.docsWithValue(getSorted(field), maxDoc);
       case BINARY:
-        BinaryEntry be = binaries.get(field.number);
+        BinaryEntry be = binaries.get(field.name);
         return getMissingBits(be.missingOffset);
       case NUMERIC:
-        NumericEntry ne = numerics.get(field.number);
+        NumericEntry ne = numerics.get(field.name);
         return getMissingBits(ne.missingOffset);
       default:
         throw new AssertionError();
@@ -779,9 +802,27 @@ class Lucene410DocValuesProducer extends
   }
   
   // used for reverse lookup to a small range of blocks
-  static class ReverseTermsIndex {
+  static class ReverseTermsIndex implements Accountable {
     public MonotonicBlockPackedReader termAddresses;
     public PagedBytes.Reader terms;
+    
+    @Override
+    public long ramBytesUsed() {
+      return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
+    }
+    
+    @Override
+    public Iterable<? extends Accountable> getChildResources() {
+      List<Accountable> resources = new ArrayList<>();
+      resources.add(Accountables.namedAccountable("term bytes", terms));
+      resources.add(Accountables.namedAccountable("term addresses", termAddresses));
+      return Collections.unmodifiableList(resources);
+    }
+
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
+    }
   }
   
   //in the compressed case, we add a few additional operations for

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Tue Sep 16 19:00:49 2014
@@ -19,7 +19,10 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -42,6 +45,8 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -66,19 +71,21 @@ import org.apache.lucene.util.packed.Pac
  */
 class Lucene42DocValuesProducer extends DocValuesProducer {
   // metadata maps (just file pointers and minimal stuff)
-  private final Map<Integer,NumericEntry> numerics;
-  private final Map<Integer,BinaryEntry> binaries;
-  private final Map<Integer,FSTEntry> fsts;
+  private final Map<String,NumericEntry> numerics;
+  private final Map<String,BinaryEntry> binaries;
+  private final Map<String,FSTEntry> fsts;
   private final IndexInput data;
   private final int version;
+  private final int numEntries;
   
   // ram instances we have already loaded
-  private final Map<Integer,NumericDocValues> numericInstances = 
-      new HashMap<>();
-  private final Map<Integer,BinaryDocValues> binaryInstances =
-      new HashMap<>();
-  private final Map<Integer,FST<Long>> fstInstances =
-      new HashMap<>();
+  private final Map<String,NumericDocValues> numericInstances = new HashMap<>();
+  private final Map<String,BinaryDocValues> binaryInstances = new HashMap<>();
+  private final Map<String,FST<Long>> fstInstances = new HashMap<>();
+  
+  private final Map<String,Accountable> numericInfo = new HashMap<>();
+  private final Map<String,Accountable> binaryInfo = new HashMap<>();
+  private final Map<String,Accountable> addressInfo = new HashMap<>();
   
   private final int maxDoc;
   private final AtomicLong ramBytesUsed;
@@ -113,7 +120,7 @@ class Lucene42DocValuesProducer extends 
       numerics = new HashMap<>();
       binaries = new HashMap<>();
       fsts = new HashMap<>();
-      readFields(in, state.fieldInfos);
+      numEntries = readFields(in, state.fieldInfos);
 
       if (version >= VERSION_CHECKSUM) {
         CodecUtil.checkFooter(in);
@@ -157,10 +164,13 @@ class Lucene42DocValuesProducer extends 
     }
   }
   
-  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+  private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    int numEntries = 0;
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
-      if (infos.fieldInfo(fieldNumber) == null) {
+      numEntries++;
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
         // trickier to validate more: because we re-use for norms, because we use multiple entries
         // for "composite" types like sortedset, etc.
         throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
@@ -182,7 +192,7 @@ class Lucene42DocValuesProducer extends 
         if (entry.format != UNCOMPRESSED) {
           entry.packedIntsVersion = meta.readVInt();
         }
-        numerics.put(fieldNumber, entry);
+        numerics.put(info.name, entry);
       } else if (fieldType == BYTES) {
         BinaryEntry entry = new BinaryEntry();
         entry.offset = meta.readLong();
@@ -193,25 +203,26 @@ class Lucene42DocValuesProducer extends 
           entry.packedIntsVersion = meta.readVInt();
           entry.blockSize = meta.readVInt();
         }
-        binaries.put(fieldNumber, entry);
+        binaries.put(info.name, entry);
       } else if (fieldType == FST) {
         FSTEntry entry = new FSTEntry();
         entry.offset = meta.readLong();
         entry.numOrds = meta.readVLong();
-        fsts.put(fieldNumber, entry);
+        fsts.put(info.name, entry);
       } else {
         throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
       }
       fieldNumber = meta.readVInt();
     }
+    return numEntries;
   }
 
   @Override
   public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericDocValues instance = numericInstances.get(field.number);
+    NumericDocValues instance = numericInstances.get(field.name);
     if (instance == null) {
       instance = loadNumeric(field);
-      numericInstances.put(field.number, instance);
+      numericInstances.put(field.name, instance);
     }
     return instance;
   }
@@ -222,14 +233,29 @@ class Lucene42DocValuesProducer extends 
   }
   
   @Override
+  public synchronized Iterable<? extends Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    resources.addAll(Accountables.namedAccountables("numeric field", numericInfo));
+    resources.addAll(Accountables.namedAccountables("binary field", binaryInfo));
+    resources.addAll(Accountables.namedAccountables("addresses field", addressInfo));
+    resources.addAll(Accountables.namedAccountables("terms dict field", fstInstances));
+    return Collections.unmodifiableList(resources);
+  }
+  
+  @Override
   public void checkIntegrity() throws IOException {
     if (version >= VERSION_CHECKSUM) {
       CodecUtil.checksumEntireFile(data);
     }
   }
 
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(entries=" + numEntries + ")";
+  }
+
   private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
-    NumericEntry entry = numerics.get(field.number);
+    NumericEntry entry = numerics.get(field.name);
     data.seek(entry.offset);
     switch (entry.format) {
       case TABLE_COMPRESSED:
@@ -245,6 +271,7 @@ class Lucene42DocValuesProducer extends 
         final int bitsPerValue = data.readVInt();
         final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
         ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
+        numericInfo.put(field.name, ordsReader);
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
@@ -255,11 +282,13 @@ class Lucene42DocValuesProducer extends 
         final int blockSize = data.readVInt();
         final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
         ramBytesUsed.addAndGet(reader.ramBytesUsed());
+        numericInfo.put(field.name, reader);
         return reader;
       case UNCOMPRESSED:
         final byte bytes[] = new byte[maxDoc];
         data.readBytes(bytes, 0, bytes.length);
         ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
+        numericInfo.put(field.name, Accountables.namedAccountable("byte array", maxDoc));
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
@@ -272,6 +301,7 @@ class Lucene42DocValuesProducer extends 
         final int quotientBlockSize = data.readVInt();
         final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
         ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
+        numericInfo.put(field.name, quotientReader);
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
@@ -285,20 +315,21 @@ class Lucene42DocValuesProducer extends 
 
   @Override
   public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
-    BinaryDocValues instance = binaryInstances.get(field.number);
+    BinaryDocValues instance = binaryInstances.get(field.name);
     if (instance == null) {
       instance = loadBinary(field);
-      binaryInstances.put(field.number, instance);
+      binaryInstances.put(field.name, instance);
     }
     return instance;
   }
   
   private BinaryDocValues loadBinary(FieldInfo field) throws IOException {
-    BinaryEntry entry = binaries.get(field.number);
+    BinaryEntry entry = binaries.get(field.name);
     data.seek(entry.offset);
     PagedBytes bytes = new PagedBytes(16);
     bytes.copy(data, entry.numBytes);
     final PagedBytes.Reader bytesReader = bytes.freeze(true);
+    binaryInfo.put(field.name, bytesReader);
     if (entry.minLength == entry.maxLength) {
       final int fixedLength = entry.minLength;
       ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
@@ -312,6 +343,7 @@ class Lucene42DocValuesProducer extends 
       };
     } else {
       final MonotonicBlockPackedReader addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
+      addressInfo.put(field.name, addresses);
       ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addresses.ramBytesUsed());
       return new BinaryDocValues() {
 
@@ -329,15 +361,15 @@ class Lucene42DocValuesProducer extends 
   
   @Override
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
-    final FSTEntry entry = fsts.get(field.number);
+    final FSTEntry entry = fsts.get(field.name);
     FST<Long> instance;
     synchronized(this) {
-      instance = fstInstances.get(field.number);
+      instance = fstInstances.get(field.name);
       if (instance == null) {
         data.seek(entry.offset);
         instance = new FST<>(data, PositiveIntOutputs.getSingleton());
         ramBytesUsed.addAndGet(instance.ramBytesUsed());
-        fstInstances.put(field.number, instance);
+        fstInstances.put(field.name, instance);
       }
     }
     final NumericDocValues docToOrd = getNumeric(field);
@@ -403,18 +435,18 @@ class Lucene42DocValuesProducer extends 
   
   @Override
   public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
-    final FSTEntry entry = fsts.get(field.number);
+    final FSTEntry entry = fsts.get(field.name);
     if (entry.numOrds == 0) {
       return DocValues.emptySortedSet(); // empty FST!
     }
     FST<Long> instance;
     synchronized(this) {
-      instance = fstInstances.get(field.number);
+      instance = fstInstances.get(field.name);
       if (instance == null) {
         data.seek(entry.offset);
         instance = new FST<>(data, PositiveIntOutputs.getSingleton());
         ramBytesUsed.addAndGet(instance.ramBytesUsed());
-        fstInstances.put(field.number, instance);
+        fstInstances.put(field.name, instance);
       }
     }
     final BinaryDocValues docToOrds = getBinary(field);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java?rev=1625356&r1=1625355&r2=1625356&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java Tue Sep 16 19:00:49 2014
@@ -18,11 +18,13 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import java.io.IOException;
+import java.util.Collections;
 
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.util.Accountable;
 
 /**
  * Reads 4.2-4.8 norms.
@@ -51,9 +53,19 @@ class Lucene42NormsProducer extends Norm
   public long ramBytesUsed() {
     return impl.ramBytesUsed();
   }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return impl.getChildResources();
+  }
 
   @Override
   public void close() throws IOException {
     impl.close();
   }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(" + impl + ")";
+  }
 }