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/30 11:37:52 UTC

svn commit: r1628382 [2/2] - in /lucene/dev/branches/lucene5969/lucene: backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ backward-codecs/src/java/org/apache/lucene/codecs/lucene42/ backward-codecs/src/java/org/apache/lucene/codecs/lucene45/ ...

Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java?rev=1628382&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java Tue Sep 30 09:37:52 2014
@@ -0,0 +1,319 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NormsConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.FilterIterator;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
+
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
+
+/**
+ * Writer for {@link Lucene50NormsFormat}
+ */
+class Lucene50NormsConsumer extends NormsConsumer { 
+  static final byte DELTA_COMPRESSED = 0;
+  static final byte TABLE_COMPRESSED = 1;
+  static final byte CONST_COMPRESSED = 2;
+  static final byte UNCOMPRESSED = 3;
+  static final byte INDIRECT = 4;
+  static final int BLOCK_SIZE = 1 << 14;
+  
+  // threshold for indirect encoding, computed as 1 - 1/log2(maxint)
+  // norms are only read for matching postings... so this is the threshold
+  // where n log n operations < maxdoc (e.g. it performs similar to other fields)
+  static final float INDIRECT_THRESHOLD = 1 - 1 / 31F;
+
+  IndexOutput data, meta;
+  
+  Lucene50NormsConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    boolean success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.createOutput(dataName, state.context);
+      CodecUtil.writeSegmentHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId());
+      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeSegmentHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId());
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+  
+  // we explicitly use only certain bits per value and a specified format, so we statically check this will work
+  static {
+    assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(1);
+    assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(2);
+    assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(4);
+  }
+
+  @Override
+  public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    // TODO: more efficient?
+    NormMap uniqueValues = new NormMap();
+    
+    int count = 0;
+    int missingCount = 0;
+    
+    for (Number nv : values) {
+      if (nv == null) {
+        throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
+      }
+      final long v = nv.longValue();
+      if (v == 0) {
+        missingCount++;
+      }
+      
+      minValue = Math.min(minValue, v);
+      maxValue = Math.max(maxValue, v);
+      
+      if (uniqueValues != null) {
+        if (uniqueValues.add(v)) {
+          if (uniqueValues.size > 256) {
+            uniqueValues = null;
+          }
+        }
+      }
+      count++;
+    }
+    if (uniqueValues != null && uniqueValues.size == 1) {
+      // 0 bpv
+      addConstant(minValue);
+    } else if (count > 256 && missingCount > count * INDIRECT_THRESHOLD) {
+      // sparse encoding
+      addIndirect(field, values, count, missingCount);
+    } else if (uniqueValues != null) {
+      // small number of unique values: this is the typical case:
+      FormatAndBits compression = fastestFormatAndBits(uniqueValues.size-1);
+      
+      if (compression.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
+        addUncompressed(values, count);
+      } else {
+        addTableCompressed(values, compression, count, uniqueValues);
+      }
+    } else {
+      addDeltaCompressed(values, count);
+    }
+  }
+  
+  private FormatAndBits fastestFormatAndBits(int max) {
+    // we only use bpv=1,2,4,8     
+    PackedInts.Format format = PackedInts.Format.PACKED_SINGLE_BLOCK;
+    int bitsPerValue = PackedInts.bitsRequired(max);
+    if (bitsPerValue == 3) {
+      bitsPerValue = 4;
+    } else if (bitsPerValue > 4) {
+      bitsPerValue = 8;
+    }
+    return new FormatAndBits(format, bitsPerValue);
+  }
+  
+  private void addConstant(long constant) throws IOException {
+    meta.writeVInt(0);
+    meta.writeByte(CONST_COMPRESSED);
+    meta.writeLong(constant);
+  }
+  
+  private void addUncompressed(Iterable<Number> values, int count) throws IOException {
+    meta.writeVInt(count);
+    meta.writeByte(UNCOMPRESSED); // uncompressed byte[]
+    meta.writeLong(data.getFilePointer());
+    for (Number nv : values) {
+      data.writeByte((byte) nv.longValue());
+    }
+  }
+  
+  private void addTableCompressed(Iterable<Number> values, FormatAndBits compression, int count, NormMap uniqueValues) throws IOException {
+    meta.writeVInt(count);
+    meta.writeByte(TABLE_COMPRESSED); // table-compressed
+    meta.writeLong(data.getFilePointer());
+    data.writeVInt(PackedInts.VERSION_CURRENT);
+    
+    long[] decode = uniqueValues.getDecodeTable();
+    // upgrade to power of two sized array
+    int size = 1 << compression.bitsPerValue;
+    data.writeVInt(size);
+    for (int i = 0; i < decode.length; i++) {
+      data.writeLong(decode[i]);
+    }
+    for (int i = decode.length; i < size; i++) {
+      data.writeLong(0);
+    }
+
+    data.writeVInt(compression.format.getId());
+    data.writeVInt(compression.bitsPerValue);
+
+    final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, compression.format, count, compression.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
+    for(Number nv : values) {
+      writer.add(uniqueValues.getOrd(nv.longValue()));
+    }
+    writer.finish();
+  }
+  
+  private void addDeltaCompressed(Iterable<Number> values, int count) throws IOException {
+    meta.writeVInt(count);
+    meta.writeByte(DELTA_COMPRESSED); // delta-compressed
+    meta.writeLong(data.getFilePointer());
+    data.writeVInt(PackedInts.VERSION_CURRENT);
+    data.writeVInt(BLOCK_SIZE);
+
+    final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
+    for (Number nv : values) {
+      writer.add(nv.longValue());
+    }
+    writer.finish();
+  }
+  
+  private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, int missingCount) throws IOException {
+    meta.writeVInt(count - missingCount);
+    meta.writeByte(INDIRECT);
+    meta.writeLong(data.getFilePointer());
+    data.writeVInt(PackedInts.VERSION_CURRENT);
+    data.writeVInt(BLOCK_SIZE);
+    
+    // write docs with value
+    final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+    int doc = 0;
+    for (Number n : values) {
+      long v = n.longValue();
+      if (v != 0) {
+        writer.add(doc);
+      }
+      doc++;
+    }
+    writer.finish();
+    
+    // write actual values
+    addNormsField(field, new Iterable<Number>() {
+      @Override
+      public Iterator<Number> iterator() {
+        return new FilterIterator<Number,Number>(values.iterator()) {
+          @Override
+          protected boolean predicateFunction(Number value) {
+            return value.longValue() != 0;
+          }
+        };
+      }
+    });
+  }
+  
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeVInt(-1); // write EOF marker
+        CodecUtil.writeFooter(meta); // write checksum
+      }
+      if (data != null) {
+        CodecUtil.writeFooter(data); // write checksum
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(data, meta);
+      } else {
+        IOUtils.closeWhileHandlingException(data, meta);
+      }
+      meta = data = null;
+    }
+  }
+  
+  // specialized deduplication of long->ord for norms: 99.99999% of the time this will be a single-byte range.
+  static class NormMap {
+    // we use short: at most we will add 257 values to this map before its rejected as too big above.
+    final short[] singleByteRange = new short[256];
+    final Map<Long,Short> other = new HashMap<Long,Short>();
+    int size;
+    
+    {
+      Arrays.fill(singleByteRange, (short)-1);
+    }
+
+    /** adds an item to the mapping. returns true if actually added */
+    public boolean add(long l) {
+      assert size <= 256; // once we add > 256 values, we nullify the map in addNumericField and don't use this strategy
+      if (l >= Byte.MIN_VALUE && l <= Byte.MAX_VALUE) {
+        int index = (int) (l + 128);
+        short previous = singleByteRange[index];
+        if (previous < 0) {
+          singleByteRange[index] = (short) size;
+          size++;
+          return true;
+        } else {
+          return false;
+        }
+      } else {
+        if (!other.containsKey(l)) {
+          other.put(l, (short)size);
+          size++;
+          return true;
+        } else {
+          return false;
+        }
+      }
+    }
+    
+    /** gets the ordinal for a previously added item */
+    public int getOrd(long l) {
+      if (l >= Byte.MIN_VALUE && l <= Byte.MAX_VALUE) {
+        int index = (int) (l + 128);
+        return singleByteRange[index];
+      } else {
+        // NPE if something is screwed up
+        return other.get(l);
+      }
+    }
+    
+    /** retrieves the ordinal table for previously added items */
+    public long[] getDecodeTable() {
+      long decode[] = new long[size];
+      for (int i = 0; i < singleByteRange.length; i++) {
+        short s = singleByteRange[i];
+        if (s >= 0) {
+          decode[s] = i - 128;
+        }
+      }
+      for (Map.Entry<Long,Short> entry : other.entrySet()) {
+        decode[entry.getValue()] = entry.getKey();
+      }
+      return decode;
+    }
+  }
+}

Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java?rev=1628382&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java Tue Sep 30 09:37:52 2014
@@ -0,0 +1,125 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NormsConsumer;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Lucene 5.0 Score normalization format.
+ * <p>
+ * Encodes normalization values with these strategies:
+ * <p>
+ * <ul>
+ *    <li>Uncompressed: when values fit into a single byte and would require more than 4 bits
+ *        per value, they are just encoded as an uncompressed byte array.
+ *    <li>Constant: when there is only one value present for the entire field, no actual data
+ *        is written: this constant is encoded in the metadata
+ *    <li>Table-compressed: when the number of unique values is very small (&lt; 64), and
+ *        when there are unused "gaps" in the range of values used (such as {@link SmallFloat}), 
+ *        a lookup table is written instead. Each per-document entry is instead the ordinal 
+ *        to this table, and those ordinals are compressed with bitpacking ({@link PackedInts}). 
+ *    <li>Delta-compressed: per-document integers written as deltas from the minimum value,
+ *        compressed with bitpacking. For more information, see {@link BlockPackedWriter}.
+ *        This is only used when norms of larger than one byte are present.
+ *    <li>Indirect: when norms are extremely sparse, missing values are omitted.
+ *        Access to an individual value is slower, but missing norm values are never accessed
+ *        by search code.
+ * </ul>
+ * <p>
+ * Files:
+ * <ol>
+ *   <li><tt>.nvd</tt>: Norms data</li>
+ *   <li><tt>.nvm</tt>: Norms metadata</li>
+ * </ol>
+ * <ol>
+ *   <li><a name="nvm" id="nvm"></a>
+ *   <p>The Norms metadata or .nvm file.</p>
+ *   <p>For each norms field, this stores metadata, such as the offset into the 
+ *      Norms data (.nvd)</p>
+ *   <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
+ *   <ul>
+ *     <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *     <li>Entry --&gt; FieldNumber,Type,Offset</li>
+ *     <li>FieldNumber --&gt; {@link DataOutput#writeVInt vInt}</li>
+ *     <li>Type --&gt; {@link DataOutput#writeByte Byte}</li>
+ *     <li>Offset --&gt; {@link DataOutput#writeLong Int64}</li>
+ *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   </ul>
+ *   <p>FieldNumber of -1 indicates the end of metadata.</p>
+ *   <p>Offset is the pointer to the start of the data in the norms data (.nvd), or the singleton value for Constant</p>
+ *   <p>Type indicates how Numeric values will be compressed:
+ *      <ul>
+ *         <li>0 --&gt; delta-compressed. For each block of 16k integers, every integer is delta-encoded
+ *             from the minimum value within the block. 
+ *         <li>1 --&gt; table-compressed. When the number of unique numeric values is small and it would save space,
+ *             a lookup table of unique values is written, followed by the ordinal for each document.
+ *         <li>2 --&gt; constant. When there is a single value for the entire field.
+ *         <li>3 --&gt; uncompressed: Values written as a simple byte[].
+ *         <li>4 --&gt; indirect. Only documents with a value are written with a sparse encoding.
+ *      </ul>
+ *   <li><a name="nvd" id="nvd"></a>
+ *   <p>The Norms data or .nvd file.</p>
+ *   <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)</p>
+ *   <p>Norms data (.nvd) --&gt; Header,&lt;Uncompressed | TableCompressed | DeltaCompressed&gt;<sup>NumFields</sup>,Footer</p>
+ *   <ul>
+ *     <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *     <li>Uncompressed --&gt;  {@link DataOutput#writeByte Byte}<sup>maxDoc</sup></li>
+ *     <li>TableCompressed --&gt; PackedIntsVersion,Table,BitPackedData</li>
+ *     <li>Table --&gt; TableSize, {@link DataOutput#writeLong int64}<sup>TableSize</sup></li>
+ *     <li>BitpackedData --&gt; {@link PackedInts}</li>
+ *     <li>DeltaCompressed --&gt; PackedIntsVersion,BlockSize,DeltaCompressedData</li>
+ *     <li>DeltaCompressedData --&gt; {@link BlockPackedWriter BlockPackedWriter(blockSize=16k)}</li>
+ *     <li>PackedIntsVersion,BlockSize,TableSize --&gt; {@link DataOutput#writeVInt vInt}</li>
+ *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   </ul>
+ * </ol>
+ * @lucene.experimental
+ */
+public class Lucene50NormsFormat extends NormsFormat {
+
+  /** Sole Constructor */
+  public Lucene50NormsFormat() {}
+  
+  @Override
+  public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene50NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+
+  @Override
+  public NormsProducer normsProducer(SegmentReadState state) throws IOException {
+    return new Lucene50NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+  
+  private static final String DATA_CODEC = "Lucene50NormsData";
+  private static final String DATA_EXTENSION = "nvd";
+  private static final String METADATA_CODEC = "Lucene50NormsMetadata";
+  private static final String METADATA_EXTENSION = "nvm";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+}

Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java?rev=1628382&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java Tue Sep 30 09:37:52 2014
@@ -0,0 +1,320 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SegmentReadState;
+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.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.BlockPackedReader;
+import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_START;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.CONST_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.TABLE_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.INDIRECT;
+
+/**
+ * Reader for {@link Lucene50NormsFormat}
+ */
+class Lucene50NormsProducer extends NormsProducer {
+  // metadata maps (just file pointers and minimal stuff)
+  private final Map<String,NormsEntry> norms = new HashMap<>();
+  private final IndexInput data;
+  
+  // ram instances we have already loaded
+  final Map<String,NumericDocValues> instances = new HashMap<>();
+  final Map<String,Accountable> instancesInfo = new HashMap<>();
+  
+  private final AtomicLong ramBytesUsed;
+  private final AtomicInteger activeCount = new AtomicInteger();
+  
+  private final boolean merging;
+  
+  // clone for merge: when merging we don't do any instances.put()s
+  Lucene50NormsProducer(Lucene50NormsProducer original) {
+    assert Thread.holdsLock(original);
+    norms.putAll(original.norms);
+    data = original.data.clone();
+    instances.putAll(original.instances);
+    instancesInfo.putAll(original.instancesInfo);
+    ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
+    activeCount.set(original.activeCount.get());
+    merging = true;
+  }
+    
+  Lucene50NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    merging = false;
+    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+    ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+    int version = -1;
+    
+    // read in the entries from the metadata file.
+    try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
+      Throwable priorE = null;
+      try {
+        version = CodecUtil.checkSegmentHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId());
+        readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
+      }
+    }
+
+    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+    this.data = state.directory.openInput(dataName, state.context);
+    boolean success = false;
+    try {
+      final int version2 = CodecUtil.checkSegmentHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId());
+      if (version != version2) {
+        throw new CorruptIndexException("Format versions mismatch: meta=" + version + ",data=" + version2, data);
+      }
+      
+      // NOTE: data file is too costly to verify checksum against all the bytes on open,
+      // but for now we at least verify proper structure of the checksum footer: which looks
+      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+      // such as file truncation.
+      CodecUtil.retrieveChecksum(data);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.data);
+      }
+    }
+  }
+  
+  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    int fieldNumber = meta.readVInt();
+    while (fieldNumber != -1) {
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      } else if (!info.hasNorms()) {
+        throw new CorruptIndexException("Invalid field: " + info.name, meta);
+      }
+      NormsEntry entry = readEntry(info, meta);
+      norms.put(info.name, entry);
+      fieldNumber = meta.readVInt();
+    }
+  }
+  
+  private NormsEntry readEntry(FieldInfo info, IndexInput meta) throws IOException {
+    NormsEntry entry = new NormsEntry();
+    entry.count = meta.readVInt();
+    entry.format = meta.readByte();
+    entry.offset = meta.readLong();
+    switch(entry.format) {
+      case CONST_COMPRESSED:
+      case UNCOMPRESSED:
+      case TABLE_COMPRESSED:
+      case DELTA_COMPRESSED:
+        break;
+      case INDIRECT:
+        if (meta.readVInt() != info.number) {
+          throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta);
+        }
+        entry.nested = readEntry(info, meta);
+        break;
+      default:
+        throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+    }
+    return entry;
+  }
+
+  @Override
+  public synchronized NumericDocValues getNorms(FieldInfo field) throws IOException {
+    NumericDocValues instance = instances.get(field.name);
+    if (instance == null) {
+      LoadedNorms loaded = loadNorms(norms.get(field.name));
+      instance = loaded.norms;
+      if (!merging) {
+        instances.put(field.name, instance);
+        activeCount.incrementAndGet();
+        ramBytesUsed.addAndGet(loaded.ramBytesUsed);
+        instancesInfo.put(field.name, loaded.info);
+      }
+    }
+    return instance;
+  }
+  
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed.get();
+  }
+  
+  @Override
+  public synchronized Iterable<? extends Accountable> getChildResources() {
+    return Accountables.namedAccountables("field", instancesInfo);
+  }
+  
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(data);
+  }
+
+  private LoadedNorms loadNorms(NormsEntry entry) throws IOException {
+    LoadedNorms instance = new LoadedNorms();
+    switch(entry.format) {
+      case CONST_COMPRESSED: {
+        final long v = entry.offset;
+        instance.info = Accountables.namedAccountable("constant", 8);
+        instance.ramBytesUsed = 8;
+        instance.norms = new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return v;
+          }
+        };
+        break;
+      }
+      case UNCOMPRESSED: {
+        data.seek(entry.offset);
+        final byte bytes[] = new byte[entry.count];
+        data.readBytes(bytes, 0, bytes.length);
+        instance.info = Accountables.namedAccountable("byte array", bytes.length);
+        instance.ramBytesUsed = RamUsageEstimator.sizeOf(bytes);
+        instance.norms = new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return bytes[docID];
+          }
+        };
+        break;
+      }
+      case DELTA_COMPRESSED: {
+        data.seek(entry.offset);
+        int packedIntsVersion = data.readVInt();
+        int blockSize = data.readVInt();
+        final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, entry.count, false);
+        instance.info = Accountables.namedAccountable("delta compressed", reader);
+        instance.ramBytesUsed = reader.ramBytesUsed();
+        instance.norms = reader;
+        break;
+      }
+      case TABLE_COMPRESSED: {
+        data.seek(entry.offset);
+        int packedIntsVersion = data.readVInt();
+        int size = data.readVInt();
+        if (size > 256) {
+          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + size, data);
+        }
+        final long decode[] = new long[size];
+        for (int i = 0; i < decode.length; i++) {
+          decode[i] = data.readLong();
+        }
+        final int formatID = data.readVInt();
+        final int bitsPerValue = data.readVInt();
+        final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue);
+        instance.info = Accountables.namedAccountable("table compressed", ordsReader);
+        instance.ramBytesUsed = RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed();
+        instance.norms = new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return decode[(int)ordsReader.get(docID)];
+          }
+        };
+        break;
+      }
+      case INDIRECT: {
+        data.seek(entry.offset);
+        int packedIntsVersion = data.readVInt();
+        int blockSize = data.readVInt();
+        final MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, false);
+        LoadedNorms nestedInstance = loadNorms(entry.nested);
+        instance.ramBytesUsed = live.ramBytesUsed() + nestedInstance.ramBytesUsed;
+        instance.info = Accountables.namedAccountable("indirect -> " + nestedInstance.info, instance.ramBytesUsed);
+        final NumericDocValues values = nestedInstance.norms;
+        final int upperBound = entry.count-1;
+        instance.norms = new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            int low = 0;
+            int high = upperBound;
+            while (low <= high) {
+              int mid = (low + high) >>> 1;
+              long doc = live.get(mid);
+              
+              if (doc < docID) {
+                low = mid + 1;
+              } else if (doc > docID) {
+                high = mid - 1;
+              } else {
+                return values.get(mid);
+              }
+            }
+            return 0;
+          }
+        };
+        break;
+      }
+      default:
+        throw new AssertionError();
+    }
+    return instance;
+  }
+
+  @Override
+  public void close() throws IOException {
+    data.close();
+  }
+  
+  static class NormsEntry {
+    byte format;
+    long offset;
+    int count;
+    NormsEntry nested;
+  }
+  
+  static class LoadedNorms {
+    NumericDocValues norms;
+    long ramBytesUsed;
+    Accountable info;
+  }
+
+  @Override
+  public synchronized NormsProducer getMergeInstance() throws IOException {
+    return new Lucene50NormsProducer(this);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + norms.size() + ",active=" + activeCount.get() + ")";
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Tue Sep 30 09:37:52 2014
@@ -225,6 +225,24 @@ public abstract class PerFieldDocValuesF
 
     private final Map<String,DocValuesProducer> fields = new TreeMap<>();
     private final Map<String,DocValuesProducer> formats = new HashMap<>();
+    
+    // clone for merge
+    FieldsReader(FieldsReader other) throws IOException {
+      Map<DocValuesProducer,DocValuesProducer> oldToNew = new IdentityHashMap<>();
+      // First clone all formats
+      for(Map.Entry<String,DocValuesProducer> ent : other.formats.entrySet()) {
+        DocValuesProducer values = ent.getValue().getMergeInstance();
+        formats.put(ent.getKey(), values);
+        oldToNew.put(ent.getValue(), values);
+      }
+
+      // Then rebuild fields:
+      for(Map.Entry<String,DocValuesProducer> ent : other.fields.entrySet()) {
+        DocValuesProducer producer = oldToNew.get(ent.getValue());
+        assert producer != null;
+        fields.put(ent.getKey(), producer);
+      }
+    }
 
     public FieldsReader(final SegmentReadState readState) throws IOException {
 
@@ -259,24 +277,6 @@ public abstract class PerFieldDocValuesF
       }
     }
 
