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/06/08 22:49:32 UTC

svn commit: r1348236 - in /lucene/dev/trunk/lucene/core/src: java/org/apache/lucene/codecs/ java/org/apache/lucene/codecs/lucene40/values/ java/org/apache/lucene/codecs/simpletext/ java/org/apache/lucene/document/ java/org/apache/lucene/index/ test/org...

Author: simonw
Date: Fri Jun  8 20:49:30 2012
New Revision: 1348236

URL: http://svn.apache.org/viewvc?rev=1348236&view=rev
Log:
LUCENE-4087: throw exceptions on illegal DocValue#Type changes in IW

Added:
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesTypeCompatibility.java   (with props)
Modified:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DirectSource.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TypePromoter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTypePromotion.java

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java Fri Jun  8 20:49:30 2012
@@ -106,6 +106,10 @@ public abstract class DocValuesArraySour
     public byte[] getArray() {
       return values;
     }
+    
+    public double getFloat(int docID) {
+      return getInt(docID);
+    }
 
     @Override
     public long getInt(int docID) {
@@ -167,6 +171,10 @@ public abstract class DocValuesArraySour
     public short[] getArray() {
       return values;
     }
+    
+    public double getFloat(int docID) {
+      return getInt(docID);
+    }
 
     @Override
     public long getInt(int docID) {
@@ -223,6 +231,10 @@ public abstract class DocValuesArraySour
     public int[] getArray() {
       return values;
     }
+    
+    public double getFloat(int docID) {
+      return getInt(docID);
+    }
 
     @Override
     public long getInt(int docID) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Fri Jun  8 20:49:30 2012
@@ -76,7 +76,20 @@ public abstract class DocValuesConsumer 
    * @throws IOException
    */
   public abstract void finish(int docCount) throws IOException;
-
+  
+  
+  /**
+   * Returns the value size this consumer accepts or <tt>-1</tt> iff this
+   * consumer is value size agnostic ie. accepts variable length values.
+   * <p>
+   * NOTE: the return value is undefined until the consumer has successfully
+   * consumed at least one value.
+   * 
+   * @return the value size this consumer accepts or <tt>-1</tt> iff this
+   *         consumer is value size agnostic ie. accepts variable length values.
+   */
+  public abstract int getValueSize();
+  
   /**
    * Merges the given {@link org.apache.lucene.index.MergeState} into
    * this {@link DocValuesConsumer}.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.codecs.lucene40.values;
 
-/**
+/*
  * 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.
@@ -473,6 +473,10 @@ public final class Bytes {
       }
     }
     
+    public int getValueSize() {
+      return size;
+    }
+    
     // Important that we get docCount, in case there were
     // some last docs that we didn't see
     @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DirectSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DirectSource.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DirectSource.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DirectSource.java Fri Jun  8 20:49:30 2012
@@ -43,6 +43,11 @@ abstract class DirectSource extends Sour
       toNumeric = new ShortToLong();
       break;
     case FLOAT_32:
+      toNumeric = new BytesToFloat();
+      break;
+    case FLOAT_64:
+      toNumeric = new BytesToDouble();
+      break;
     case FIXED_INTS_32:
       toNumeric = new IntToLong();
       break;
@@ -103,7 +108,6 @@ abstract class DirectSource extends Sour
     long toLong(IndexInput input) throws IOException {
       return input.readByte();
     }
-
   }
 
   private static final class ShortToLong extends ToNumeric {
@@ -118,11 +122,30 @@ abstract class DirectSource extends Sour
     long toLong(IndexInput input) throws IOException {
       return input.readInt();
     }
+  }
+  
+  private static final class BytesToFloat extends ToNumeric {
+    @Override
+    long toLong(IndexInput input) throws IOException {
+      throw new UnsupportedOperationException("ints are not supported");
+    }
 
     double toDouble(IndexInput input) throws IOException {
       return Float.intBitsToFloat(input.readInt());
     }
   }
+  
+  private static final class BytesToDouble extends ToNumeric {
+    @Override
+    long toLong(IndexInput input) throws IOException {
+      throw new UnsupportedOperationException("ints are not supported");
+    }
+
+    double toDouble(IndexInput input) throws IOException {
+      return Double.longBitsToDouble(input.readLong());
+    }
+  }
+
 
   private static final class LongToLong extends ToNumeric {
     @Override
@@ -131,7 +154,7 @@ abstract class DirectSource extends Sour
     }
 
     double toDouble(IndexInput input) throws IOException {
-      return Double.longBitsToDouble(input.readLong());
+      throw new UnsupportedOperationException("doubles are not supported");
     }
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java Fri Jun  8 20:49:30 2012
@@ -67,6 +67,7 @@ class FixedDerefBytesImpl {
       idxOut.writeInt(numValues);
       writeIndex(idxOut, docCount, numValues, docToEntry);
     }
+
   }
 
   public static class FixedDerefReader extends BytesReaderBase {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.codecs.lucene40.values;
 
-/**
+/*
  * 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.
@@ -129,6 +129,11 @@ class FixedStraightBytesImpl {
         out.writeBytes(zeros, zeros.length);
       }
     }
+    
+    @Override
+    public int getValueSize() {
+      return size;
+    }
   }
 
   static class Writer extends FixedBytesWriterBase {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.codecs.lucene40.values;
 
-/**
+/*
  * 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.
@@ -232,6 +232,11 @@ class VarStraightBytesImpl {
     public long ramBytesUsed() {
       return bytesUsed.get();
     }
+
+    @Override
+    public int getValueSize() {
+      return -1;
+    }
   }
 
   public static class VarStraightReader extends BytesReaderBase {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java Fri Jun  8 20:49:30 2012
@@ -1,5 +1,5 @@
 package org.apache.lucene.codecs.simpletext;
-/**
+/*
  * 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
@@ -55,7 +55,7 @@ public class SimpleTextDocValuesConsumer
   protected final Type type;
   protected final BytesRefHash hash;
   private int[] ords;
-  private int fixedSize = Integer.MIN_VALUE;
+  private int valueSize = Integer.MIN_VALUE;
   private BytesRef zeroBytes;
   private final String segmentSuffix;
   
@@ -137,12 +137,12 @@ public class SimpleTextDocValuesConsumer
       throw new RuntimeException("should not reach this line");
     }
     
-    if (fixedSize == Integer.MIN_VALUE) {
+    if (valueSize == Integer.MIN_VALUE) {
       assert maxDocId == -1;
-      fixedSize = vSize;
+      valueSize = vSize;
     } else {
-      if (fixedSize != vSize) {
-        throw new IllegalArgumentException("value size must be " + fixedSize + " but was: " + vSize);
+      if (valueSize != vSize) {
+        throw new IllegalArgumentException("value size must be " + valueSize + " but was: " + vSize);
       }
     }
     maxDocId = Math.max(docID, maxDocId);
@@ -178,7 +178,7 @@ public class SimpleTextDocValuesConsumer
       SimpleTextUtil.write(output, getHeader());
       SimpleTextUtil.writeNewline(output);
       SimpleTextUtil.write(output, VALUE_SIZE);
-      SimpleTextUtil.write(output, Integer.toString(this.fixedSize), scratch);
+      SimpleTextUtil.write(output, Integer.toString(this.valueSize), scratch);
       SimpleTextUtil.writeNewline(output);
       prepareFlush(docCount);
       for (int i = 0; i < docCount; i++) {
@@ -253,8 +253,8 @@ public class SimpleTextDocValuesConsumer
       case BYTES_FIXED_SORTED:
       case BYTES_FIXED_STRAIGHT:
         if(zeroBytes == null) {
-          assert fixedSize > 0;
-          zeroBytes = new BytesRef(new byte[fixedSize]);
+          assert valueSize > 0;
+          zeroBytes = new BytesRef(new byte[valueSize]);
         }
         SimpleTextUtil.write(output, zeroBytes);
         break;
@@ -286,4 +286,9 @@ public class SimpleTextDocValuesConsumer
   protected Type getType() {
     return type;
   }
+
+  @Override
+  public int getValueSize() {
+    return valueSize;
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -32,6 +32,8 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * 
+ * @see DocValues for further information
  * */
 
 public class ByteDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -37,6 +37,8 @@ import org.apache.lucene.util.BytesRef;
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * 
+ * @see DocValues for further information
  * */
 
 public class DerefBytesDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -32,6 +32,8 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * 
+ * @see DocValues for further information
  * */
 
 public class DoubleDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -32,6 +32,7 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * @see DocValues for further information
  * */
 
 public class FloatDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -32,6 +32,7 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * @see DocValues for further information
  * */
 
 public class IntDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -32,6 +32,7 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * @see DocValues for further information
  * */
 
 public class LongDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -35,6 +35,8 @@ import org.apache.lucene.index.AtomicRea
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * 
+ * @see DocValues for further information
  * */
 
 public class PackedLongDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -32,6 +32,8 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * 
+ * @see DocValues for further information
  * */
 
 public class ShortDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -33,6 +33,8 @@ import org.apache.lucene.util.BytesRef;
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * 
+ * @see DocValues for further information
  * */
 
 public class SortedBytesDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.document;
 
-/**
+/*
  * 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.
@@ -36,6 +36,8 @@ import org.apache.lucene.util.BytesRef;
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
+ * 
+ * @see DocValues for further information
  * */
 
 public class StraightBytesDocValuesField extends Field {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 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
  * this work for additional information regarding copyright ownership.
@@ -29,6 +29,7 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
+import org.apache.lucene.index.TypePromoter.TypeCompatibility;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils;
@@ -80,7 +81,7 @@ final class DocFieldProcessor extends Do
     fieldsWriter.flush(state);
     consumer.flush(childFields, state);
 
-    for (DocValuesConsumerAndDocID consumer : docValues.values()) {
+    for (DocValuesConsumerHolder consumer : docValues.values()) {
       consumer.docValuesConsumer.finish(state.segmentInfo.getDocCount());
     }
     
@@ -271,7 +272,26 @@ final class DocFieldProcessor extends Do
       }
       final DocValues.Type dvType = field.fieldType().docValueType();
       if (dvType != null) {
-        docValuesConsumer(dvType, docState, fp.fieldInfo).add(docState.docID, field);
+        DocValuesConsumerHolder docValuesConsumer = docValuesConsumer(dvType,
+            docState, fp.fieldInfo);
+        DocValuesConsumer consumer = docValuesConsumer.docValuesConsumer;
+        if (docValuesConsumer.compatibility == null) {
+          consumer.add(docState.docID, field);
+          docValuesConsumer.compatibility = new TypeCompatibility(dvType,
+              consumer.getValueSize());
+        } else if (docValuesConsumer.compatibility.isCompatible(dvType,
+            TypePromoter.getValueSize(dvType, field.binaryValue()))) {
+          consumer.add(docState.docID, field);
+        } else {
+          docValuesConsumer.compatibility.isCompatible(dvType,
+              TypePromoter.getValueSize(dvType, field.binaryValue()));
+          TypeCompatibility compatibility = docValuesConsumer.compatibility;
+          throw new IllegalArgumentException("Incompatible DocValues type: "
+              + dvType.name() + " size: "
+              + TypePromoter.getValueSize(dvType, field.binaryValue())
+              + " expected: " + " type: " + compatibility.getBaseType()
+              + " size: " + compatibility.getBaseSize());
+        }
       }
     }
 
@@ -308,30 +328,31 @@ final class DocFieldProcessor extends Do
     }
   }
 
