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 15:26:58 UTC

svn commit: r1625275 [1/4] - in /lucene/dev/trunk: lucene/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lu...

Author: rmuir
Date: Tue Sep 16 13:26:55 2014
New Revision: 1625275

URL: http://svn.apache.org/r1625275
Log:
LUCENE-5949: Add Accountable.getChildResources

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/Accountables.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsFieldReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/FuzzySet.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/MurmurHash2.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesProducer.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMFile.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/Accountable.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/NoOutputs.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedLongValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CachedOrdinalsReader.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCache.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionFieldReader.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellTernarySearchTrie.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Sep 16 13:26:55 2014
@@ -130,6 +130,8 @@ New Features
 * LUCENE-5820: SuggestStopFilter should have a factory. 
   (Varun Thacker via Steve Rowe)
 
+* LUCENE-5949: Add Accountable.getChildResources(). (Robert Muir)
+
 API Changes:
 
 * LUCENE-5900: Deprecated more constructors taking Version in *InfixSuggester and

Modified: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Tue Sep 16 13:26:55 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/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsReader.java Tue Sep 16 13:26:55 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/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java Tue Sep 16 13:26:55 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;
@@ -1163,8 +1165,17 @@ public class Lucene40PostingsReader exte
   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() + "(positions=" + (proxIn != null) + ")";
+  }
 }

Modified: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java Tue Sep 16 13:26:55 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/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Tue Sep 16 13:26:55 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -26,6 +27,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;
@@ -38,6 +40,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;
@@ -121,8 +124,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);
 
@@ -705,8 +712,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/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Tue Sep 16 13:26:55 2014
@@ -18,7 +18,10 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import java.io.IOException;
+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;
 
@@ -41,6 +44,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;
@@ -65,19 +70,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;
@@ -112,7 +119,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);
@@ -156,10 +163,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 + ")");
@@ -181,7 +191,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();
@@ -192,25 +202,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;
   }
@@ -221,14 +232,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:
@@ -244,6 +270,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) {
@@ -254,11 +281,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) {
@@ -271,6 +300,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) {
@@ -284,20 +314,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());
@@ -311,6 +342,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() {
 
@@ -328,15 +360,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);
@@ -402,18 +434,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/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsProducer.java Tue Sep 16 13:26:55 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 + ")";
+  }
 }

Modified: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java Tue Sep 16 13:26:55 2014
@@ -29,7 +29,10 @@ import static org.apache.lucene.codecs.l
 
 import java.io.Closeable; // javadocs
 import java.io.IOException;
+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;
 
@@ -53,6 +56,8 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.index.TermsEnum.SeekStatus;
 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.IOUtils;
@@ -75,6 +80,7 @@ class Lucene45DocValuesProducer extends 
   private final IndexInput data;
   private final int maxDoc;
   private final int version;
+  private final int numFields;
   
   // We need this for pre-4.9 indexes which recorded multiple fields' DocValues
   // updates under the same generation, and therefore the passed FieldInfos may
@@ -107,7 +113,7 @@ class Lucene45DocValuesProducer extends 
       ordIndexes = new HashMap<>();
       binaries = new HashMap<>();
       sortedSets = new HashMap<>();
-      readFields(in, state.fieldInfos);
+      numFields = readFields(in, state.fieldInfos);
 
       if (version >= Lucene45DocValuesFormat.VERSION_CHECKSUM) {
         CodecUtil.checkFooter(in);
@@ -204,9 +210,11 @@ class Lucene45DocValuesProducer extends 
     ordIndexes.put(fieldNumber, 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) {
+      numFields++;
       if ((lenientFieldInfoCheck && fieldNumber < 0) || (!lenientFieldInfoCheck && 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.
@@ -241,6 +249,7 @@ class Lucene45DocValuesProducer extends 
       }
       fieldNumber = meta.readVInt();
     }
+    return numFields;
   }
   
   static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
@@ -330,11 +339,24 @@ class Lucene45DocValuesProducer extends 
   }
   
   @Override