-    private FieldsReader(FieldsReader other) {
-
-      Map<DocValuesProducer,DocValuesProducer> oldToNew = new IdentityHashMap<>();
-      // First clone all formats
-      for(Map.Entry<String,DocValuesProducer> ent : other.formats.entrySet()) {
-        DocValuesProducer values = ent.getValue();
-        formats.put(ent.getKey(), values);
-        oldToNew.put(ent.getValue(), values);
-      }
-
-      // Then rebuild fields:
-      for(Map.Entry<String,DocValuesProducer> ent : other.fields.entrySet()) {
-        DocValuesProducer producer = oldToNew.get(ent.getValue());
-        assert producer != null;
-        fields.put(ent.getKey(), producer);
-      }
-    }
-
     @Override
     public NumericDocValues getNumeric(FieldInfo field) throws IOException {
       DocValuesProducer producer = fields.get(field.name);
@@ -319,11 +319,6 @@ public abstract class PerFieldDocValuesF
     }
 
     @Override
-    public DocValuesProducer clone() {
-      return new FieldsReader(this);
-    }
-
-    @Override
     public long ramBytesUsed() {
       long size = 0;
       for (Map.Entry<String,DocValuesProducer> entry : formats.entrySet()) {
@@ -346,6 +341,11 @@ public abstract class PerFieldDocValuesF
     }
     
     @Override
+    public DocValuesProducer getMergeInstance() throws IOException {
+      return new FieldsReader(this);
+    }
+
+    @Override
     public String toString() {
       return "PerFieldDocValues(formats=" + formats.size() + ")";
     }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Tue Sep 30 09:37:52 2014
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -215,6 +216,24 @@ public abstract class PerFieldPostingsFo
 
     private final Map<String,FieldsProducer> fields = new TreeMap<>();
     private final Map<String,FieldsProducer> formats = new HashMap<>();
+    
+    // clone for merge
+    FieldsReader(FieldsReader other) throws IOException {
+      Map<FieldsProducer,FieldsProducer> oldToNew = new IdentityHashMap<>();
+      // First clone all formats
+      for(Map.Entry<String,FieldsProducer> ent : other.formats.entrySet()) {
+        FieldsProducer values = ent.getValue().getMergeInstance();
+        formats.put(ent.getKey(), values);
+        oldToNew.put(ent.getValue(), values);
+      }
+
+      // Then rebuild fields:
+      for(Map.Entry<String,FieldsProducer> ent : other.fields.entrySet()) {
+        FieldsProducer producer = oldToNew.get(ent.getValue());
+        assert producer != null;
+        fields.put(ent.getKey(), producer);
+      }
+    }
 
     public FieldsReader(final SegmentReadState readState) throws IOException {
 
@@ -294,6 +313,11 @@ public abstract class PerFieldPostingsFo
     }
 
     @Override
+    public FieldsProducer getMergeInstance() throws IOException {
+      return new FieldsReader(this);
+    }
+
+    @Override
     public String toString() {
       return "PerFieldPostings(formats=" + formats.size() + ")";
     }

Added: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java?rev=1628382&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java Tue Sep 30 09:37:52 2014
@@ -0,0 +1,34 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.index.BaseNormsFormatTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests Lucene49NormsFormat
+ */
+public class TestLucene50NormsFormat extends BaseNormsFormatTestCase {
+  private final Codec codec = TestUtil.getDefaultCodec();
+  
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  } 
+}

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java Tue Sep 30 09:37:52 2014
@@ -301,6 +301,11 @@ public class AssertingDocValuesFormat ex
     }
     
     @Override
