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 2011/12/04 20:24:09 UTC

svn commit: r1210185 - in /lucene/dev/branches/lucene3606/lucene: contrib/misc/src/java/org/apache/lucene/index/codecs/appending/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/index/codecs/lucene3...

Author: rmuir
Date: Sun Dec  4 19:24:08 2011
New Revision: 1210185

URL: http://svn.apache.org/viewvc?rev=1210185&view=rev
Log:
LUCENE-3606: move IW's norms writing to codec

Added:
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumer.java
      - copied, changed from r1210104, lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriter.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java
      - copied, changed from r1210104, lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsFormat.java   (with props)
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsWriter.java   (with props)
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsFormat.java   (with props)
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsWriter.java   (with props)
Removed:
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriter.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java
Modified:
    lucene/dev/branches/lucene3606/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentNorms.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
    lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
    lucene/dev/branches/lucene3606/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java

Modified: lucene/dev/branches/lucene3606/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java (original)
+++ lucene/dev/branches/lucene3606/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java Sun Dec  4 19:24:08 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.a
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosFormat;
+import org.apache.lucene.index.codecs.NormsFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
@@ -27,6 +28,7 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.index.codecs.lucene40.Lucene40FieldInfosFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40DocValuesFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40NormsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40StoredFieldsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40TermVectorsFormat;
 
@@ -47,6 +49,7 @@ public class AppendingCodec extends Code
   private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat();
   private final TermVectorsFormat vectors = new Lucene40TermVectorsFormat();
   private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
+  private final NormsFormat norms = new Lucene40NormsFormat();
   
   @Override
   public PostingsFormat postingsFormat() {
@@ -77,4 +80,9 @@ public class AppendingCodec extends Code
   public FieldInfosFormat fieldInfosFormat() {
     return fieldInfos;
   }
+
+  @Override
+  public NormsFormat normsFormat() {
+    return norms;
+  }
 }

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Sun Dec  4 19:24:08 2011
@@ -52,7 +52,7 @@ import org.apache.lucene.util.InfoStream
  * are processing the document).
  *
  * Other consumers, eg {@link FreqProxTermsWriter} and
- * {@link NormsWriter}, buffer bytes in RAM and flush only
+ * {@link NormsConsumer}, buffer bytes in RAM and flush only
  * when a new segment is produced.
 
  * Once we have used our allowed RAM buffer, or the number

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sun Dec  4 19:24:08 2011
@@ -79,7 +79,7 @@ public class DocumentsWriterPerThread {
 
       final InvertedDocConsumer  termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
                                                            new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
-      final NormsWriter normsWriter = new NormsWriter();
+      final NormsConsumer normsWriter = new NormsConsumer(documentsWriterPerThread);
       final DocInverter docInverter = new DocInverter(documentsWriterPerThread.docState, termsHash, normsWriter);
       return new DocFieldProcessor(documentsWriterPerThread, docInverter);
     }

Copied: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumer.java (from r1210104, lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumer.java?p2=lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumer.java&p1=lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriter.java&r1=1210104&r2=1210185&rev=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumer.java Sun Dec  4 19:24:08 2011
@@ -21,8 +21,8 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
 
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.IOContext.Context;
+import org.apache.lucene.index.codecs.NormsFormat;
+import org.apache.lucene.index.codecs.NormsWriter;
 import org.apache.lucene.util.IOUtils;
 
 // TODO FI: norms could actually be stored as doc store
@@ -32,8 +32,12 @@ import org.apache.lucene.util.IOUtils;
  *  merges all of these together into a single _X.nrm file.
  */
 
-final class NormsWriter extends InvertedDocEndConsumer {
-
+final class NormsConsumer extends InvertedDocEndConsumer {
+  final NormsFormat normsFormat;
+  
+  public NormsConsumer(DocumentsWriterPerThread dwpt) {
+    normsFormat = dwpt.codec.normsFormat();
+  }
 
   @Override
   public void abort() {}
@@ -49,27 +53,23 @@ final class NormsWriter extends Inverted
       return;
     }
 
-    final String normsFileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.NORMS_EXTENSION);
-    IndexOutput normsOut = state.directory.createOutput(normsFileName, state.context);
+    NormsWriter normsOut = null;
     boolean success = false;
     try {
-      normsOut.writeBytes(SegmentNorms.NORMS_HEADER, 0, SegmentNorms.NORMS_HEADER.length);
-
-      int normCount = 0;
+      normsOut = normsFormat.normsWriter(state);
 
       for (FieldInfo fi : state.fieldInfos) {
-        final NormsWriterPerField toWrite = (NormsWriterPerField) fieldsToFlush.get(fi);
+        final NormsConsumerPerField toWrite = (NormsConsumerPerField) fieldsToFlush.get(fi);
         int upto = 0;
-        if (toWrite != null && toWrite.upto > 0) {
-          normCount++;
-
+        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.writeByte(toWrite.norms[upto]);
+              normsOut.writeNorm(toWrite.norms[upto]);
               upto++;
             } else {
-              normsOut.writeByte((byte) 0);
+              normsOut.writeNorm((byte) 0);
             }
           }
 
@@ -78,14 +78,13 @@ final class NormsWriter extends Inverted
 
           toWrite.reset();
         } else if (fi.isIndexed && !fi.omitNorms) {
-          normCount++;
           // Fill entire field with default norm:
+          normsOut.startField(fi);
           for(;upto<state.numDocs;upto++)
-            normsOut.writeByte((byte) 0);
+            normsOut.writeNorm((byte) 0);
         }
-
-        assert 4+normCount*state.numDocs == normsOut.getFilePointer() : ".nrm file size mismatch: expected=" + (4+normCount*state.numDocs) + " actual=" + normsOut.getFilePointer();
       }