+  public synchronized Iterable<? extends Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    resources.addAll(Accountables.namedAccountables("addresses field number", addressInstances));
+    resources.addAll(Accountables.namedAccountables("ord index field number", ordIndexInstances));
+    return Collections.unmodifiableList(resources);
+  }
+  
+  @Override
   public void checkIntegrity() throws IOException {
     if (version >= Lucene45DocValuesFormat.VERSION_CHECKSUM) {
       CodecUtil.checksumEntireFile(data);
     }
   }
+  
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + numFields + ")";
+  }
 
   LongValues getNumeric(NumericEntry entry) throws IOException {
     final IndexInput data = this.data.clone();
@@ -411,18 +433,16 @@ class Lucene45DocValuesProducer extends 
   
   /** returns an address instance for variable-length binary values.
    *  @lucene.internal */
-  protected MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
+  protected synchronized MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
     final MonotonicBlockPackedReader addresses;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
-        addressInstances.put(field.number, addrInstance);
-        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
-      }
-      addresses = addrInstance;
+    MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
+    if (addrInstance == null) {
+      data.seek(bytes.addressesOffset);
+      addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
+      addressInstances.put(field.number, addrInstance);
+      ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
+    addresses = addrInstance;
     return addresses;
   }
   
@@ -453,25 +473,23 @@ class Lucene45DocValuesProducer extends 
   
   /** returns an address instance for prefix-compressed binary values. 
    * @lucene.internal */
-  protected MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
+  protected synchronized MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
     final MonotonicBlockPackedReader addresses;
     final long interval = bytes.addressInterval;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        final long size;
-        if (bytes.count % interval == 0) {
-          size = bytes.count / interval;
-        } else {
-          size = 1L + bytes.count / interval;
-        }
-        addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-        addressInstances.put(field.number, addrInstance);
-        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+    MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
+    if (addrInstance == null) {
+      data.seek(bytes.addressesOffset);
+      final long size;
+      if (bytes.count % interval == 0) {
+        size = bytes.count / interval;
+      } else {
+        size = 1L + bytes.count / interval;
       }
-      addresses = addrInstance;
+      addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+      addressInstances.put(field.number, addrInstance);
+      ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
+    addresses = addrInstance;
     return addresses;
   }
 
@@ -532,18 +550,16 @@ class Lucene45DocValuesProducer extends 
   
   /** returns an address instance for sortedset ordinal lists
    * @lucene.internal */
-  protected MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
+  protected synchronized MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
     final MonotonicBlockPackedReader ordIndex;