-  private static class DocValuesConsumerAndDocID {
+  private static class DocValuesConsumerHolder {
     // Only used to enforce that same DV field name is never
     // added more than once per doc:
-    public int docID;
+    int docID;
     final DocValuesConsumer docValuesConsumer;
+    TypeCompatibility compatibility;
 
-    public DocValuesConsumerAndDocID(DocValuesConsumer docValuesConsumer) {
+    public DocValuesConsumerHolder(DocValuesConsumer docValuesConsumer) {
       this.docValuesConsumer = docValuesConsumer;
     }
   }
 
-  final private Map<String, DocValuesConsumerAndDocID> docValues = new HashMap<String, DocValuesConsumerAndDocID>();
+  final private Map<String, DocValuesConsumerHolder> docValues = new HashMap<String, DocValuesConsumerHolder>();
   private PerDocConsumer perDocConsumer;
 
-  DocValuesConsumer docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo) 
+  DocValuesConsumerHolder docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo) 
       throws IOException {
-    DocValuesConsumerAndDocID docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
+    DocValuesConsumerHolder docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
     if (docValuesConsumerAndDocID != null) {
       if (docState.docID == docValuesConsumerAndDocID.docID) {
         throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
       }
       assert docValuesConsumerAndDocID.docID < docState.docID;
       docValuesConsumerAndDocID.docID = docState.docID;
-      return docValuesConsumerAndDocID.docValuesConsumer;
+      return docValuesConsumerAndDocID;
     }
 
     if (perDocConsumer == null) {
@@ -345,9 +366,10 @@ final class DocFieldProcessor extends Do
     assert fieldInfo.getDocValuesType() == null || fieldInfo.getDocValuesType() == valueType;
     fieldInfo.setDocValuesType(valueType);
 
-    docValuesConsumerAndDocID = new DocValuesConsumerAndDocID(docValuesConsumer);
+    docValuesConsumerAndDocID = new DocValuesConsumerHolder(docValuesConsumer);
     docValuesConsumerAndDocID.docID = docState.docID;
     docValues.put(fieldInfo.name, docValuesConsumerAndDocID);
-    return docValuesConsumer;
+    return docValuesConsumerAndDocID;
   }
+  
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 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
  * this work for additional information regarding copyright ownership.
@@ -48,6 +48,17 @@ import org.apache.lucene.util.packed.Pac
  * IndexReader.
  * <p>
  * {@link DocValues} are fully integrated into the {@link DocValuesFormat} API.
+ * <p>
+ * NOTE: DocValues is a strongly typed per-field API. Type changes within an
+ * indexing session can result in exceptions if the type has changed in a way that
+ * the previously give type for a field can't promote the value without losing
+ * information. For instance a field initially indexed with {@link Type#FIXED_INTS_32}
+ * can promote a value with {@link Type#FIXED_INTS_8} but can't promote
+ * {@link Type#FIXED_INTS_64}. During segment merging type-promotion exceptions are suppressed. 
+ * Fields will be promoted to their common denominator or automatically transformed
+ * into a 3rd type like {@link Type#BYTES_VAR_STRAIGHT} to prevent data loss and merge exceptions.
+ * This behavior is considered <i>best-effort</i> might change in future releases.
+ * </p>
  * 
  * @see Type for limitations and default implementation documentation
  * @see ByteDocValuesField for adding byte values to the index

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Fri Jun  8 20:49:30 2012
@@ -185,13 +185,7 @@ final class SegmentMerger {
     if (previous == null) {
       previous = TypePromoter.getIdentityPromoter();
     }
-    TypePromoter promoted = previous.promote(incoming);
-    if (promoted == null) {
-      // type is incompatible: promote to BYTES_VAR_STRAIGHT
-      return TypePromoter.create(DocValues.Type.BYTES_VAR_STRAIGHT, TypePromoter.VAR_TYPE_VALUE_SIZE);
-    } else {
-      return promoted;
-    }
+    return previous.promote(incoming);
   }
 
   // NOTE: this is actually merging all the fieldinfos

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TypePromoter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TypePromoter.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TypePromoter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TypePromoter.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 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
  * this work for additional information regarding copyright ownership.
@@ -20,26 +20,27 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.util.BytesRef;
 
 // TODO: maybe we should not automagically promote
 // types... and instead require a given field always has the
 // same type?
 
 /**
- * Type promoter that promotes {@link DocValues} during merge based on
- * their {@link Type} and {@link #getValueSize()}
+ * Type promoter that promotes {@link DocValues} during merge based on their
+ * {@link Type} and {@link #getValueSize()}
  * 
  * @lucene.internal
  */
 class TypePromoter {
-
-  private final static Map<Integer, Type> FLAGS_MAP = new HashMap<Integer, Type>();
+  
+  private final static Map<Integer,Type> FLAGS_MAP = new HashMap<Integer,Type>();
   private static final TypePromoter IDENTITY_PROMOTER = new IdentityTypePromoter();
   public static final int VAR_TYPE_VALUE_SIZE = -1;
-
-  private static final int IS_INT = 1 << 0;
+  
+  private static final int IS_INT = 1 << 0 | 1 << 2;
   private static final int IS_BYTE = 1 << 1;
-  private static final int IS_FLOAT = 1 << 2;
+  private static final int IS_FLOAT = 1 << 2 ;
   /* VAR & FIXED == VAR */
   private static final int IS_VAR = 1 << 3;
   private static final int IS_FIXED = 1 << 3 | 1 << 4;
@@ -50,15 +51,17 @@ class TypePromoter {
   private static final int IS_DEREF = 1 << 5 | 1 << 6;
   private static final int IS_SORTED = 1 << 7;
   /* more bits wins (int16 & int32 == int32) */
-  private static final int IS_8_BIT = 1 << 8 | 1 << 9 | 1 << 10 | 1 << 11;
-  private static final int IS_16_BIT = 1 << 9 | 1 << 10 | 1 << 11;
-  private static final int IS_32_BIT = 1 << 10 | 1 << 11;
+  private static final int IS_8_BIT = 1 << 8 | 1 << 9 | 1 << 10 | 1 << 11 | 1 << 12 | 1 << 13; // 8
+  private static final int IS_16_BIT = 1 << 9 | 1 << 10 | 1 << 11 | 1 << 12 | 1 << 13; // 9
+  private static final int IS_32_BIT = 1 << 10 | 1 << 11 | 1 << 13;
   private static final int IS_64_BIT = 1 << 11;
-
-  private final Type type;
-  private final int flags;
-  private final int valueSize;
-
+  private static final int IS_32_BIT_FLOAT = 1 << 12 | 1 << 13;
+  private static final int IS_64_BIT_FLOAT = 1 << 13;
+  
+  private Type type;
+  private int flags;
+  private int valueSize;
+  
   /**
    * Returns a positive value size if this {@link TypePromoter} represents a
    * fixed variant, otherwise <code>-1</code>
@@ -69,19 +72,26 @@ class TypePromoter {
   public int getValueSize() {
     return valueSize;
   }
-
+  
   static {
     for (Type type : Type.values()) {
       TypePromoter create = create(type, VAR_TYPE_VALUE_SIZE);
       FLAGS_MAP.put(create.flags, type);
     }
   }
-
+  
+  /**
+   * Creates a new {@link TypePromoter}
+   * 
+   */
+  protected TypePromoter() {}
+  
   /**
    * Creates a new {@link TypePromoter}
    * 
    * @param type
    *          the {@link Type} this promoter represents
+   * 
    * @param flags
    *          the promoters flags
    * @param valueSize
@@ -92,7 +102,25 @@ class TypePromoter {
     this.flags = flags;
     this.valueSize = valueSize;
   }
-
+  
+  /**
+   * Resets the {@link TypePromoter}
+   * 
+   * @param type
+   *          the {@link Type} this promoter represents
+   * 
+   * @param flags
+   *          the promoters flags
+   * @param valueSize
+   *          the value size if {@link #IS_FIXED} or <code>-1</code> otherwise.
+   */
+  protected TypePromoter set(Type type, int flags, int valueSize) {
+    this.type = type;
+    this.flags = flags;
+    this.valueSize = valueSize;
+    return this;
+  }
+  
   /**
    * Creates a new promoted {@link TypePromoter} based on this and the given
    * {@link TypePromoter} or <code>null</code> iff the {@link TypePromoter} 
@@ -105,23 +133,29 @@ class TypePromoter {
    *         {@link TypePromoter} aren't compatible.
    */
   public TypePromoter promote(TypePromoter promoter) {
-
+    return promote(promoter, newPromoter());
+  }
+  
+  private TypePromoter promote(TypePromoter promoter, TypePromoter spare) {
     int promotedFlags = promoter.flags & this.flags;
-    TypePromoter promoted = create(FLAGS_MAP.get(promotedFlags), valueSize);
+    TypePromoter promoted = reset(FLAGS_MAP.get(promotedFlags), valueSize,
+        spare);
     if (promoted == null) {
-      return promoted;
+      return TypePromoter.create(DocValues.Type.BYTES_VAR_STRAIGHT,
+          TypePromoter.VAR_TYPE_VALUE_SIZE);
     }
-    if ((promoted.flags & IS_BYTE) != 0 && (promoted.flags & IS_FIXED) == IS_FIXED) {
+    if ((promoted.flags & IS_BYTE) != 0
+        && (promoted.flags & IS_FIXED) == IS_FIXED) {
       if (this.valueSize == promoter.valueSize) {
         return promoted;
       }
-      return create(FLAGS_MAP.get(promoted.flags & PROMOTE_TO_VAR_SIZE_MASK),
-          VAR_TYPE_VALUE_SIZE);
+      return reset(FLAGS_MAP.get(promoted.flags & PROMOTE_TO_VAR_SIZE_MASK),
+          VAR_TYPE_VALUE_SIZE, spare);
     }
-
+    
     return promoted;
   }
-
+  
   /**
    * Returns the {@link Type} of this {@link TypePromoter}
    * 
@@ -130,12 +164,51 @@ class TypePromoter {
   public Type type() {
     return type;
   }
-
+  
+  private boolean isTypeCompatible(TypePromoter promoter) {
+    int promotedFlags = promoter.flags & this.flags;
+    return (promotedFlags & 0x7) > 0;
+  }
+  
+  private boolean isBytesCompatible(TypePromoter promoter) {
+    int promotedFlags = promoter.flags & this.flags;
+    return (promotedFlags & IS_BYTE) > 0
+        && (promotedFlags & (IS_FIXED | IS_VAR)) > 0;
+  }
+  
+  private boolean isNumericSizeCompatible(TypePromoter promoter) {
+    int promotedFlags = promoter.flags & this.flags;
+    return (promotedFlags & IS_BYTE) == 0
+        && (((promotedFlags & IS_FIXED) > 0 && (promotedFlags & (IS_8_BIT)) > 0) || (promotedFlags & IS_VAR) > 0);
+  }
+  
   @Override
   public String toString() {
     return "TypePromoter [type=" + type + ", sizeInBytes=" + valueSize + "]";
   }
-
+  
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + flags;
+    result = prime * result + ((type == null) ? 0 : type.hashCode());
+    result = prime * result + valueSize;
+    return result;
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (obj == null) return false;
+    if (getClass() != obj.getClass()) return false;
+    TypePromoter other = (TypePromoter) obj;
+    if (flags != other.flags) return false;
+    if (type != other.type) return false;
+    if (valueSize != other.valueSize) return false;
+    return true;
+  }
+  
   /**
    * Creates a new {@link TypePromoter} for the given type and size per value.
    * 
@@ -147,47 +220,84 @@ class TypePromoter {
    * @return a new {@link TypePromoter}
    */
   public static TypePromoter create(Type type, int valueSize) {
+    return reset(type, valueSize, new TypePromoter());
+  }
+  
+  private static TypePromoter reset(Type type, int valueSize,
+      TypePromoter promoter) {
     if (type == null) {
       return null;
     }
     switch (type) {
-    case BYTES_FIXED_DEREF:
-      return new TypePromoter(type, IS_BYTE | IS_FIXED | IS_DEREF, valueSize);
-    case BYTES_FIXED_SORTED:
-      return new TypePromoter(type, IS_BYTE | IS_FIXED | IS_SORTED, valueSize);
-    case BYTES_FIXED_STRAIGHT:
-      return new TypePromoter(type, IS_BYTE | IS_FIXED | IS_STRAIGHT, valueSize);
-    case BYTES_VAR_DEREF:
-      return new TypePromoter(type, IS_BYTE | IS_VAR | IS_DEREF, VAR_TYPE_VALUE_SIZE);
-    case BYTES_VAR_SORTED:
-      return new TypePromoter(type, IS_BYTE | IS_VAR | IS_SORTED, VAR_TYPE_VALUE_SIZE);
-    case BYTES_VAR_STRAIGHT:
-      return new TypePromoter(type, IS_BYTE | IS_VAR | IS_STRAIGHT, VAR_TYPE_VALUE_SIZE);
-    case FIXED_INTS_16:
-      return new TypePromoter(type,
-          IS_INT | IS_FIXED | IS_STRAIGHT | IS_16_BIT, valueSize);
-    case FIXED_INTS_32:
-      return new TypePromoter(type,
-          IS_INT | IS_FIXED | IS_STRAIGHT | IS_32_BIT, valueSize);
-    case FIXED_INTS_64:
-      return new TypePromoter(type,
-          IS_INT | IS_FIXED | IS_STRAIGHT | IS_64_BIT, valueSize);
-    case FIXED_INTS_8:
-      return new TypePromoter(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_8_BIT,
-          valueSize);
-    case FLOAT_32:
-      return new TypePromoter(type, IS_FLOAT | IS_FIXED | IS_STRAIGHT
-          | IS_32_BIT, valueSize);
-    case FLOAT_64:
-      return new TypePromoter(type, IS_FLOAT | IS_FIXED | IS_STRAIGHT
-          | IS_64_BIT, valueSize);
-    case VAR_INTS:
-      return new TypePromoter(type, IS_INT | IS_VAR | IS_STRAIGHT, VAR_TYPE_VALUE_SIZE);
-    default:
-      throw new IllegalStateException();
+      case BYTES_FIXED_DEREF:
+        return promoter.set(type, IS_BYTE | IS_FIXED | IS_DEREF, valueSize);
+      case BYTES_FIXED_SORTED:
+        return promoter.set(type, IS_BYTE | IS_FIXED | IS_SORTED, valueSize);
+      case BYTES_FIXED_STRAIGHT:
+        return promoter.set(type, IS_BYTE | IS_FIXED | IS_STRAIGHT, valueSize);
+      case BYTES_VAR_DEREF:
+        return promoter.set(type, IS_BYTE | IS_VAR | IS_DEREF,
+            VAR_TYPE_VALUE_SIZE);
+      case BYTES_VAR_SORTED:
+        return promoter.set(type, IS_BYTE | IS_VAR | IS_SORTED,
+            VAR_TYPE_VALUE_SIZE);
+      case BYTES_VAR_STRAIGHT:
+        return promoter.set(type, IS_BYTE | IS_VAR | IS_STRAIGHT,
+            VAR_TYPE_VALUE_SIZE);
+      case FIXED_INTS_16:
+        return promoter.set(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_16_BIT,
+            valueSize);
+      case FIXED_INTS_32:
+        return promoter.set(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_32_BIT,
+            valueSize);
+      case FIXED_INTS_64:
+        return promoter.set(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_64_BIT,
+            valueSize);
+      case FIXED_INTS_8:
+        return promoter.set(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_8_BIT,
+            valueSize);
+      case FLOAT_32:
+        return promoter.set(type,
+            IS_FLOAT | IS_FIXED | IS_STRAIGHT | IS_32_BIT_FLOAT, valueSize);
+      case FLOAT_64:
+        return promoter.set(type,
+            IS_FLOAT | IS_FIXED | IS_STRAIGHT | IS_64_BIT_FLOAT, valueSize);
+      case VAR_INTS:
+        return promoter.set(type, IS_INT | IS_VAR | IS_STRAIGHT,
+            VAR_TYPE_VALUE_SIZE);
+      default:
+        throw new IllegalStateException();
     }
   }
-
+  
+  public static int getValueSize(DocValues.Type type, BytesRef ref) {
+    switch (type) {
+      case VAR_INTS:
+      case BYTES_VAR_DEREF:
+      case BYTES_VAR_SORTED:
+      case BYTES_VAR_STRAIGHT:
+        return -1;
+      case BYTES_FIXED_DEREF:
+      case BYTES_FIXED_SORTED:
+      case BYTES_FIXED_STRAIGHT:
+        assert ref != null;
+        return ref.length;
+      case FIXED_INTS_16:
+        return 2;
+      case FLOAT_32:
+      case FIXED_INTS_32:
+        return 4;
+      case FLOAT_64:
+      case FIXED_INTS_64:
+        return 8;
+      case FIXED_INTS_8:
+        return 1;
+      default:
+        throw new IllegalArgumentException("unknonw docvalues type: "
+            + type.name());
+    }
+  }
+  
   /**
    * Returns a {@link TypePromoter} that always promotes to the type provided to
    * {@link #promote(TypePromoter)}
@@ -195,16 +305,58 @@ class TypePromoter {
   public static TypePromoter getIdentityPromoter() {
     return IDENTITY_PROMOTER;
   }
-
+  
+  private static TypePromoter newPromoter() {
+    return new TypePromoter(null, 0, -1);
+  }
+  
   private static class IdentityTypePromoter extends TypePromoter {
-
+    
     public IdentityTypePromoter() {
       super(null, 0, -1);
     }
-
+    
+    @Override
+    protected TypePromoter set(Type type, int flags, int valueSize) {
+      throw new UnsupportedOperationException("can not reset IdendityPromotoer");
+    }
+    
     @Override
     public TypePromoter promote(TypePromoter promoter) {
       return promoter;
     }
   }
+  
+  static class TypeCompatibility {
+    private final TypePromoter base;
+    private final TypePromoter spare;
+    
+    TypeCompatibility(Type type, int valueSize) {
+      this.base = create(type, valueSize);
+      spare = newPromoter();
+    }
+    
+    boolean isCompatible(Type type, int valueSize) {
+      TypePromoter reset = reset(type, valueSize, spare);
+      if (base.isTypeCompatible(reset)) {
+        if (base.isBytesCompatible(reset)) {
+          return base.valueSize == -1 || base.valueSize == valueSize;
+        } else if (base.flags == reset.flags) {
+          return true;
+        } else if (base.isNumericSizeCompatible(reset)) {
+          return base.valueSize == -1
+              || (base.valueSize > valueSize && valueSize > 0);
+        }
+      }
+      return false;
+    }
+    
+    Type getBaseType() {
+      return base.type();
+    }
+    
+    int getBaseSize() {
+      return base.valueSize;
+    }
+  }
 }
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java Fri Jun  8 20:49:30 2012
@@ -47,8 +47,6 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.junit.Before;
 
 /**
  * 

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesTypeCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesTypeCompatibility.java?rev=1348236&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesTypeCompatibility.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesTypeCompatibility.java Fri Jun  8 20:49:30 2012
@@ -0,0 +1,317 @@
+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
+ * 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.analysis.MockAnalyzer;
+import org.apache.lucene.document.ByteDocValuesField;
+import org.apache.lucene.document.DerefBytesDocValuesField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.IntDocValuesField;
+import org.apache.lucene.document.LongDocValuesField;
+import org.apache.lucene.document.PackedLongDocValuesField;
+import org.apache.lucene.document.ShortDocValuesField;
+import org.apache.lucene.document.SortedBytesDocValuesField;
+import org.apache.lucene.document.StraightBytesDocValuesField;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * Tests compatibility of {@link DocValues.Type} during indexing
+ */
+public class TestDocValuesTypeCompatibility extends LuceneTestCase {
+  
+  public void testAddCompatibleIntTypes() throws CorruptIndexException,
+      LockObtainFailedException, IOException {
+    int numIter = atLeast(10);
+    for (int i = 0; i < numIter; i++) {
+      Directory dir = newDirectory();
+      IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
+          new MockAnalyzer(random()));
+      int numDocs = atLeast(100);
+      
+      iwc.setMaxBufferedDocs(2 * numDocs); // make sure we hit the same DWPT
+                                           // here
+      iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+      iwc.setRAMPerThreadHardLimitMB(2000);
+      IndexWriter writer = new IndexWriter(dir, iwc);
+      Type[] types = new Type[] {Type.VAR_INTS, Type.FIXED_INTS_16,
+          Type.FIXED_INTS_64, Type.FIXED_INTS_16, Type.FIXED_INTS_8};
+      Type maxType = types[random().nextInt(types.length)];
+      for (int j = 0; j < numDocs; j++) {
+        addDoc(writer, getRandomIntsField(maxType, j == 0));
+      }
+      writer.close();
+      dir.close();
+    }
+    
+  }
+  
+  @SuppressWarnings("fallthrough")
+  public IndexableField getRandomIntsField(Type maxType, boolean force) {
+    switch (maxType) {
+    
+      case VAR_INTS:
+        if (random().nextInt(5) == 0 || force) {
+          return new PackedLongDocValuesField("f", 1);
+        }
+      case FIXED_INTS_64:
+        if (random().nextInt(4) == 0 || force) {
+          return new LongDocValuesField("f", 1);
+        }
+      case FIXED_INTS_32:
+        if (random().nextInt(3) == 0 || force) {
+          return new IntDocValuesField("f", 1);
+        }
+      case FIXED_INTS_16:
+        if (random().nextInt(2) == 0 || force) {
+          return new ShortDocValuesField("f", (short) 1);
+        }
+      case FIXED_INTS_8:
+        return new ByteDocValuesField("f", (byte) 1);
+        
+      default:
+        throw new IllegalArgumentException();
+        
+    }
+  }
+  
+  public void testAddCompatibleDoubleTypes() throws CorruptIndexException,
+      LockObtainFailedException, IOException {
+    int numIter = atLeast(10);
+    for (int i = 0; i < numIter; i++) {
+      Directory dir = newDirectory();
+      IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
+          new MockAnalyzer(random()));
+      int numDocs = atLeast(100);
+      
+      iwc.setMaxBufferedDocs(2 * numDocs); // make sure we hit the same DWPT
+                                           // here
+      iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+      iwc.setRAMPerThreadHardLimitMB(2000);
+      IndexWriter writer = new IndexWriter(dir, iwc);
+      
+      Type[] types = new Type[] {Type.FLOAT_64, Type.FLOAT_32};
+      Type maxType = types[random().nextInt(types.length)];
+      for (int j = 0; j < numDocs; j++) {
+        addDoc(writer, getRandomFloatField(maxType, j == 0));
+      }
+      writer.close();
+      dir.close();
+    }
+    
+  }
+  @SuppressWarnings("fallthrough")
+  public IndexableField getRandomFloatField(Type maxType, boolean force) {
+    switch (maxType) {
+    
+      case FLOAT_64:
+        if (random().nextInt(5) == 0 || force) {
+          return new PackedLongDocValuesField("f", 1);
+        }
+      case FIXED_INTS_32:
+        if (random().nextInt(4) == 0 || force) {
+          return new LongDocValuesField("f", 1);
+        }
+      case FLOAT_32:
+        if (random().nextInt(3) == 0 || force) {
+          return new IntDocValuesField("f", 1);
+        }
+      case FIXED_INTS_16:
+        if (random().nextInt(2) == 0 || force) {
+          return new ShortDocValuesField("f", (short) 1);
+        }
+      case FIXED_INTS_8:
+        return new ByteDocValuesField("f", (byte) 1);
+        
+      default:
+        throw new IllegalArgumentException();
+        
+    }
+  }
+  
+  public void testAddCompatibleDoubleTypes2() throws CorruptIndexException,
+      LockObtainFailedException, IOException {
+    int numIter = atLeast(10);
+    for (int i = 0; i < numIter; i++) {
+      Directory dir = newDirectory();
+      IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
+          new MockAnalyzer(random()));
+      int numDocs = atLeast(100);
+      
+      iwc.setMaxBufferedDocs(2 * numDocs); // make sure we hit the same DWPT
+                                           // here
+      iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+      iwc.setRAMPerThreadHardLimitMB(2000);
+      IndexWriter writer = new IndexWriter(dir, iwc);
+      IndexableField[] fields = new IndexableField[] {
+          new DoubleDocValuesField("f", 1.0), new IntDocValuesField("f", 1),
+          new ShortDocValuesField("f", (short) 1),
+          new ByteDocValuesField("f", (byte) 1)};
+      int base = random().nextInt(fields.length - 1);
+      
+      addDoc(writer, fields[base]);
+      
+      for (int j = 0; j < numDocs; j++) {
+        int f = base + random().nextInt(fields.length - base);
+        addDoc(writer, fields[f]);
+      }
+      writer.close();
+      dir.close();
+    }
+    
+  }
+  
+  public void testAddCompatibleByteTypes() throws CorruptIndexException,
+      LockObtainFailedException, IOException {
+    int numIter = atLeast(10);
+    for (int i = 0; i < numIter; i++) {
+      Directory dir = newDirectory();
+      IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
+          new MockAnalyzer(random()));
+      int numDocs = atLeast(100);
+      
+      iwc.setMaxBufferedDocs(2 * numDocs); // make sure we hit the same DWPT
+                                           // here
+      iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+      iwc.setRAMPerThreadHardLimitMB(2000);
+      IndexWriter writer = new IndexWriter(dir, iwc);
+      boolean mustBeFixed = random().nextBoolean();
+      int maxSize = 2 + random().nextInt(15);
+      IndexableField bytesField = getRandomBytesField(mustBeFixed, maxSize,
+          true);
+      addDoc(writer, bytesField);
+      for (int j = 0; j < numDocs; j++) {
+        bytesField = getRandomBytesField(mustBeFixed, maxSize, false);
+        addDoc(writer, bytesField);
+        
+      }
+      writer.close();
+      dir.close();
+    }
+  }
+  
+  public IndexableField getRandomBytesField(boolean mustBeFixed, int maxSize,
+      boolean mustBeVariableIfNotFixed) {
+    int size = mustBeFixed ? maxSize : random().nextInt(maxSize) + 1;
+    StringBuilder s = new StringBuilder();
+    for (int i = 0; i < size; i++) {
+      s.append("a");
+    }
+    BytesRef bytesRef = new BytesRef(s);
+    boolean fixed = mustBeFixed ? true : mustBeVariableIfNotFixed ? false
+        : random().nextBoolean();
+    switch (random().nextInt(3)) {
+      case 0:
+        return new SortedBytesDocValuesField("f", bytesRef, fixed);
+      case 1:
+        return new DerefBytesDocValuesField("f", bytesRef, fixed);
+      default:
+        return new StraightBytesDocValuesField("f", bytesRef, fixed);
+    }
+  }
+  
+  public void testIncompatibleTypesBytes() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
+        new MockAnalyzer(random()));
+    int numDocs = atLeast(100);
+
+    iwc.setMaxBufferedDocs(numDocs); // make sure we hit the same DWPT
+    iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+    iwc.setRAMPerThreadHardLimitMB(2000);
+    IndexWriter writer = new IndexWriter(dir, iwc);
+    
+    int numDocsIndexed = 0;
+    for (int j = 1; j < numDocs; j++) {
+      try {
+        addDoc(writer, getRandomIndexableDVField());
+        numDocsIndexed++;
+      } catch (IllegalArgumentException e) {
+        assertTrue(e.getMessage().startsWith("Incompatible DocValues type:"));
+      }
+    }
+    
+    writer.commit();
+    DirectoryReader open = DirectoryReader.open(dir);
+    assertEquals(numDocsIndexed, open.numDocs());
+    open.close();
+    writer.close();
+    dir.close();
+  }
+  
+  private void addDoc(IndexWriter writer, IndexableField... fields)
+      throws CorruptIndexException, IOException {
+    Document doc = new Document();
+    for (IndexableField indexableField : fields) {
+      doc.add(indexableField);
+    }
+    writer.addDocument(doc);
+  }
+  
+  public IndexableField getRandomIndexableDVField() {
+    int size = random().nextInt(100) + 1;
+    StringBuilder s = new StringBuilder();
+    for (int i = 0; i < size; i++) {
+      s.append("a");
+    }
+    BytesRef bytesRef = new BytesRef(s);
+    
+    Type[] values = Type.values();
+    Type t = values[random().nextInt(values.length)];
+    switch (t) {
+      case BYTES_FIXED_DEREF:
+        return new DerefBytesDocValuesField("f", bytesRef, true);
+      case BYTES_FIXED_SORTED:
+        return new SortedBytesDocValuesField("f", bytesRef, true);
+      case BYTES_FIXED_STRAIGHT:
+        return new StraightBytesDocValuesField("f", bytesRef, true);
+      case BYTES_VAR_DEREF:
+        return new DerefBytesDocValuesField("f", bytesRef, false);
+      case BYTES_VAR_SORTED:
+        return new SortedBytesDocValuesField("f", bytesRef, false);
+      case BYTES_VAR_STRAIGHT:
+        return new StraightBytesDocValuesField("f", bytesRef, false);
+      case FIXED_INTS_16:
+        return new ShortDocValuesField("f", (short) 1);
+      case FIXED_INTS_32:
+        return new IntDocValuesField("f", 1);
+      case FIXED_INTS_64:
+        return new LongDocValuesField("f", 1);
+      case FIXED_INTS_8:
+        return new ByteDocValuesField("f", (byte) 1);
+      case FLOAT_32:
+        return new FloatDocValuesField("f", 1.0f);
+      case FLOAT_64:
+        return new DoubleDocValuesField("f", 1.0f);
+      case VAR_INTS:
+        return new PackedLongDocValuesField("f", 1);
+      default:
+        throw new IllegalArgumentException();
+        
+    }
+    
+  }
+  
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNorms.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNorms.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNorms.java Fri Jun  8 20:49:30 2012
@@ -30,6 +30,7 @@ import org.apache.lucene.search.similari
 import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase;
@@ -174,6 +175,57 @@ public class TestNorms extends LuceneTes
     dir.close();
     otherDir.close();
   }
+  
+  public void testIllegalCustomEncoder() throws Exception {
+    Directory dir = newDirectory();
+    IllegalCustomEncodingSimilarity similarity = new IllegalCustomEncodingSimilarity();
+    IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    config.setSimilarity(similarity);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, config);
+    Document doc = new Document();
+    Field foo = newField("foo", "", TextField.TYPE_UNSTORED);
+    Field bar = newField("bar", "", TextField.TYPE_UNSTORED);
+    doc.add(foo);
+    doc.add(bar);
+    
+    int numAdded = 0;
+    for (int i = 0; i < 100; i++) {
+      try {
+        bar.setStringValue("singleton");
+        similarity.useByte = random().nextBoolean();
+        writer.addDocument(doc);
+        numAdded++;
+      } catch (IllegalArgumentException e) {}
+    }
+    
+    
+    IndexReader reader = writer.getReader();
+    writer.close();
+    assertEquals(numAdded, reader.numDocs());
+    IndexReaderContext topReaderContext = reader.getTopReaderContext();
+    AtomicReaderContext[] leaves = topReaderContext.leaves();
+    for (int j = 0; j < leaves.length; j++) {
+      AtomicReader atomicReader = leaves[j].reader();
+    Source source = random().nextBoolean() ? atomicReader.normValues("foo").getSource() : atomicReader.normValues("foo").getDirectSource();
+    Bits liveDocs = atomicReader.getLiveDocs();
+    Type t = source.getType();
+    for (int i = 0; i < atomicReader.maxDoc(); i++) {
+        assertEquals(0, source.getFloat(i), 0.000f);
+    }
+    
+
+    source = random().nextBoolean() ? atomicReader.normValues("bar").getSource() : atomicReader.normValues("bar").getDirectSource();
+    for (int i = 0; i < atomicReader.maxDoc(); i++) {
+      if (liveDocs == null || liveDocs.get(i)) {
+        assertEquals("type: " + t, 1, source.getFloat(i), 0.000f);
+      } else {
+        assertEquals("type: " + t, 0, source.getFloat(i), 0.000f);
+      }
+    }
+    }
+    reader.close();
+    dir.close();
+  }
 
   public void buildIndex(Directory dir, boolean writeNorms) throws IOException,
       CorruptIndexException {
@@ -248,5 +300,29 @@ public class TestNorms extends LuceneTes
       }
     }
   }