+      normsOut.finish(state.numDocs);
       success = true;
     } finally {
       if (success) {
@@ -105,6 +104,6 @@ final class NormsWriter extends Inverted
   @Override
   InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField,
       FieldInfo fieldInfo) {
-    return new NormsWriterPerField(docInverterPerField, fieldInfo);
+    return new NormsConsumerPerField(docInverterPerField, fieldInfo);
   }
 }

Copied: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java (from r1210104, lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java?p2=lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java&p1=lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java&r1=1210104&r2=1210185&rev=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java Sun Dec  4 19:24:08 2011
@@ -25,7 +25,7 @@ import org.apache.lucene.util.ArrayUtil;
  *  just look at the length for the field (docState.length)
  *  and record the norm. */
 
-final class NormsWriterPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsWriterPerField> {
+final class NormsConsumerPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsConsumerPerField> {
 
   final FieldInfo fieldInfo;
   final DocumentsWriterPerThread.DocState docState;
@@ -45,7 +45,7 @@ final class NormsWriterPerField extends 
     upto = 0;
   }
 
-  public NormsWriterPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
+  public NormsConsumerPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
     this.fieldInfo = fieldInfo;
     docState = docInverterPerField.docState;
     fieldState = docInverterPerField.fieldState;
@@ -57,7 +57,7 @@ final class NormsWriterPerField extends 
     upto = 0;
   }
 
-  public int compareTo(NormsWriterPerField other) {
+  public int compareTo(NormsConsumerPerField other) {
     return fieldInfo.name.compareTo(other.fieldInfo.name);
   }
   

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Sun Dec  4 19:24:08 2011
@@ -31,6 +31,7 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.StoredFieldsWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.TermVectorsWriter;
+import org.apache.lucene.index.codecs.lucene40.Lucene40NormsWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
@@ -324,6 +325,7 @@ final class SegmentMerger {
       }
   }
 
+  // TODO: implement merge in normsformat instead.
   private void mergeNorms() throws IOException {
     IndexOutput output = null;
     boolean success = false;
@@ -332,7 +334,7 @@ final class SegmentMerger {
         if (fi.isIndexed && !fi.omitNorms) {
           if (output == null) {
             output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION), context);
-            output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
+            output.writeBytes(Lucene40NormsWriter.NORMS_HEADER, Lucene40NormsWriter.NORMS_HEADER.length);
           }
           for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
             final int maxDoc = reader.reader.maxDoc();

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentNorms.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentNorms.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentNorms.java Sun Dec  4 19:24:08 2011
@@ -35,9 +35,6 @@ import org.apache.lucene.store.IndexOutp
 
 final class SegmentNorms implements Cloneable {
 
-  /** norms header placeholder */
-  static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
-
   int refCount = 1;
 
   // If this instance is a clone, the originalNorm

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/SegmentReader.java Sun Dec  4 19:24:08 2011
@@ -33,6 +33,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.codecs.StoredFieldsReader;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.codecs.TermVectorsReader;
+import org.apache.lucene.index.codecs.lucene40.Lucene40NormsWriter;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.Bits;
@@ -540,7 +541,8 @@ public class SegmentReader extends Index
   }
 
   private void openNorms(Directory cfsDir, IOContext context) throws IOException {
-    long nextNormSeek = SegmentNorms.NORMS_HEADER.length; //skip header (header unused for now)
+    boolean normsInitiallyEmpty = norms.isEmpty(); // only used for assert
+    long nextNormSeek = Lucene40NormsWriter.NORMS_HEADER.length; //skip header (header unused for now)
     int maxDoc = maxDoc();
     for (FieldInfo fi : core.fieldInfos) {
       if (norms.containsKey(fi.name)) {
@@ -585,7 +587,7 @@ public class SegmentReader extends Index
           if (isUnversioned) {
             normSeek = 0;
           } else {
-            normSeek = SegmentNorms.NORMS_HEADER.length;
+            normSeek = Lucene40NormsWriter.NORMS_HEADER.length;
           }
         }
 
@@ -593,6 +595,8 @@ public class SegmentReader extends Index
         nextNormSeek += maxDoc; // increment also if some norms are separate
       }
     }
+    // nocommit: change to a real check? see LUCENE-3619
+    assert singleNormStream == null || !normsInitiallyEmpty || nextNormSeek == singleNormStream.length();
   }
 
   // for testing only

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/Codec.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/Codec.java Sun Dec  4 19:24:08 2011
@@ -51,6 +51,8 @@ public abstract class Codec implements N
     // TODO: segmentInfosFormat should be allowed to declare additional files
     // if it wants, in addition to segments_N
     docValuesFormat().files(dir, info, files);
+    // nocommit: not yet
+    //normsFormat().files(dir, info, files);
   }
   
   /** Encodes/decodes postings */