-    synchronized (ordIndexInstances) {
-      MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
-      if (ordIndexInstance == null) {
-        data.seek(entry.offset);
-        ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
-        ordIndexInstances.put(field.number, ordIndexInstance);
-        ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
-      }
-      ordIndex = ordIndexInstance;
+    MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
+    if (ordIndexInstance == null) {
+      data.seek(entry.offset);
+      ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
+      ordIndexInstances.put(field.number, ordIndexInstance);
+      ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
+    ordIndex = ordIndexInstance;
     return ordIndex;
   }
   

Modified: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java Tue Sep 16 13:26:55 2014
@@ -29,7 +29,10 @@ import static org.apache.lucene.codecs.l
 
 import java.io.Closeable; // javadocs
 import java.io.IOException;
+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;
 
@@ -54,6 +57,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;
@@ -64,20 +69,21 @@ import org.apache.lucene.util.packed.Mon
 
 /** reader for {@link Lucene49DocValuesFormat} */
 class Lucene49DocValuesProducer 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 AtomicLong ramBytesUsed;
   private final IndexInput data;
+  private final int numFields;
   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<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
+  private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
   
   /** expert: instantiates a new reader */
   Lucene49DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
@@ -96,7 +102,7 @@ class Lucene49DocValuesProducer extends 
       binaries = new HashMap<>();
       sortedSets = new HashMap<>();
       sortedNumerics = new HashMap<>();
-      readFields(in, state.fieldInfos);
+      numFields = readFields(in, state.fieldInfos);
 
       CodecUtil.checkFooter(in);
       success = true;
@@ -135,108 +141,110 @@ class Lucene49DocValuesProducer 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() != Lucene49DocValuesFormat.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() != Lucene49DocValuesFormat.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() != Lucene49DocValuesFormat.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() != Lucene49DocValuesFormat.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() != Lucene49DocValuesFormat.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 == Lucene49DocValuesFormat.NUMERIC) {
-        numerics.put(fieldNumber, readNumericEntry(meta));
+        numerics.put(info.name, readNumericEntry(meta));
       } else if (type == Lucene49DocValuesFormat.BINARY) {
         BinaryEntry b = readBinaryEntry(meta);
-        binaries.put(fieldNumber, b);
+        binaries.put(info.name, b);
       } else if (type == Lucene49DocValuesFormat.SORTED) {
-        readSortedField(fieldNumber, meta, infos);
+        readSortedField(info, meta);
       } else if (type == Lucene49DocValuesFormat.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() != Lucene49DocValuesFormat.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 == Lucene49DocValuesFormat.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() != Lucene49DocValuesFormat.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() != Lucene49DocValuesFormat.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();
         }
@@ -245,6 +253,7 @@ class Lucene49DocValuesProducer extends 
       }
       fieldNumber = meta.readVInt();
     }
+    return numFields;
   }
   
   static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
@@ -324,7 +333,7 @@ class Lucene49DocValuesProducer extends 
 
   @Override
   public NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericEntry entry = numerics.get(field.number);
+    NumericEntry entry = numerics.get(field.name);
     return getNumeric(entry);
   }
   
@@ -334,9 +343,22 @@ class Lucene49DocValuesProducer 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));
+    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);
@@ -376,7 +398,7 @@ class Lucene49DocValuesProducer 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);
@@ -415,18 +437,16 @@ class Lucene49DocValuesProducer extends 
   }
   
   /** returns an address instance for variable-length binary values. */
-  private MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
+  private synchronized MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
     final MonotonicBlockPackedReader addresses;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
-        addressInstances.put(field.number, addrInstance);
-        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
-      }
-      addresses = addrInstance;
+    MonotonicBlockPackedReader addrInstance = addressInstances.get(field.name);
+    if (addrInstance == null) {
+      data.seek(bytes.addressesOffset);
+      addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
+      addressInstances.put(field.name, addrInstance);
+      ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
+    addresses = addrInstance;
     return addresses;
   }
   
@@ -456,25 +476,23 @@ class Lucene49DocValuesProducer extends 
   }
   
   /** returns an address instance for prefix-compressed binary values. */
-  private MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
+  private synchronized MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
     final MonotonicBlockPackedReader addresses;
     final long interval = bytes.addressInterval;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        final long size;
-        if (bytes.count % interval == 0) {
-          size = bytes.count / interval;
-        } else {
-          size = 1L + bytes.count / interval;
-        }
-        addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-        addressInstances.put(field.number, addrInstance);
-        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+    MonotonicBlockPackedReader addrInstance = addressInstances.get(field.name);
+    if (addrInstance == null) {
+      data.seek(bytes.addressesOffset);
+      final long size;
+      if (bytes.count % interval == 0) {
+        size = bytes.count / interval;
+      } else {
+        size = 1L + bytes.count / interval;
       }
-      addresses = addrInstance;
+      addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+      addressInstances.put(field.name, addrInstance);
+      ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
+    addresses = addrInstance;
     return addresses;
   }
 
@@ -489,9 +507,9 @@ class Lucene49DocValuesProducer 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() {
@@ -532,32 +550,30 @@ class Lucene49DocValuesProducer extends 
   }
   
   /** returns an address instance for sortedset ordinal lists */