+    public DocValuesProducer getMergeInstance() throws IOException {
+      return new AssertingDocValuesProducer(in.getMergeInstance(), maxDoc);
+    }
+
+    @Override
     public String toString() {
       return getClass().getSimpleName() + "(" + in.toString() + ")";
     }

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java Tue Sep 30 09:37:52 2014
@@ -124,6 +124,11 @@ public class AssertingNormsFormat extend
     }
     
     @Override
+    public NormsProducer getMergeInstance() throws IOException {
+      return new AssertingNormsProducer(in.getMergeInstance(), maxDoc);
+    }
+    
+    @Override
     public String toString() {
       return getClass().getSimpleName() + "(" + in.toString() + ")";
     }

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java Tue Sep 30 09:37:52 2014
@@ -111,6 +111,11 @@ public final class AssertingPostingsForm
     }
     
     @Override
+    public FieldsProducer getMergeInstance() throws IOException {
+      return new AssertingFieldsProducer(in.getMergeInstance());
+    }
+
+    @Override
     public String toString() {
       return getClass().getSimpleName() + "(" + in.toString() + ")";
     }

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java Tue Sep 30 09:37:52 2014
@@ -97,6 +97,11 @@ public class AssertingStoredFieldsFormat
     }
 
     @Override
+    public StoredFieldsReader getMergeInstance() throws IOException {
+      return new AssertingStoredFieldsReader(in.getMergeInstance(), maxDoc);
+    }
+
+    @Override
     public String toString() {
       return getClass().getSimpleName() + "(" + in.toString() + ")";
     }

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java Tue Sep 30 09:37:52 2014
@@ -96,6 +96,11 @@ public class AssertingTermVectorsFormat 
     }
     
     @Override
+    public TermVectorsReader getMergeInstance() throws IOException {
+      return new AssertingTermVectorsReader(in.getMergeInstance());
+    }
+
+    @Override
     public String toString() {
       return getClass().getSimpleName() + "(" + in.toString() + ")";
     }

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java Tue Sep 30 09:37:52 2014
@@ -139,6 +139,19 @@ public abstract class BaseNormsFormatTes
     }
   }
   
+  public void testSparse() throws Exception {
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      doTestNormsVersusStoredFields(new LongProducer() {
+        @Override
+        long next() {
+          return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : 0;
+        }
+      });
+    }
+  }
+  
   private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception {
     int numDocs = atLeast(500);
     long norms[] = new long[numDocs];