@@ -71,6 +73,9 @@ public abstract class Codec implements N
   /** Encodes/decodes segments file */
   public abstract SegmentInfosFormat segmentInfosFormat();
   
+  /** Encodes/decodes document normalization values */
+  public abstract NormsFormat normsFormat();
+  
   /** looks up a codec by name */
   public static Codec forName(String name) {
     return loader.lookup(name);

Added: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsFormat.java?rev=1210185&view=auto
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsFormat.java (added)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsFormat.java Sun Dec  4 19:24:08 2011
@@ -0,0 +1,33 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * 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.Set;
+
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.Directory;
+
+/**
+ * format for normalization factors
+ */
+public abstract class NormsFormat {
+  public abstract NormsWriter normsWriter(SegmentWriteState state) throws IOException;
+  public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
+}

Added: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsWriter.java?rev=1210185&view=auto
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsWriter.java (added)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/NormsWriter.java Sun Dec  4 19:24:08 2011
@@ -0,0 +1,34 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * 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.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInfo;
+
+// simple api just for now before switching to docvalues apis
+public abstract class NormsWriter implements Closeable {
+
+  // TODO: I think IW should set info.normValueType from Similarity,
+  // and then this method just returns DocValuesConsumer
+  public abstract void startField(FieldInfo info) throws IOException;
+  public abstract void writeNorm(byte norm) throws IOException;
+  public abstract void finish(int numDocs) throws IOException;
+  
+}

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java Sun Dec  4 19:24:08 2011
@@ -26,6 +26,7 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosFormat;
+import org.apache.lucene.index.codecs.NormsFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
@@ -33,6 +34,7 @@ import org.apache.lucene.index.codecs.Po
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
 import org.apache.lucene.index.codecs.TermVectorsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40FieldInfosFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40NormsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40SegmentInfosFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40StoredFieldsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40TermVectorsFormat;
@@ -62,6 +64,9 @@ public class Lucene3xCodec extends Codec
   // this way IR.commit fails on delete/undelete/setNorm/etc ?
   private final SegmentInfosFormat infosFormat = new Lucene40SegmentInfosFormat();
   
+  // TODO: this should really be a different impl
+  private final NormsFormat normsFormat = new Lucene40NormsFormat();
+  
   // 3.x doesn't support docvalues
   private final DocValuesFormat docValuesFormat = new DocValuesFormat() {
     @Override
@@ -107,4 +112,9 @@ public class Lucene3xCodec extends Codec
   public SegmentInfosFormat segmentInfosFormat() {
     return infosFormat;
   }
+
+  @Override
+  public NormsFormat normsFormat() {
+    return normsFormat;
+  }
 }

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java Sun Dec  4 19:24:08 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.l
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosFormat;
+import org.apache.lucene.index.codecs.NormsFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
@@ -40,6 +41,7 @@ public class Lucene40Codec extends Codec
   private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
   private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
   private final SegmentInfosFormat infosFormat = new Lucene40SegmentInfosFormat();
+  private final NormsFormat normsFormat = new Lucene40NormsFormat();
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
     @Override
     public PostingsFormat getPostingsFormatForField(String field) {
@@ -81,6 +83,11 @@ public class Lucene40Codec extends Codec
     return infosFormat;
   }
 
+  @Override
+  public NormsFormat normsFormat() {
+    return normsFormat;
+  }
+
   /** Returns the postings format that should be used for writing 
    *  new segments of <code>field</code>.
    *  

Added: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsFormat.java?rev=1210185&view=auto
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsFormat.java (added)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsFormat.java Sun Dec  4 19:24:08 2011
@@ -0,0 +1,41 @@
+package org.apache.lucene.index.codecs.lucene40;
+
+/**
+ * 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.Set;
+
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.codecs.NormsFormat;
+import org.apache.lucene.index.codecs.NormsWriter;
+import org.apache.lucene.store.Directory;
+
+public class Lucene40NormsFormat extends NormsFormat {
+
+  @Override
+  public NormsWriter normsWriter(SegmentWriteState state) throws IOException {
+    return new Lucene40NormsWriter(state.directory, state.segmentName, state.context);
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+    // nocommit: hairy calculations involving .s files, etc
+  }
+  
+}

Added: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsWriter.java?rev=1210185&view=auto
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsWriter.java (added)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40NormsWriter.java Sun Dec  4 19:24:08 2011
@@ -0,0 +1,78 @@
+package org.apache.lucene.index.codecs.lucene40;
+
+/**
+ * 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.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.codecs.NormsWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+
+public class Lucene40NormsWriter extends NormsWriter {
+  private IndexOutput out;
+  private int normCount = 0;
+  
+  /** norms header placeholder */
+  // nocommit: not public
+  public static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
+  
+  public Lucene40NormsWriter(Directory directory, String segment, IOContext context) throws IOException {
+    final String normsFileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.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);
+      }
+    }
+  }
+
+  @Override
+  public void startField(FieldInfo info) throws IOException {
+    assert info.omitNorms == false;
+    normCount++;
+  }
+  
+  @Override
+  public void writeNorm(byte norm) throws IOException {
+    out.writeByte(norm);
+  }
+  
+  @Override
+  public void finish(int numDocs) throws IOException {
+    if (4+normCount*numDocs != out.getFilePointer()) {
+      throw new RuntimeException(".nrm file size mismatch: expected=" + (4+normCount*numDocs) + " actual=" + out.getFilePointer());
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.close(out);
+    } finally {
+      out = null;
+    }
+  }
+}