-  private MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
+  private synchronized MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
     final MonotonicBlockPackedReader ordIndex;
-    synchronized (ordIndexInstances) {
-      MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
-      if (ordIndexInstance == null) {
-        data.seek(entry.offset);
-        ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
-        ordIndexInstances.put(field.number, ordIndexInstance);
-        ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
-      }
-      ordIndex = ordIndexInstance;
+    MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.name);
+    if (ordIndexInstance == null) {
+      data.seek(entry.offset);
+      ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
+      ordIndexInstances.put(field.name, ordIndexInstance);
+      ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
+    ordIndex = ordIndexInstance;
     return ordIndex;
   }
   
   @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 IndexInput data = this.data.clone();
-      final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.number));
+      final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.name));
       
       return new SortedNumericDocValues() {
         long startOffset;
@@ -586,7 +602,7 @@ class Lucene49DocValuesProducer 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);
@@ -595,12 +611,12 @@ class Lucene49DocValuesProducer extends 
     }
 
     final IndexInput data = this.data.clone();
-    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(data, field, ordIndexes.get(field.number));
+    final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.name));
     
     return new RandomAccessOrds() {
       long startOffset;
@@ -698,10 +714,10 @@ class Lucene49DocValuesProducer 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();

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java Tue Sep 16 13:26:55 2014
@@ -18,8 +18,10 @@ package org.apache.lucene.codecs.blockte
  */
 
 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.BlockTermState;
@@ -42,6 +44,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.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -263,6 +266,11 @@ public class BlockTermsReader extends Fi
     public long ramBytesUsed() {
       return FIELD_READER_RAM_BYTES_USED;
     }
+    
+    @Override
+    public Iterable<? extends Accountable> getChildResources() {
+      return Collections.emptyList();
+    }
 
     @Override
     public TermsEnum iterator(TermsEnum reuse) throws IOException {
@@ -877,6 +885,23 @@ public class BlockTermsReader extends Fi
     }
     return ramBytesUsed;
   }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    if (indexReader != null) {
+      resources.add(Accountables.namedAccountable("term index", indexReader));
+    }
+    if (postingsReader != null) {
+      resources.add(Accountables.namedAccountable("delegate", postingsReader));
+    }
+    return Collections.unmodifiableList(resources);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(index=" + indexReader + ",delegate=" + postingsReader + ")";
+  }
 
   @Override
   public void checkIntegrity() throws IOException {   

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java Tue Sep 16 13:26:55 2014
@@ -25,13 +25,16 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldInfo;
 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.PagedBytes;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Comparator;
+import java.util.List;
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexFileNames;
@@ -61,7 +64,7 @@ public class FixedGapTermsIndexReader ex
   // all fields share this single logical byte[]
   private final PagedBytes.Reader termBytesReader;
 
-  final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<>();
+  final HashMap<String,FieldIndexData> fields = new HashMap<>();
   
   // start of the field info data
   private long dirOffset;
@@ -115,7 +118,7 @@ public class FixedGapTermsIndexReader ex
           throw new CorruptIndexException("invalid packedIndexStart: " + packedIndexStart + " indexStart: " + indexStart + "numIndexTerms: " + numIndexTerms + " (resource=" + in + ")");
         }
         final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
