You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/01/05 17:21:19 UTC

svn commit: r1227676 [2/3] - in /lucene/dev/trunk/lucene: ./ contrib/memory/src/java/org/apache/lucene/index/memory/ src/java/org/apache/lucene/codecs/ src/java/org/apache/lucene/codecs/lucene3x/ src/java/org/apache/lucene/codecs/lucene40/ src/java/org...

Copied: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java (from r1226391, lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java?p2=lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsWriter.java&r1=1226391&r2=1227676&rev=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java Thu Jan  5 16:21:17 2012
@@ -18,13 +18,22 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
+import java.util.Set;
 
-import org.apache.lucene.codecs.NormsWriter;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.index.DocValue;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
@@ -32,83 +41,236 @@ import org.apache.lucene.util.IOUtils;
  * Writes plain-text norms
  * <p>
  * <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+ * 
  * @lucene.experimental
  */
-public class SimpleTextNormsWriter extends NormsWriter {
-  private IndexOutput out;
-  private int docid = 0;
-    
+public class SimpleTextNormsConsumer extends PerDocConsumer {
+  
   /** Extension of norms file */
   static final String NORMS_EXTENSION = "len";
+  final static BytesRef END = new BytesRef("END");
+  final static BytesRef FIELD = new BytesRef("field ");
+  final static BytesRef DOC = new BytesRef("  doc ");
+  final static BytesRef NORM = new BytesRef("    norm ");
   
-  private final BytesRef scratch = new BytesRef();
-  
-  final static BytesRef END     = new BytesRef("END");
-  final static BytesRef FIELD   = new BytesRef("field ");
-  final static BytesRef DOC     = new BytesRef("  doc ");
-  final static BytesRef NORM    = new BytesRef("    norm ");
+  private NormsWriter writer;
+
+  private final Directory directory;
+
+  private final String segment;
+
+  private final IOContext context;
+
+  public SimpleTextNormsConsumer(Directory directory, String segment,
+      IOContext context) throws IOException {
+    this.directory = directory;
+    this.segment = segment;
+    this.context = context;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (writer != null) {
+      writer.finish();
+    }
+  }
   
-  public SimpleTextNormsWriter(Directory directory, String segment, IOContext context) throws IOException {
-    final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
-    out = directory.createOutput(normsFileName, context);
+  @Override
+  protected DocValues getDocValuesForMerge(IndexReader reader, FieldInfo info)
+      throws IOException {
+    return reader.normValues(info.name);
   }
 
   @Override
-  public void startField(FieldInfo info) throws IOException {
-    assert info.omitNorms == false;
-    docid = 0;
-    write(FIELD);
-    write(info.name);
-    newLine();
+  protected boolean canMerge(FieldInfo info) {
+    return !info.omitNorms && info.isIndexed;
   }
-    
+
   @Override
-  public void writeNorm(byte norm) throws IOException {
-    write(DOC);
-    write(Integer.toString(docid));
-    newLine();
-    
-    write(NORM);
-    write(norm);
-    newLine();
-    docid++;
+  protected Type getDocValuesType(FieldInfo info) {
+    return Type.BYTES_FIXED_STRAIGHT;
   }
-    
+
   @Override
-  public void finish(int numDocs) throws IOException {
-    if (docid != numDocs) {
-      throw new RuntimeException("mergeNorms produced an invalid result: docCount is " + numDocs
-          + " but only saw " + docid + " file=" + out.toString() + "; now aborting this merge to prevent index corruption");
-    }
-    write(END);
-    newLine();
+  public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
+      throws IOException {
+    return new SimpleTextNormsDocValuesConsumer(fieldInfo);
   }
 
   @Override
-  public void close() throws IOException {
-    try {
-      IOUtils.close(out);
-    } finally {
-      out = null;
+  public void abort() {
+    if (writer != null) {
+      try {
+        writer.abort();
+      } catch (IOException e) {
+      }
     }
   }
-  
-  private void write(String s) throws IOException {
-    SimpleTextUtil.write(out, s, scratch);
+
+  private class SimpleTextNormsDocValuesConsumer extends DocValuesConsumer {
+    // Holds all docID/norm pairs we've seen
+    int[] docIDs = new int[1];
+    byte[] norms = new byte[1];
+    int upto;
+    private final FieldInfo fi;
+
+    public SimpleTextNormsDocValuesConsumer(FieldInfo fieldInfo) {
+      fi = fieldInfo;
+    }
+
+    @Override
+    public void add(int docID, DocValue docValue) throws IOException {
+      add(docID, docValue.getBytes());
+    }
+    
+    protected void add(int docID, BytesRef value) throws IOException {
+      if (docIDs.length <= upto) {
+        assert docIDs.length == upto;
+        docIDs = ArrayUtil.grow(docIDs, 1 + upto);
+      }
+      if (norms.length <= upto) {
+        assert norms.length == upto;
+        norms = ArrayUtil.grow(norms, 1 + upto);
+      }
+      assert value.length == 1;
+      norms[upto] = value.bytes[value.offset];
+      docIDs[upto] = docID;
+      upto++;
+    }
+
+    @Override
+    public void finish(int docCount) throws IOException {
+      final NormsWriter normsWriter = getNormsWriter();
+      boolean success = false;
+      try {
+        int uptoDoc = 0;
+        normsWriter.setNumTotalDocs(docCount);
+        if (upto > 0) {
+          normsWriter.startField(fi);
+          int docID = 0;
+          for (; docID < docCount; docID++) {
+            if (uptoDoc < upto && docIDs[uptoDoc] == docID) {
+              normsWriter.writeNorm(norms[uptoDoc]);
+              uptoDoc++;
+            } else {
+              normsWriter.writeNorm((byte) 0);
+            }
+          }
+          // we should have consumed every norm
+          assert uptoDoc == upto;
+
+        } else {
+          // Fill entire field with default norm:
+          normsWriter.startField(fi);
+          for (; upto < docCount; upto++)
+            normsWriter.writeNorm((byte) 0);
+        }
+        success = true;
+      } finally {
+        if (!success) {
+          normsWriter.abort();
+        }
+      }
+    }
   }
-  
-  private void write(BytesRef bytes) throws IOException {
-    SimpleTextUtil.write(out, bytes);
+
+  public NormsWriter getNormsWriter() throws IOException {
+    if (writer == null) {
+      writer = new NormsWriter(directory, segment, context);
+    }
+    return writer;
   }
-  
-  private void write(byte b) throws IOException {
-    scratch.grow(1);
-    scratch.bytes[scratch.offset] = b;
-    scratch.length = 1;
-    SimpleTextUtil.write(out, scratch);
+
+  private static class NormsWriter {
+
+    private final IndexOutput output;
+    private int numTotalDocs = 0;
+    private int docid = 0;
+
+    private final BytesRef scratch = new BytesRef();
+
+
+    public NormsWriter(Directory directory, String segment, IOContext context)
+        throws IOException {
+      final String normsFileName = IndexFileNames.segmentFileName(segment, "",
+          NORMS_EXTENSION);
+      output = directory.createOutput(normsFileName, context);
+
+    }
+
+    public void startField(FieldInfo info) throws IOException {
+      assert info.omitNorms == false;
+      docid = 0;
+      write(FIELD);
+      write(info.name);
+      newLine();
+    }
+
+    public void writeNorm(byte norm) throws IOException {
+      write(DOC);
+      write(Integer.toString(docid));
+      newLine();
+
+      write(NORM);
+      write(norm);
+      newLine();
+      docid++;
+    }
+
+    public void finish(int numDocs) throws IOException {
+      if (docid != numDocs) {
+        throw new RuntimeException(
+            "mergeNorms produced an invalid result: docCount is " + numDocs
+                + " but only saw " + docid + " file=" + output.toString()
+                + "; now aborting this merge to prevent index corruption");
+      }
+      write(END);
+      newLine();
+    }
+
+    private void write(String s) throws IOException {
+      SimpleTextUtil.write(output, s, scratch);
+    }
+
+    private void write(BytesRef bytes) throws IOException {
+      SimpleTextUtil.write(output, bytes);
+    }
+
+    private void write(byte b) throws IOException {
+      scratch.grow(1);
+      scratch.bytes[scratch.offset] = b;
+      scratch.length = 1;
+      SimpleTextUtil.write(output, scratch);
+    }
+
+    private void newLine() throws IOException {
+      SimpleTextUtil.writeNewline(output);
+    }
+
+    public void setNumTotalDocs(int numTotalDocs) {
+      assert this.numTotalDocs == 0 || numTotalDocs == this.numTotalDocs;
+      this.numTotalDocs = numTotalDocs;
+    }
+
+    public void abort() throws IOException {
+      IOUtils.close(output);
+    }
+
+    public void finish() throws IOException {
+      finish(numTotalDocs);
+      IOUtils.close(output);
+    }
   }
-  
-  private void newLine() throws IOException {
-    SimpleTextUtil.writeNewline(out);
+
+  public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+    FieldInfos fieldInfos = info.getFieldInfos();
+    
+    for (FieldInfo fieldInfo : fieldInfos) {
+      if (!fieldInfo.omitNorms && fieldInfo.isIndexed) {
+        files.add(IndexFileNames.segmentFileName(info.name, "",
+            NORMS_EXTENSION));  
+        break;
+      }
+    }
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java Thu Jan  5 16:21:17 2012
@@ -21,13 +21,12 @@ import java.io.IOException;
 import java.util.Set;
 
 import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.NormsReader;
-import org.apache.lucene.codecs.NormsWriter;
-import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.codecs.PerDocProducer;
+import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
 
 /**
  * plain-text norms format
@@ -36,19 +35,28 @@ import org.apache.lucene.store.IOContext
  * @lucene.experimental
  */
 public class SimpleTextNormsFormat extends NormsFormat {
+  
+  @Override
+  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
+    return new SimpleTextNormsConsumer(state.directory, state.segmentName, state.context);
+  }
 
   @Override
-  public NormsReader normsReader(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context, Directory separateNormsDir) throws IOException {
-    return new SimpleTextNormsReader(dir, info, fields, context);
+  public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
+    return new SimpleTextNormsProducer(state.dir, state.segmentInfo, state.fieldInfos, state.context);
   }
 
   @Override
-  public NormsWriter normsWriter(SegmentWriteState state) throws IOException {
-    return new SimpleTextNormsWriter(state.directory, state.segmentName, state.context);
+  public void files(Directory dir, SegmentInfo info, Set<String> files)
+      throws IOException {
+    SimpleTextNormsConsumer.files(dir, info, files);
+
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
-    SimpleTextNormsReader.files(dir, info, files);
+  public PerDocProducer docsProducer(SegmentReadState state,
+      Directory separateNormsDir) throws IOException {
+    return docsProducer(state);
   }
+   
 }

Copied: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java (from r1226391, lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java?p2=lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsReader.java&r1=1226391&r2=1227676&rev=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java Thu Jan  5 16:21:17 2012
@@ -17,12 +17,21 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.DOC;
+import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.END;
+import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.FIELD;
+import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.NORM;
+import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.NORMS_EXTENSION;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.codecs.NormsReader;
+import org.apache.lucene.codecs.PerDocProducer;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
@@ -33,18 +42,17 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 
-import static org.apache.lucene.codecs.simpletext.SimpleTextNormsWriter.*;
-
 /**
  * Reads plain-text norms
  * <p>
  * <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
  * @lucene.experimental
  */
-public class SimpleTextNormsReader extends NormsReader {
-  private Map<String,byte[]> norms = new HashMap<String,byte[]>();
+public class SimpleTextNormsProducer extends PerDocProducer {
   
-  public SimpleTextNormsReader(Directory directory, SegmentInfo si, FieldInfos fields, IOContext context) throws IOException {
+  Map<String,NormsDocValues> norms = new HashMap<String,NormsDocValues>();
+  
+  public SimpleTextNormsProducer(Directory directory, SegmentInfo si, FieldInfos fields, IOContext context) throws IOException {
     if (fields.hasNorms()) {
       readNorms(directory.openInput(IndexFileNames.segmentFileName(si.name, "", NORMS_EXTENSION), context), si.docCount);
     }
@@ -58,7 +66,7 @@ public class SimpleTextNormsReader exten
       SimpleTextUtil.readLine(in, scratch);
       while (!scratch.equals(END)) {
         assert StringHelper.startsWith(scratch, FIELD);
-        String fieldName = readString(FIELD.length, scratch);
+        final String fieldName = readString(FIELD.length, scratch);
         byte bytes[] = new byte[maxDoc];
         for (int i = 0; i < bytes.length; i++) {
           SimpleTextUtil.readLine(in, scratch);
@@ -67,7 +75,7 @@ public class SimpleTextNormsReader exten
           assert StringHelper.startsWith(scratch, NORM);
           bytes[i] = scratch.bytes[scratch.offset + NORM.length];
         }
-        norms.put(fieldName, bytes);
+        norms.put(fieldName, new NormsDocValues(new Norm(bytes)));
         SimpleTextUtil.readLine(in, scratch);
         assert StringHelper.startsWith(scratch, FIELD) || scratch.equals(END);
       }
@@ -82,11 +90,6 @@ public class SimpleTextNormsReader exten
   }
   
   @Override
-  public byte[] norms(String name) throws IOException {
-    return norms.get(name);
-  }
-  
-  @Override
   public void close() throws IOException {
     norms = null;
   }
@@ -94,7 +97,7 @@ public class SimpleTextNormsReader exten
   static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
     // TODO: This is what SI always did... but we can do this cleaner?
     // like first FI that has norms but doesn't have separate norms?
-    final String normsFileName = IndexFileNames.segmentFileName(info.name, "", SimpleTextNormsWriter.NORMS_EXTENSION);
+    final String normsFileName = IndexFileNames.segmentFileName(info.name, "", SimpleTextNormsConsumer.NORMS_EXTENSION);
     if (dir.fileExists(normsFileName)) {
       files.add(normsFileName);
     }
@@ -103,4 +106,58 @@ public class SimpleTextNormsReader exten
   private String readString(int offset, BytesRef scratch) {
     return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
   }
+
+  @Override
+  public DocValues docValues(String field) throws IOException {
+    return norms.get(field);
+  }
+  
+  private class NormsDocValues extends DocValues {
+    private final Source source;
+    public NormsDocValues(Source source) {
+      this.source = source;
+    }
+
+    @Override
+    public Source load() throws IOException {
+      return source;
+    }
+
+    @Override
+    public Source getDirectSource() throws IOException {
+      return getSource();
+    }
+
+    @Override
+    public Type type() {
+      return Type.BYTES_FIXED_STRAIGHT;
+    }
+  }
+  
+  static final class Norm extends Source {
+    protected Norm(byte[] bytes) {
+      super(Type.BYTES_FIXED_STRAIGHT);
+      this.bytes = bytes;
+    }
+    final byte bytes[];
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      ref.bytes = bytes;
+      ref.offset = docID;
+      ref.length = 1;
+      return ref;
+    }
+
+    @Override
+    public boolean hasArray() {
+      return true;
+    }
+
+    @Override
+    public Object getArray() {
+      return bytes;
+    }
+    
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java Thu Jan  5 16:21:17 2012
@@ -166,9 +166,7 @@ public class SimpleTextStoredFieldsWrite
     try {
       close();
     } catch (IOException ignored) {}
-    try {
-      directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
-    } catch (IOException ignored) {}
+    IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java Thu Jan  5 16:21:17 2012
@@ -149,10 +149,7 @@ public class SimpleTextTermVectorsWriter
     try {
       close();
     } catch (IOException ignored) {}
-    
-    try {
-      directory.deleteFile(IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION));
-    } catch (IOException ignored) {}
+    IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION));
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BaseMultiReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BaseMultiReader.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BaseMultiReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BaseMultiReader.java Thu Jan  5 16:21:17 2012
@@ -118,11 +118,6 @@ abstract class BaseMultiReader<R extends
   }
   
   @Override
-  public synchronized byte[] norms(String field) throws IOException {
-    throw new UnsupportedOperationException("please use MultiNorms.norms, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level norms");
-  }
-  
-  @Override
   public int docFreq(String field, BytesRef t) throws IOException {
     ensureOpen();
     int total = 0;          // sum freqs in segments
@@ -157,4 +152,9 @@ abstract class BaseMultiReader<R extends
   public DocValues docValues(String field) throws IOException {
     throw new UnsupportedOperationException("please use MultiDocValues#getDocValues, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level DocValues");
   }
+  
+  @Override
+  public DocValues normValues(String field) throws IOException {
+    throw new UnsupportedOperationException("please use MultiDocValues#getNormValues, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Norm DocValues ");
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Thu Jan  5 16:21:17 2012
@@ -131,6 +131,17 @@ final class DocFieldProcessor extends Do
       }
     }
     
+    try {
+      PerDocConsumer perDocConsumer = perDocConsumers.get(0);
+      if (perDocConsumer != null) {
+        perDocConsumer.abort();  
+      }
+    } catch (Throwable t) {
+      if (th == null) {
+        th = t;
+      }
+    }
+    
     // If any errors occured, throw it.
     if (th != null) {
       if (th instanceof RuntimeException) throw (RuntimeException) th;
@@ -329,7 +340,6 @@ final class DocFieldProcessor extends Do
       perDocConsumer = dvFormat.docsConsumer(perDocWriteState);
       perDocConsumers.put(0, perDocConsumer);
     }
-
     DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
     fieldInfo.setDocValuesType(valueType);
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverterPerField.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverterPerField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverterPerField.java Thu Jan  5 16:21:17 2012
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.io.Reader;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
@@ -34,7 +33,6 @@ import org.apache.lucene.analysis.tokena
 
 final class DocInverterPerField extends DocFieldConsumerPerField {
 
-  final private DocInverter parent;
   final FieldInfo fieldInfo;
   final InvertedDocConsumerPerField consumer;
   final InvertedDocEndConsumerPerField endConsumer;
@@ -42,7 +40,6 @@ final class DocInverterPerField extends 
   final FieldInvertState fieldState;
 
   public DocInverterPerField(DocInverter parent, FieldInfo fieldInfo) {
-    this.parent = parent;
     this.fieldInfo = fieldInfo;
     docState = parent.docState;
     fieldState = parent.fieldState;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocValue.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocValue.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocValue.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocValue.java Thu Jan  5 16:21:17 2012
@@ -20,8 +20,6 @@ import java.util.Comparator;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.document.DocValuesField;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Type; // javadocs
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -33,36 +31,6 @@ import org.apache.lucene.util.BytesRef;
 public interface DocValue {
 
   /**
-   * Sets the given <code>long</code> value.
-   */
-  public void setInt(long value);
-
-  /**
-   * Sets the given <code>float</code> value.
-   */
-  public void setFloat(float value);
-
-  /**
-   * Sets the given <code>double</code> value.
-   */
-  public void setFloat(double value);
-
-  /**
-   * Sets the given {@link BytesRef} value and the field's {@link Type}. The
-   * comparator for this field is set to <code>null</code>. If a
-   * <code>null</code> comparator is set the default comparator for the given
-   * {@link Type} is used.
-   */
-  public void setBytes(BytesRef value, DocValues.Type type);
-
-  /**
-   * Sets the given {@link BytesRef} value, the field's {@link Type} and the
-   * field's comparator. If the {@link Comparator} is set to <code>null</code>
-   * the default for the given {@link Type} is used instead.
-   */
-  public void setBytes(BytesRef value, DocValues.Type type, Comparator<BytesRef> comp);
-
-  /**
    * Returns the set {@link BytesRef} or <code>null</code> if not set.
    */
   public BytesRef getBytes();
@@ -82,19 +50,4 @@ public interface DocValue {
    */
   public long getInt();
 
-  /**
-   * Sets the {@link BytesRef} comparator for this field. If the field has a
-   * numeric {@link Type} the comparator will be ignored.
-   */
-  public void setBytesComparator(Comparator<BytesRef> comp);
-
-  /**
-   * Sets the {@link Type}
-   */
-  public void setDocValuesType(DocValues.Type type);
-
-  /**
-  * Returns the {@link Type}
-  */
-  public DocValues.Type docValueType();
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java Thu Jan  5 16:21:17 2012
@@ -332,12 +332,6 @@ public class FilterIndexReader extends I
   }
 
   @Override
-  public byte[] norms(String f) throws IOException {
-    ensureOpen();
-    return in.norms(f);
-  }
-
-  @Override
   public int docFreq(String field, BytesRef t) throws IOException {
     ensureOpen();
     return in.docFreq(field, t);
@@ -419,6 +413,12 @@ public class FilterIndexReader extends I
     ensureOpen();
     return in.docValues(field);
   }
+  
+  @Override
+  public DocValues normValues(String field) throws IOException {
+    ensureOpen();
+    return in.normValues(field);
+  }
 
   @Override
   public IndexCommit getIndexCommit() throws IOException {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java Thu Jan  5 16:21:17 2012
@@ -29,6 +29,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
+import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.search.SearcherManager; // javadocs
 import org.apache.lucene.store.*;
 import org.apache.lucene.util.Bits;
@@ -735,7 +736,17 @@ public abstract class IndexReader implem
    *
    * @see org.apache.lucene.document.Field#setBoost(float)
    */
-  public abstract byte[] norms(String field) throws IOException;
+  // TODO: cut over to source once we support other formats like float
+  public final byte[] norms(String field) throws IOException {
+    DocValues docValues = normValues(field);
+    if (docValues != null) {
+      Source source = docValues.getSource();
+      assert source.hasArray(); // TODO cut over to source
+      return (byte[])source.getArray();  
+    }
+    return null;
+  }
+  
 
   /**
    * Returns {@link Fields} for this reader.
@@ -1056,6 +1067,8 @@ public abstract class IndexReader implem
    * using {@link ReaderUtil#gatherSubReaders} and iterate
    * through them yourself. */
   public abstract DocValues docValues(String field) throws IOException;
+  
+  public abstract DocValues normValues(String field) throws IOException;
 
   private volatile Fields fields;
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java Thu Jan  5 16:21:17 2012
@@ -1,5 +1,7 @@
 package org.apache.lucene.index;
 
+import java.io.IOException;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -18,6 +20,6 @@ package org.apache.lucene.index;
  */
 
 abstract class InvertedDocEndConsumerPerField {
-  abstract void finish();
+  abstract void finish() throws IOException;
   abstract void abort();
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiDocValues.java Thu Jan  5 16:21:17 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 import java.io.IOException;
+import java.lang.reflect.Array;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -33,6 +34,13 @@ import org.apache.lucene.util.ReaderUtil
  * @lucene.internal
  */
 public class MultiDocValues extends DocValues {
+  
+  private static DocValuesPuller DEFAULT_PULLER = new DocValuesPuller();
+  private static final DocValuesPuller NORMS_PULLER = new DocValuesPuller() {
+    public DocValues pull(IndexReader reader, String field) throws IOException {
+      return reader.normValues(field);
+    }
+  };
 
   public static class DocValuesSlice {
     public final static DocValuesSlice[] EMPTY_ARRAY = new DocValuesSlice[0];
@@ -46,6 +54,12 @@ public class MultiDocValues extends DocV
       this.length = length;
     }
   }
+  
+  private static class DocValuesPuller {
+    public DocValues pull(IndexReader reader, String field) throws IOException {
+      return reader.docValues(field);
+    }
+  }
 
   private DocValuesSlice[] slices;
   private int[] starts;
@@ -58,7 +72,6 @@ public class MultiDocValues extends DocV
     this.type = promotedType.type();
     this.valueSize = promotedType.getValueSize();
   }
-  
   /**
    * Returns a single {@link DocValues} instance for this field, merging
    * their values on the fly.
@@ -68,15 +81,32 @@ public class MultiDocValues extends DocV
    * sub-readers (using {@link Gather}) and iterate through them yourself.
    */
   public static DocValues getDocValues(IndexReader r, final String field) throws IOException {
+    return getDocValues(r, field, DEFAULT_PULLER);
+  }
+  
+  /**
+   * Returns a single {@link DocValues} instance for this norms field, merging
+   * their values on the fly.
+   * 
+   * <p>
+   * <b>NOTE</b>: this is a slow way to access DocValues. It's better to get the
+   * sub-readers (using {@link Gather}) and iterate through them yourself.
+   */
+  public static DocValues getNormDocValues(IndexReader r, final String field) throws IOException {
+    return getDocValues(r, field, NORMS_PULLER);
+  }
+  
+ 
+  private static DocValues getDocValues(IndexReader r, final String field, final DocValuesPuller puller) throws IOException {
     final IndexReader[] subs = r.getSequentialSubReaders();
     if (subs == null) {
       // already an atomic reader
-      return r.docValues(field);
+      return puller.pull(r, field);
     } else if (subs.length == 0) {
       // no fields
       return null;
     } else if (subs.length == 1) {
-      return getDocValues(subs[0], field);
+      return getDocValues(subs[0], field, puller);
     } else {      
       final List<DocValuesSlice> slices = new ArrayList<DocValuesSlice>();
       
@@ -89,7 +119,7 @@ public class MultiDocValues extends DocV
       new ReaderUtil.Gather(r) {
         @Override
         protected void add(int base, IndexReader r) throws IOException {
-          final DocValues d = r.docValues(field);
+          final DocValues d = puller.pull(r, field);
           if (d != null) {
             TypePromoter incoming = TypePromoter.create(d.type(), d.getValueSize());
             promotedType[0] = promotedType[0].promote(incoming);
@@ -195,6 +225,7 @@ public class MultiDocValues extends DocV
     private final int[] starts;
     private final DocValuesSlice[] slices;
     private boolean direct;
+    private Object cachedArray; // cached array if supported
 
     public MultiSource(DocValuesSlice[] slices, int[] starts, boolean direct, Type type) {
       super(type);
@@ -243,6 +274,77 @@ public class MultiDocValues extends DocV
       final int doc = ensureSource(docID);
       return current.getBytes(doc, bytesRef);
     }
+
+    @Override
+    public boolean hasArray() {
+      boolean oneRealSource = false;
+      for (DocValuesSlice slice : slices) {
+        try {
+          Source source = slice.docValues.getSource();
+          if (source instanceof EmptySource) {
+            /*
+             * empty source marks a gap in the array skip if we encounter one
+             */
+            continue;
+          }
+          oneRealSource = true;
+          if (!source.hasArray()) {
+            return false;
+          }
+        } catch (IOException e) {
+          throw new RuntimeException("load failed", e);
+        }
+      }
+      return oneRealSource;
+    }
+
+    @Override
+    public Object getArray() {
+      if (!hasArray()) {
+        return null;
+      }
+      try {
+        Class<?> componentType = null;
+        Object[] arrays = new Object[slices.length];
+        int numDocs = 0;
+        for (int i = 0; i < slices.length; i++) {
+          DocValuesSlice slice = slices[i];
+          Source source = slice.docValues.getSource();
+          Object array = null;
+          if (!(source instanceof EmptySource)) {
+            // EmptySource is skipped - marks a gap in the array
+            array = source.getArray();
+          }
+          numDocs += slice.length;
+          if (array != null) {
+            if (componentType == null) {
+              componentType = array.getClass().getComponentType();
+            }
+            assert componentType == array.getClass().getComponentType();
+          }
+          arrays[i] = array;
+        }
+        assert componentType != null;
+        synchronized (this) {
+          if (cachedArray != null) {
+            return cachedArray;
+          }
+          final Object globalArray = Array.newInstance(componentType, numDocs);
+
+          for (int i = 0; i < slices.length; i++) {
+            DocValuesSlice slice = slices[i];
+            if (arrays[i] != null) {
+              assert slice.length == Array.getLength(arrays[i]);
+              System.arraycopy(arrays[i], 0, globalArray, slice.start,
+                  slice.length);
+            }
+          }
+          return cachedArray = globalArray;
+        }
+      } catch (IOException e) {
+        throw new RuntimeException("load failed", e);
+      }
+    }
   }
 
   // TODO: this is dup of DocValues.getDefaultSource()?
@@ -269,7 +371,7 @@ public class MultiDocValues extends DocV
     }
   }
   
-  private static class EmptyFixedSource extends Source {
+  private static class EmptyFixedSource extends EmptySource {
     private final int valueSize;
     
     public EmptyFixedSource(Type type, int valueSize) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsConsumer.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsConsumer.java Thu Jan  5 16:21:17 2012
@@ -21,8 +21,12 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
 
+import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.NormsWriter;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.document.DocValuesField;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 // TODO FI: norms could actually be stored as doc store
@@ -33,14 +37,21 @@ import org.apache.lucene.util.IOUtils;
  */
 
 final class NormsConsumer extends InvertedDocEndConsumer {
-  final NormsFormat normsFormat;
+  private final NormsFormat normsFormat;
+  private PerDocConsumer consumer;
+  private final DocumentsWriterPerThread dwpt;
   
   public NormsConsumer(DocumentsWriterPerThread dwpt) {
     normsFormat = dwpt.codec.normsFormat();
+    this.dwpt = dwpt;
   }
 
   @Override
-  public void abort() {}
+  public void abort(){
+    if (consumer != null) {
+      consumer.abort();
+    }
+  }
 
   // We only write the _X.nrm file at flush
   void files(Collection<String> files) {}
@@ -49,50 +60,39 @@ final class NormsConsumer extends Invert
    *  not disabled */
   @Override
   public void flush(Map<FieldInfo,InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
-    if (!state.fieldInfos.hasNorms()) {
-      return;
-    }
-
-    NormsWriter normsOut = null;
     boolean success = false;
+    boolean anythingFlushed = false;
     try {
-      normsOut = normsFormat.normsWriter(state);
-
-      for (FieldInfo fi : state.fieldInfos) {
-        final NormsConsumerPerField toWrite = (NormsConsumerPerField) fieldsToFlush.get(fi);
-        int upto = 0;
-        // we must check the final value of omitNorms for the fieldinfo, it could have 
-        // changed for this field since the first time we added it.
-        if (!fi.omitNorms && toWrite != null && toWrite.upto > 0) {
-          normsOut.startField(fi);
-          int docID = 0;
-          for (; docID < state.numDocs; docID++) {
-            if (upto < toWrite.upto && toWrite.docIDs[upto] == docID) {
-              normsOut.writeNorm(toWrite.norms[upto]);
-              upto++;
-            } else {
-              normsOut.writeNorm((byte) 0);
+      if (state.fieldInfos.hasNorms()) {
+        for (FieldInfo fi : state.fieldInfos) {
+          final NormsConsumerPerField toWrite = (NormsConsumerPerField) fieldsToFlush.get(fi);
+          if (!fi.omitNorms) {
+            if (toWrite != null) {
+              anythingFlushed = true;
+              toWrite.flush(state.numDocs);
+            } else if (fi.isIndexed) {
+              anythingFlushed = true;
+              // we must check the final value of omitNorms for the fieldinfo, it could have 
+              // changed for this field since the first time we added it.
+              final DocValuesConsumer valuesConsumer = newConsumer(new PerDocWriteState(state), fi);
+              final DocValuesField value = new DocValuesField("");
+              value.setBytes(new BytesRef(new byte[] {0x00}), Type.BYTES_FIXED_STRAIGHT);
+              valuesConsumer.add(state.numDocs-1, value);
+              valuesConsumer.finish(state.numDocs);
             }
           }
-
-          // we should have consumed every norm
-          assert upto == toWrite.upto;
-
-          toWrite.reset();
-        } else if (fi.isIndexed && !fi.omitNorms) {
-          // Fill entire field with default norm:
-          normsOut.startField(fi);
-          for(;upto<state.numDocs;upto++)
-            normsOut.writeNorm((byte) 0);
         }
-      }
-      normsOut.finish(state.numDocs);
+      } 
+      
       success = true;
+      if (!anythingFlushed && consumer != null) {
+        consumer.abort();
+      }
     } finally {
       if (success) {
-        IOUtils.close(normsOut);
+        IOUtils.close(consumer);
       } else {
-        IOUtils.closeWhileHandlingException(normsOut);
+        IOUtils.closeWhileHandlingException(consumer);
       }
     }
   }
@@ -106,6 +106,16 @@ final class NormsConsumer extends Invert
   @Override
   InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField,
       FieldInfo fieldInfo) {
-    return new NormsConsumerPerField(docInverterPerField, fieldInfo);
+    return new NormsConsumerPerField(docInverterPerField, fieldInfo, this);
+  }
+  
+  DocValuesConsumer newConsumer(PerDocWriteState perDocWriteState,
+      FieldInfo fieldInfo) throws IOException {
+    if (consumer == null) {
+      consumer = normsFormat.docsConsumer(perDocWriteState);
+    }
+    DocValuesConsumer addValuesField = consumer.addValuesField(
+        Type.BYTES_FIXED_STRAIGHT, fieldInfo);
+    return addValuesField;
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java Thu Jan  5 16:21:17 2012
@@ -1,5 +1,4 @@
 package org.apache.lucene.index;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,65 +15,74 @@ package org.apache.lucene.index;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import java.io.IOException;
 
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.document.DocValuesField;
+import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 
-/** Taps into DocInverter, as an InvertedDocEndConsumer,
- *  which is called at the end of inverting each field.  We
- *  just look at the length for the field (docState.length)
- *  and record the norm. */
-
-final class NormsConsumerPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsConsumerPerField> {
-
-  final FieldInfo fieldInfo;
-  final DocumentsWriterPerThread.DocState docState;
-  final Similarity similarity;
+public class NormsConsumerPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsConsumerPerField> {
+  private final FieldInfo fieldInfo;
+  private final DocumentsWriterPerThread.DocState docState;
+  private final Similarity similarity;
+  private final FieldInvertState fieldState;
+  private DocValuesConsumer consumer;
+  private final DocValuesField value = new DocValuesField("");
+  private final BytesRef spare = new BytesRef(1);
+  private final NormsConsumer parent;
   
-  // Holds all docID/norm pairs we've seen
-  int[] docIDs = new int[1];
-  byte[] norms = new byte[1];
-  int upto;
-
-  final FieldInvertState fieldState;
-
-  public void reset() {
-    // Shrink back if we are overallocated now:
-    docIDs = ArrayUtil.shrink(docIDs, upto);
-    norms = ArrayUtil.shrink(norms, upto);
-    upto = 0;
-  }
-
-  public NormsConsumerPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
+  public NormsConsumerPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo, NormsConsumer parent) {
     this.fieldInfo = fieldInfo;
+    this.parent = parent;
     docState = docInverterPerField.docState;
     fieldState = docInverterPerField.fieldState;
     similarity = docState.similarityProvider.get(fieldInfo.name);
-  }
+    spare.length = 1;
+    spare.offset = 0;
 
-  @Override
-  void abort() {
-    upto = 0;
   }
-
+  @Override
   public int compareTo(NormsConsumerPerField other) {
     return fieldInfo.name.compareTo(other.fieldInfo.name);
   }
-  
+
   @Override
-  void finish() {
+  void finish() throws IOException {
     if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
-      if (docIDs.length <= upto) {
-        assert docIDs.length == upto;
-        docIDs = ArrayUtil.grow(docIDs, 1+upto);
-      }
-      if (norms.length <= upto) {
-        assert norms.length == upto;
-        norms = ArrayUtil.grow(norms, 1+upto);
-      }
-      norms[upto] = similarity.computeNorm(fieldState);
-      docIDs[upto] = docState.docID;
-      upto++;
+      DocValuesConsumer consumer = getConsumer();
+      spare.bytes[0] = similarity.computeNorm(fieldState);
+      value.setBytes(spare, Type.BYTES_FIXED_STRAIGHT);
+      consumer.add(docState.docID, value);
+      
+    }    
+  }
+  
+  void flush(int docCount) throws IOException {
+    DocValuesConsumer consumer = this.consumer;
+    if (consumer == null && fieldInfo.isIndexed) {
+       consumer = getConsumer();
+      spare.bytes[0] = 0x00;
+      value.setBytes(spare, Type.BYTES_FIXED_STRAIGHT);
+      consumer.add(docCount-1, value);
+    } 
+    if (consumer != null) {
+      consumer.finish(docCount);
+    }
+  }
+  
+  private DocValuesConsumer getConsumer() throws IOException {
+    if (consumer == null) {
+      consumer = parent.newConsumer(docState.docWriter.newPerDocWriteState(""), fieldInfo);
     }
+    return consumer;
   }
+  
+
+  @Override
+  void abort() {
+    //
+  }
+
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelReader.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelReader.java Thu Jan  5 16:21:17 2012
@@ -48,7 +48,7 @@ public class ParallelReader extends Inde
   private SortedMap<String,IndexReader> fieldToReader = new TreeMap<String,IndexReader>();
   private Map<IndexReader,Collection<String>> readerToFields = new HashMap<IndexReader,Collection<String>>();
   private List<IndexReader> storedFieldReaders = new ArrayList<IndexReader>();
-  private Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
+  private Map<String, DocValues> normsCache = new HashMap<String,DocValues>();
   private final ReaderContext topLevelReaderContext = new AtomicReaderContext(this);
   private int maxDoc;
   private int numDocs;
@@ -337,27 +337,6 @@ public class ParallelReader extends Inde
   }
 
   @Override
-  public synchronized byte[] norms(String field) throws IOException {
-    ensureOpen();
-    IndexReader reader = fieldToReader.get(field);
-
-    if (reader==null)
-      return null;
-    
-    byte[] bytes = normsCache.get(field);
-    if (bytes != null)
-      return bytes;
-    if (!hasNorms(field))
-      return null;
-    if (normsCache.containsKey(field)) // cached omitNorms, not missing key
-      return null;
-
-    bytes = MultiNorms.norms(reader, field);
-    normsCache.put(field, bytes);
-    return bytes;
-  }
-
-  @Override
   public int docFreq(String field, BytesRef term) throws IOException {
     ensureOpen();
     IndexReader reader = fieldToReader.get(field);
@@ -427,4 +406,16 @@ public class ParallelReader extends Inde
     IndexReader reader = fieldToReader.get(field);
     return reader == null ? null : MultiDocValues.getDocValues(reader, field);
   }
+  
+  // TODO: I suspect this is completely untested!!!!!
+  @Override
+  public synchronized DocValues normValues(String field) throws IOException {
+    DocValues values = normsCache.get(field);
+    if (values == null) {
+      IndexReader reader = fieldToReader.get(field);
+      values = reader == null ? null : MultiDocValues.getNormDocValues(reader, field);
+      normsCache.put(field, values);
+    } 
+    return values;
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java Thu Jan  5 16:21:17 2012
@@ -25,7 +25,6 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.NormsReader;
 import org.apache.lucene.codecs.PerDocProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
@@ -54,7 +53,7 @@ final class SegmentCoreReaders {
   
   final FieldsProducer fields;
   final PerDocProducer perDocProducer;
-  final NormsReader norms;
+  final PerDocProducer norms;
 
   final Directory dir;
   final Directory cfsDir;
@@ -120,7 +119,7 @@ final class SegmentCoreReaders {
       // ask codec for its Norms: 
       // TODO: since we don't write any norms file if there are no norms,
       // kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
-      norms = codec.normsFormat().normsReader(cfsDir, si, fieldInfos, context, dir);
+      norms = codec.normsFormat().docsProducer(segmentReadState, dir);
       perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
 
       final Directory storeDir;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Thu Jan  5 16:21:17 2012
@@ -27,7 +27,6 @@ import java.util.Map;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.NormsWriter;
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.codecs.TermVectorsWriter;
@@ -125,8 +124,7 @@ final class SegmentMerger {
     mergePerDoc(segmentWriteState);
     
     if (mergeState.fieldInfos.hasNorms()) {
-      int numMerged = mergeNorms(segmentWriteState);
-      assert numMerged == mergeState.mergedDocCount;
+      mergeNorms(segmentWriteState);
     }
 
     if (mergeState.fieldInfos.hasVectors()) {
@@ -379,20 +377,24 @@ final class SegmentMerger {
         }
       }
   }
-
-  private int mergeNorms(SegmentWriteState segmentWriteState) throws IOException {
-    final NormsWriter writer = codec.normsFormat().normsWriter(segmentWriteState);
-    
+  
+  private void mergeNorms(SegmentWriteState segmentWriteState) throws IOException {
+    final PerDocConsumer docsConsumer = codec.normsFormat()
+        .docsConsumer(new PerDocWriteState(segmentWriteState));
+    // TODO: remove this check when 3.x indexes are no longer supported
+    // (3.x indexes don't have docvalues)
+    if (docsConsumer == null) {
+      return;
+    }
     boolean success = false;
     try {
-      int numMerged = writer.merge(mergeState);
+      docsConsumer.merge(mergeState);
       success = true;
-      return numMerged;
     } finally {
       if (success) {
-        IOUtils.close(writer);
+        IOUtils.close(docsConsumer);
       } else {
-        IOUtils.closeWhileHandlingException(writer);
+        IOUtils.closeWhileHandlingException(docsConsumer);
       }
     }
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Thu Jan  5 16:21:17 2012
@@ -18,16 +18,15 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.codecs.PerDocProducer;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.FieldCache; // javadocs
 import org.apache.lucene.store.IOContext;
@@ -262,12 +261,6 @@ public final class SegmentReader extends
     return fi != null && fi.isIndexed && !fi.omitNorms;
   }
 
-  @Override
-  public byte[] norms(String field) throws IOException {
-    ensureOpen();
-    return core.norms.norms(field);
-  }
-
   /** @lucene.internal */
   public TermVectorsReader getTermVectorsReader() {
     ensureOpen();
@@ -352,6 +345,17 @@ public final class SegmentReader extends
     }
     return perDoc.docValues(field);
   }
+  
+  @Override
+  public DocValues normValues(String field) throws IOException {
+    ensureOpen();
+    final PerDocProducer perDoc = core.norms;
+    if (perDoc == null) {
+      return null;
+    }
+    return perDoc.docValues(field);
+  }
+  
 
   /**
    * Called when the shared core for this SegmentReader

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java Thu Jan  5 16:21:17 2012
@@ -34,7 +34,7 @@ import org.apache.lucene.index.MultiRead
  * IndexReader#getSequentialSubReaders}) to emulate an
  * atomic reader.  This requires implementing the postings
  * APIs on-the-fly, using the static methods in {@link
- * MultiFields}, {@link MultiNorms}, {@link MultiDocValues}, 
+ * MultiFields}, {@link MultiDocValues}, 
  * by stepping through the sub-readers to merge fields/terms, 
  * appending docs, etc.
  *
@@ -53,7 +53,7 @@ import org.apache.lucene.index.MultiRead
 public final class SlowMultiReaderWrapper extends FilterIndexReader {
 
   private final ReaderContext readerContext;
-  private final Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
+  private final Map<String, DocValues> normsCache = new HashMap<String, DocValues>();
   
   public SlowMultiReaderWrapper(IndexReader other) {
     super(other);
@@ -76,7 +76,17 @@ public final class SlowMultiReaderWrappe
     ensureOpen();
     return MultiDocValues.getDocValues(in, field);
   }
-
+  
+  @Override
+  public synchronized DocValues normValues(String field) throws IOException {
+    ensureOpen();
+    DocValues values = normsCache.get(field);
+    if (values == null) {
+      values = MultiDocValues.getNormDocValues(in, field);
+      normsCache.put(field, values);
+    }
+    return values;
+  }
   @Override
   public Bits getLiveDocs() {
     ensureOpen();
@@ -87,22 +97,6 @@ public final class SlowMultiReaderWrappe
   public IndexReader[] getSequentialSubReaders() {
     return null;
   }
-
-  @Override
-  public synchronized byte[] norms(String field) throws IOException {
-    ensureOpen();
-    byte[] bytes = normsCache.get(field);
-    if (bytes != null)
-      return bytes;
-    if (!hasNorms(field))
-      return null;
-    if (normsCache.containsKey(field)) // cached omitNorms, not missing key
-      return null;
-    
-    bytes = MultiNorms.norms(in, field);
-    normsCache.put(field, bytes);
-    return bytes;
-  }
   
   @Override
   public ReaderContext getTopReaderContext() {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java Thu Jan  5 16:21:17 2012
@@ -91,7 +91,7 @@ final class CompoundFileWriter implement
   // all entries that are written to a sep. file but not yet moved into CFS
   private final Queue<FileEntry> pendingEntries = new LinkedList<FileEntry>();
   private boolean closed = false;
-  private volatile IndexOutput dataOut;
+  private IndexOutput dataOut;
   private final AtomicBoolean outputTaken = new AtomicBoolean(false);
   final String entryTableName;
   final String dataFileName;
@@ -113,16 +113,25 @@ final class CompoundFileWriter implement
         IndexFileNames.stripExtension(name), "",
         IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
     dataFileName = name;
-    boolean success = false;
-    try {
-      dataOut = directory.createOutput(dataFileName, IOContext.DEFAULT);
-      dataOut.writeVInt(FORMAT_CURRENT);
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(dataOut);
+    
+  }
+  
+  private synchronized IndexOutput getOutput() throws IOException {
+    if (dataOut == null) {
+      IndexOutput dataOutput = null;
+      boolean success = false;
+      try {
+        dataOutput = directory.createOutput(dataFileName, IOContext.DEFAULT);
+        dataOutput.writeVInt(FORMAT_CURRENT);
+        dataOut = dataOutput;
+        success = true;
+      } finally {
+        if (!success) {
+          IOUtils.closeWhileHandlingException(dataOutput);
+        }
       }
-    }
+    } 
+    return dataOut;
   }
 
   /** Returns the directory of the compound file. */
@@ -154,6 +163,7 @@ final class CompoundFileWriter implement
       }
       closed = true;
       // open the compound stream
+      getOutput();
       assert dataOut != null;
       long finalLength = dataOut.getFilePointer();
       assert assertFileLength(finalLength, dataOut);
@@ -246,7 +256,7 @@ final class CompoundFileWriter implement
       seenIDs.add(id);
       final DirectCFSIndexOutput out;
       if (outputTaken.compareAndSet(false, true)) {
-        out = new DirectCFSIndexOutput(dataOut, entry, false);
+        out = new DirectCFSIndexOutput(getOutput(), entry, false);
         outputLocked = true;
         success = true;
       } else {
@@ -280,7 +290,7 @@ final class CompoundFileWriter implement
       try {
         while (!pendingEntries.isEmpty()) {
           FileEntry entry = pendingEntries.poll();
-          copyFileEntry(dataOut, entry);
+          copyFileEntry(getOutput(), entry);
           entries.put(entry.file, entry);
         }
       } finally {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IOUtils.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IOUtils.java Thu Jan  5 16:21:17 2012
@@ -30,6 +30,8 @@ import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 
+import org.apache.lucene.store.Directory;
+
 /** This class emulates the new Java 7 "Try-With-Resources" statement.
  * Remove once Lucene is on Java 7.
  * @lucene.internal */
@@ -318,6 +320,16 @@ public final class IOUtils {
       }
     }
   }
+  
+  public static void deleteFilesIgnoringExceptions(Directory dir, String... files) {
+    for (String name : files) {
+      try {
+        dir.deleteFile(name);
+      } catch (IOException ignored) {
+        // ignore
+      }
+    }
+  }
 
 
 }

Copied: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java (from r1226391, lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java?p2=lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsWriter.java&r1=1226391&r2=1227676&rev=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java Thu Jan  5 16:21:17 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene40;
+package org.apache.lucene.codecs.preflexrw;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,111 +20,258 @@ package org.apache.lucene.codecs.lucene4
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.lucene.codecs.NormsWriter;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.index.DocValue;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
-public class Lucene40NormsWriter extends NormsWriter {
-  private IndexOutput out;
-  private int normCount = 0;
+/**
+ * Writes and Merges Lucene 3.x norms format
+ * @lucene.experimental
+ */
+class PreFlexNormsConsumer extends PerDocConsumer {
   
   /** norms header placeholder */
-  static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
+  private static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
   
   /** Extension of norms file */
-  static final String NORMS_EXTENSION = "nrm";
+  private static final String NORMS_EXTENSION = "nrm";
   
   /** Extension of separate norms file
    * @deprecated */
   @Deprecated
-  static final String SEPARATE_NORMS_EXTENSION = "s";
+  private static final String SEPARATE_NORMS_EXTENSION = "s";
+
+  private final Directory directory;
+
+  private final String segment;
+
+  private final IOContext context;
+
+  private NormsWriter writer;
   
-  public Lucene40NormsWriter(Directory directory, String segment, IOContext context) throws IOException {
-    final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
-    boolean success = false;
-    try {
-      out = directory.createOutput(normsFileName, context);
-      out.writeBytes(NORMS_HEADER, 0, NORMS_HEADER.length);
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(out);
-      }
+  public PreFlexNormsConsumer(Directory directory, String segment, IOContext context){
+    this.directory = directory;
+    this.segment = segment;
+    this.context = context;
+  }
+
+  @Override
+  public void merge(MergeState mergeState) throws IOException {
+    getNormsWriter().merge(mergeState);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (writer != null) {
+      writer.finish();
     }
   }
 
   @Override
-  public void startField(FieldInfo info) throws IOException {
-    assert info.omitNorms == false;
-    normCount++;
+  public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
+      throws IOException {
+    return new Lucene3xNormsDocValuesConsumer(fieldInfo);
   }
   
-  @Override
-  public void writeNorm(byte norm) throws IOException {
-    out.writeByte(norm);
+  class Lucene3xNormsDocValuesConsumer extends DocValuesConsumer {
+    // Holds all docID/norm pairs we've seen
+    private int[] docIDs = new int[1];
+    private byte[] norms = new byte[1];
+    private int upto;
+    private final FieldInfo fi;
+    
+    Lucene3xNormsDocValuesConsumer(FieldInfo fieldInfo) {
+      fi = fieldInfo;
+    }
+
+    @Override
+    public void finish(int docCount) throws IOException {
+      final NormsWriter normsWriter = getNormsWriter();
+      boolean success = false;
+      try {
+        int uptoDoc = 0;
+        normsWriter.setNumTotalDocs(docCount);
+        if (upto > 0) {
+          normsWriter.startField(fi);
+          int docID = 0;
+          for (; docID < docCount; docID++) {
+            if (uptoDoc < upto && docIDs[uptoDoc] == docID) {
+              normsWriter.writeNorm(norms[uptoDoc]);
+              uptoDoc++;
+            } else {
+              normsWriter.writeNorm((byte) 0);
+            }
+          }
+          // we should have consumed every norm
+          assert uptoDoc == upto;
+  
+        } else {
+          // Fill entire field with default norm:
+          normsWriter.startField(fi);
+          for (; upto < docCount; upto++)
+            normsWriter.writeNorm((byte) 0);
+        }
+        success = true;
+      } finally {
+        if (!success) {
+          normsWriter.abort();
+        }
+      }
+    }
+    
+    @Override
+    public void add(int docID, DocValue docValue) throws IOException {
+      add(docID, docValue.getBytes());
+    }
+    
+    protected void add(int docID, BytesRef value) throws IOException {
+      if (docIDs.length <= upto) {
+        assert docIDs.length == upto;
+        docIDs = ArrayUtil.grow(docIDs, 1 + upto);
+      }
+      if (norms.length <= upto) {
+        assert norms.length == upto;
+        norms = ArrayUtil.grow(norms, 1 + upto);
+      }
+      assert value.length == 1;
+      norms[upto] = value.bytes[value.offset];
+      
+      docIDs[upto] = docID;
+      upto++;
+    }
+    
+    
   }
   
-  @Override
-  public void finish(int numDocs) throws IOException {
-    if (4+normCount*(long)numDocs != out.getFilePointer()) {
-      throw new RuntimeException(".nrm file size mismatch: expected=" + (4+normCount*(long)numDocs) + " actual=" + out.getFilePointer());
+  public NormsWriter getNormsWriter() throws IOException {
+    if (writer == null) {
+      writer = new NormsWriter(directory, segment, context);
     }
+    return writer;
   }
   
-  /** we override merge and bulk-merge norms when there are no deletions */
-  @Override
-  public int merge(MergeState mergeState) throws IOException {
-    int numMergedDocs = 0;
-    for (FieldInfo fi : mergeState.fieldInfos) {
-      if (fi.isIndexed && !fi.omitNorms) {
-        startField(fi);
-        int numMergedDocsForField = 0;
-        for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
-          final int maxDoc = reader.reader.maxDoc();
-          byte normBuffer[] = reader.reader.norms(fi.name);
-          if (normBuffer == null) {
-            // Can be null if this segment doesn't have
-            // any docs with this field
-            normBuffer = new byte[maxDoc];
-            Arrays.fill(normBuffer, (byte)0);
-          }
-          if (reader.liveDocs == null) {
-            //optimized case for segments without deleted docs
-            out.writeBytes(normBuffer, maxDoc);
-            numMergedDocsForField += maxDoc;
-          } else {
-            // this segment has deleted docs, so we have to
-            // check for every doc if it is deleted or not
-            final Bits liveDocs = reader.liveDocs;
-            for (int k = 0; k < maxDoc; k++) {
-              if (liveDocs.get(k)) {
-                numMergedDocsForField++;
-                out.writeByte(normBuffer[k]);
+  private static class NormsWriter {
+    
+    private final IndexOutput output;
+    private int normCount = 0;
+    private int numTotalDocs = 0;
+    
+    public NormsWriter(Directory directory, String segment, IOContext context) throws IOException {
+      final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
+      boolean success = false;
+      IndexOutput out = null;
+      try {
+        out = directory.createOutput(normsFileName, context);
+        output = out;
+        output.writeBytes(NORMS_HEADER, 0, NORMS_HEADER.length);
+        success = true;
+      } finally {
+        if (!success) {
+          IOUtils.closeWhileHandlingException(out);
+        }
+      }
+      
+    }
+    
+    
+    public void setNumTotalDocs(int numTotalDocs) {
+      assert this.numTotalDocs == 0 || numTotalDocs == this.numTotalDocs;
+      this.numTotalDocs = numTotalDocs;
+    }
+    
+    public void startField(FieldInfo info) throws IOException {
+      assert info.omitNorms == false;
+      normCount++;
+    }
+    
+    public void writeNorm(byte norm) throws IOException {
+      output.writeByte(norm);
+    }
+    
+    public void abort() throws IOException {
+      IOUtils.close(output);
+    }
+    
+    public void finish() throws IOException {
+      IOUtils.close(output);
+      
+      if (4+normCount*(long)numTotalDocs != output.getFilePointer()) {
+        throw new IOException(".nrm file size mismatch: expected=" + (4+normCount*(long)numTotalDocs) + " actual=" + output.getFilePointer());
+      }
+    }
+    // TODO: we can actually use the defaul DV merge here and drop this specific stuff entirely
+    /** we override merge and bulk-merge norms when there are no deletions */
+    public void merge(MergeState mergeState) throws IOException {
+      int numMergedDocs = 0;
+      for (FieldInfo fi : mergeState.fieldInfos) {
+        if (fi.isIndexed && !fi.omitNorms) {
+          startField(fi);
+          int numMergedDocsForField = 0;
+          for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
+            final int maxDoc = reader.reader.maxDoc();
+            byte[] normBuffer;
+            DocValues normValues = reader.reader.normValues(fi.name);
+            if (normValues == null) {
+              // Can be null if this segment doesn't have
+              // any docs with this field
+              normBuffer = new byte[maxDoc];
+              Arrays.fill(normBuffer, (byte)0);
+            } else {
+              Source directSource = normValues.getDirectSource();
+              assert directSource.hasArray();
+              normBuffer = (byte[]) directSource.getArray();
+            }
+            if (reader.liveDocs == null) {
+              //optimized case for segments without deleted docs
+              output.writeBytes(normBuffer, maxDoc);
+              numMergedDocsForField += maxDoc;
+            } else {
+              // this segment has deleted docs, so we have to
+              // check for every doc if it is deleted or not
+              final Bits liveDocs = reader.liveDocs;
+              for (int k = 0; k < maxDoc; k++) {
+                if (liveDocs.get(k)) {
+                  numMergedDocsForField++;
+                  output.writeByte(normBuffer[k]);
+                }
               }
             }
+            mergeState.checkAbort.work(maxDoc);
           }
-          mergeState.checkAbort.work(maxDoc);
+          assert numMergedDocs == 0 || numMergedDocs == numMergedDocsForField;
+          numMergedDocs = numMergedDocsForField;
         }
-        assert numMergedDocs == 0 || numMergedDocs == numMergedDocsForField;
-        numMergedDocs = numMergedDocsForField;
       }
+      this.numTotalDocs = numMergedDocs;
     }
-    finish(numMergedDocs);
-    return numMergedDocs;
   }
 
   @Override
-  public void close() throws IOException {
+  public void abort() {
     try {
-      IOUtils.close(out);
-    } finally {
-      out = null;
+      try {
+        if (writer != null) {
+          writer.abort();
+        }
+      } finally {
+        directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
+            NORMS_EXTENSION));
+      }
+    } catch (IOException e) {
+      // ignore
     }
   }
 }

Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java?rev=1227676&r1=1227675&r2=1227676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java Thu Jan  5 16:21:17 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.codecs.preflex
  * limitations under the License.
  */
 
+import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
 import org.apache.lucene.util.LuceneTestCase;
@@ -27,7 +28,8 @@ import org.apache.lucene.util.LuceneTest
  */
 public class PreFlexRWCodec extends Lucene3xCodec {
   private final PostingsFormat postings = new PreFlexRWPostingsFormat();
-
+  private final NormsFormat norms = new PreFlexRWNormsFormat();
+  
   @Override
   public PostingsFormat postingsFormat() {
     if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
@@ -36,4 +38,13 @@ public class PreFlexRWCodec extends Luce
       return super.postingsFormat();
     }
   }
+
+  @Override
+  public NormsFormat normsFormat() {
+    if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
+      return norms;
+    } else {
+      return super.normsFormat();
+    }
+  }
 }

Added: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWNormsFormat.java?rev=1227676&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWNormsFormat.java (added)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWNormsFormat.java Thu Jan  5 16:21:17 2012
@@ -0,0 +1,31 @@
+package org.apache.lucene.codecs.preflexrw;
+/**
+ * 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.PerDocConsumer;
+import org.apache.lucene.codecs.lucene3x.Lucene3xNormsFormat;
+import org.apache.lucene.index.PerDocWriteState;
+
+public class PreFlexRWNormsFormat extends Lucene3xNormsFormat {
+
+  @Override
+  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
+    return new PreFlexNormsConsumer(state.directory, state.segmentName, state.context);
+  }
+
+}