Modified: lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Sun Dec  4 19:24:08 2011
@@ -20,11 +20,13 @@ package org.apache.lucene.index.codecs.s
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosFormat;
+import org.apache.lucene.index.codecs.NormsFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.TermVectorsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40DocValuesFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40NormsFormat;
 
 /**
  * plain text index format.
@@ -40,6 +42,8 @@ public final class SimpleTextCodec exten
   private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
   // TODO: need a plain-text impl
   private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
+  // TODO: need a plain-text impl (using the above)
+  private final NormsFormat normsFormat = new Lucene40NormsFormat();
   
   public SimpleTextCodec() {
     super("SimpleText");
@@ -74,4 +78,9 @@ public final class SimpleTextCodec exten
   public SegmentInfosFormat segmentInfosFormat() {
     return segmentInfos;
   }
+
+  @Override
+  public NormsFormat normsFormat() {
+    return normsFormat;
+  }
 }

Modified: lucene/dev/branches/lucene3606/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3606/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1210185&r1=1210184&r2=1210185&view=diff
==============================================================================
--- lucene/dev/branches/lucene3606/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene3606/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Sun Dec  4 19:24:08 2011
@@ -33,6 +33,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosFormat;
+import org.apache.lucene.index.codecs.NormsFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
@@ -40,6 +41,7 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.index.codecs.lucene40.Lucene40FieldInfosFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40DocValuesFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40NormsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40SegmentInfosFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40StoredFieldsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40TermVectorsFormat;
@@ -1122,6 +1124,11 @@ public class TestAddIndexes extends Luce
     public SegmentInfosFormat segmentInfosFormat() {
       return new Lucene40SegmentInfosFormat();
     }
+
+    @Override
+    public NormsFormat normsFormat() {
+      return new Lucene40NormsFormat();
+    }
   }
   
   /*