-        FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, termBytes, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
+        FieldIndexData previous = fields.put(fieldInfo.name, new FieldIndexData(in, termBytes, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
         if (previous != null) {
           throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
         }
@@ -270,11 +273,28 @@ public class FixedGapTermsIndexReader ex
       return ((termOffsets!=null)? termOffsets.ramBytesUsed() : 0) + 
           ((termsDictOffsets!=null)? termsDictOffsets.ramBytesUsed() : 0);
     }
+
+    @Override
+    public Iterable<? extends Accountable> getChildResources() {
+      List<Accountable> resources = new ArrayList<>();
+      if (termOffsets != null) {
+        resources.add(Accountables.namedAccountable("term lengths", termOffsets));
+      }
+      if (termsDictOffsets != null) {
+        resources.add(Accountables.namedAccountable("offsets", termsDictOffsets));
+      }
+      return resources;
+    }
+
+    @Override
+    public String toString() {
+      return "FixedGapTermIndex(indexterms=" + numIndexTerms + ")";
+    }
   }
 
   @Override
   public FieldIndexEnum getFieldEnum(FieldInfo fieldInfo) {
-    return new IndexEnum(fields.get(fieldInfo));
+    return new IndexEnum(fields.get(fieldInfo.name));
   }
 
   @Override
@@ -298,4 +318,14 @@ public class FixedGapTermsIndexReader ex
     }
     return sizeInBytes;
   }
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Accountables.namedAccountables("field", fields);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + fields.size() + ",interval=" + indexInterval + ")";
+  }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java Tue Sep 16 13:26:55 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.blockte
  */
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 
 import org.apache.lucene.codecs.CodecUtil;
@@ -29,6 +30,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.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
@@ -42,7 +44,7 @@ public class VariableGapTermsIndexReader
 
   private final PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton();
 
-  final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<>();
+  final HashMap<String,FieldIndexData> fields = new HashMap<>();
   
   // start of the field info data
   private long dirOffset;
@@ -76,7 +78,7 @@ public class VariableGapTermsIndexReader
         final int field = in.readVInt();
         final long indexStart = in.readVLong();
         final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
-        FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, fieldInfo, indexStart));
+        FieldIndexData previous = fields.put(fieldInfo.name, new FieldIndexData(in, fieldInfo, indexStart));
         if (previous != null) {
           throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
         }
@@ -175,11 +177,25 @@ public class VariableGapTermsIndexReader
     public long ramBytesUsed() {
       return fst == null ? 0 : fst.ramBytesUsed();
     }
+
+    @Override
+    public Iterable<? extends Accountable> getChildResources() {
+      if (fst == null) {
+        return Collections.emptyList();
+      } else {
+        return Collections.singletonList(Accountables.namedAccountable("index data", fst));
+      }
+    }
+    
+    @Override
+    public String toString() {
+      return "VarGapTermIndex";
+    }
   }
 
   @Override
   public FieldIndexEnum getFieldEnum(FieldInfo fieldInfo) {
-    final FieldIndexData fieldData = fields.get(fieldInfo);
+    final FieldIndexData fieldData = fields.get(fieldInfo.name);
     if (fieldData.fst == null) {
       return null;
     } else {
@@ -209,4 +225,14 @@ public class VariableGapTermsIndexReader
     }
     return sizeInBytes;
   }
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Accountables.namedAccountables("field", fields);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + fields.size() + ")";
+  }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java Tue Sep 16 13:26:55 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;
@@ -36,6 +38,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;
 
@@ -235,11 +239,19 @@ public final class OrdsBlockTreeTermsRea
 
   @Override
   public long ramBytesUsed() {
-    long sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0);
+    long sizeInBytes = postingsReader.ramBytesUsed();
     for (OrdsFieldReader 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
@@ -250,4 +262,9 @@ public final class OrdsBlockTreeTermsRea
     // postings
     postingsReader.checkIntegrity();
   }
+  
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader.toString() + ")";
+  }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsFieldReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsFieldReader.java?rev=1625275&r1=1625274&r2=1625275&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsFieldReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsFieldReader.java Tue Sep 16 13:26:55 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.blocktr
  */
 
 import java.io.IOException;
+import java.util.Collections;
 
 import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output;
 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.automaton.CompiledAutomaton;
 import org.apache.lucene.util.fst.FST;
@@ -170,4 +172,18 @@ final class OrdsFieldReader extends Term
   public long ramBytesUsed() {
     return ((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 "OrdsBlockTreeTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
+  }
 }