+  
+  
+  class IllegalCustomEncodingSimilarity extends DefaultSimilarity {
+    
+    public boolean useByte = false;
+    @Override
+    public byte encodeNormValue(float f) {
+      return (byte) f;
+    }
+    
+    @Override
+    public float decodeNormValue(byte b) {
+      return (float) b;
+    }
+
+    @Override
+    public void computeNorm(FieldInvertState state, Norm norm) {
+      if (useByte) {
+        norm.setByte(encodeNormValue((float) state.getLength()));
+      } else {
+        norm.setFloat((float)state.getLength());
+      }
+    }
+  }
  
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTypePromotion.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTypePromotion.java?rev=1348236&r1=1348235&r2=1348236&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTypePromotion.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTypePromotion.java Fri Jun  8 20:49:30 2012
@@ -1,6 +1,6 @@
 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 this
  * work for additional information regarding copyright ownership. The ASF
@@ -41,7 +41,6 @@ import org.apache.lucene.store.ByteArray
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 
 public class TestTypePromotion extends LuceneTestCase {
 
@@ -50,7 +49,7 @@ public class TestTypePromotion extends L
       Type.FIXED_INTS_64, Type.FIXED_INTS_8);
 
   private static EnumSet<Type> FLOATS = EnumSet.of(Type.FLOAT_32,
-      Type.FLOAT_64);
+      Type.FLOAT_64, Type.FIXED_INTS_8);
 
   private static EnumSet<Type> UNSORTED_BYTES = EnumSet.of(
       Type.BYTES_FIXED_DEREF, Type.BYTES_FIXED_STRAIGHT,
@@ -77,12 +76,13 @@ public class TestTypePromotion extends L
     int num_2 = atLeast(200);
     int num_3 = atLeast(200);
     long[] values = new long[num_1 + num_2 + num_3];
+    Type[] sourceType = new Type[num_1 + num_2 + num_3];
     index(writer,
-        randomValueType(types, random()), values, 0, num_1);
+        randomValueType(types, random()), values, sourceType, 0, num_1);
     writer.commit();
     
     index(writer,
-        randomValueType(types, random()), values, num_1, num_2);
+        randomValueType(types, random()), values, sourceType, num_1, num_2);
     writer.commit();
     
     if (random().nextInt(4) == 0) {
@@ -93,7 +93,7 @@ public class TestTypePromotion extends L
       IndexWriter writer_2 = new IndexWriter(dir_2,
           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
       index(writer_2,
-          randomValueType(types, random()), values, num_1 + num_2, num_3);
+          randomValueType(types, random()), values, sourceType, num_1 + num_2, num_3);
       writer_2.commit();
       writer_2.close();
       if (rarely()) {
@@ -107,17 +107,17 @@ public class TestTypePromotion extends L
       dir_2.close();
     } else {
       index(writer,
-          randomValueType(types, random()), values, num_1 + num_2, num_3);
+          randomValueType(types, random()), values, sourceType, num_1 + num_2, num_3);
     }
 
     writer.forceMerge(1);
     writer.close();
-    assertValues(type, dir, values);
+    assertValues(type, dir, values, sourceType);
     dir.close();
   }
 
   
-  private void assertValues(TestType type, Directory dir, long[] values)
+  private void assertValues(TestType type, Directory dir, long[] values, Type[] sourceType)
       throws CorruptIndexException, IOException {
     DirectoryReader reader = DirectoryReader.open(dir);
     assertEquals(1, reader.getSequentialSubReaders().length);
@@ -158,7 +158,13 @@ public class TestTypePromotion extends L
         assertEquals(msg  + " byteSize: " + bytes.length, values[id], value);
         break;
       case Float:
-        assertEquals(msg, values[id], Double.doubleToRawLongBits(directSource.getFloat(i)));
+          if (sourceType[id] == Type.FLOAT_32
+              || sourceType[id] == Type.FLOAT_64) {
+            assertEquals(msg, values[id],
+                Double.doubleToRawLongBits(directSource.getFloat(i)));
+          } else {
+            assertEquals(msg, values[id], directSource.getFloat(i), 0.0d);
+          }
         break;
       case Int:
         assertEquals(msg, values[id], directSource.getInt(i));
@@ -173,7 +179,7 @@ public class TestTypePromotion extends L
   }
 
   public void index(IndexWriter writer,
-      Type valueType, long[] values, int offset, int num)
+      Type valueType, long[] values, Type[] sourceTypes, int offset, int num)
       throws CorruptIndexException, IOException {
     final Field valField;
 
@@ -228,6 +234,7 @@ public class TestTypePromotion extends L
     for (int i = offset; i < offset + num; i++) {
       Document doc = new Document();
       doc.add(new Field("id", i + "", TextField.TYPE_STORED));
+      sourceTypes[i] = valueType;
       switch (valueType) {
       case VAR_INTS:
         // TODO: can we do nextLong()?
@@ -327,8 +334,9 @@ public class TestTypePromotion extends L
     int num_1 = atLeast(200);
     int num_2 = atLeast(200);
     long[] values = new long[num_1 + num_2];
+    Type[] sourceType = new Type[num_1 + num_2];
     index(writer,
-        randomValueType(INTEGERS, random()), values, 0, num_1);
+        randomValueType(INTEGERS, random()), values, sourceType, 0, num_1);
     writer.commit();
     
     if (random().nextInt(4) == 0) {
@@ -337,7 +345,7 @@ public class TestTypePromotion extends L
       IndexWriter writer_2 = new IndexWriter(dir_2,
                        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
       index(writer_2,
-          randomValueType(random().nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random()), values, num_1, num_2);
+          randomValueType(random().nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random()), values, sourceType, num_1, num_2);
       writer_2.commit();
       writer_2.close();
       if (random().nextBoolean()) {
@@ -351,7 +359,7 @@ public class TestTypePromotion extends L
       dir_2.close();
     } else {
       index(writer,
-          randomValueType(random().nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random()), values, num_1, num_2);
+          randomValueType(random().nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random()), values, sourceType, num_1, num_2);
       writer.commit();
     }
     writer.close();
@@ -369,7 +377,7 @@ public class TestTypePromotion extends L
     AtomicReaderContext[] children = topReaderContext.leaves();
     DocValues docValues = children[0].reader().docValues("promote");
     assertNotNull(docValues);
-    assertValues(TestType.Byte, dir, values);
+    assertValues(TestType.Byte, dir, values, sourceType);
     assertEquals(Type.BYTES_VAR_STRAIGHT, docValues.getType());
     reader.close();
     dir.close();