You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/08/16 20:56:24 UTC

svn commit: r1514848 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/lucene40/ lucene/core/src/java/org/apache/lucene/codecs/lucene42/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/...

Author: mikemccand
Date: Fri Aug 16 18:56:23 2013
New Revision: 1514848

URL: http://svn.apache.org/r1514848
Log:
LUCENE-4583: IndexWriter no longer places a limit on length of DV binary fields (individual codecs still have their limits, including the default codec)

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesFormat.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestPagedBytes.java
    lucene/dev/branches/branch_4x/lucene/facet/   (props changed)
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Fri Aug 16 18:56:23 2013
@@ -79,6 +79,11 @@ Bug Fixes
   boundary, made it into the top-N and went to the formatter.
   (Manuel Amoabeng, Michael McCandless, Robert Muir)
 
+* LUCENE-4583: Indexing core no longer enforces a limit on maximum
+  length binary doc values fields, but individual codecs (including
+  the default one) have their own limits (David Smiley, Robert Muir,
+  Mike McCandless)
+
 API Changes
 
 * LUCENE-5094: Add ramBytesUsed() to MultiDocValues.OrdinalMap.

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java Fri Aug 16 18:56:23 2013
@@ -118,6 +118,11 @@ import org.apache.lucene.util.packed.Pac
  *        {@code BYTES_VAR_DEREF BYTES_VAR_DEREF} it doesn't apply deduplication of the document values.
  *    </li>
  * </ul>
+ * <p>
+ * Limitations:
+ * <ul>
+ *   <li> Binary doc values can be at most {@link #MAX_BINARY_FIELD_LENGTH} in length.
+ * </ul>
  * @deprecated Only for reading old 4.0 and 4.1 segments
  */
 @Deprecated
@@ -125,6 +130,9 @@ import org.apache.lucene.util.packed.Pac
 // for back compat only!
 public class Lucene40DocValuesFormat extends DocValuesFormat {
   
+  /** Maximum length for each binary doc values field. */
+  public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
+  
   /** Sole constructor. */
   public Lucene40DocValuesFormat() {
     super("Lucene40");

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Fri Aug 16 18:56:23 2013
@@ -36,14 +36,14 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.MathUtil;
 import org.apache.lucene.util.fst.Builder;
-import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.FST.INPUT_TYPE;
+import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
 import org.apache.lucene.util.fst.Util;
 import org.apache.lucene.util.packed.BlockPackedWriter;
 import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
-import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
+import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * Writer for {@link Lucene42DocValuesFormat}
@@ -216,6 +216,9 @@ class Lucene42DocValuesConsumer extends 
     int maxLength = Integer.MIN_VALUE;
     final long startFP = data.getFilePointer();
     for(BytesRef v : values) {
+      if (v.length > Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH) {
+        throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH);
+      }
       minLength = Math.min(minLength, v.length);
       maxLength = Math.max(maxLength, v.length);
       data.writeBytes(v.bytes, v.offset, v.length);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java Fri Aug 16 18:56:23 2013
@@ -118,8 +118,17 @@ import org.apache.lucene.util.packed.Blo
  *   <p>SortedSet entries store the list of ordinals in their BinaryData as a
  *      sequences of increasing {@link DataOutput#writeVLong vLong}s, delta-encoded.</p>       
  * </ol>
+ * <p>
+ * Limitations:
+ * <ul>
+ *   <li> Binary doc values can be at most {@link #MAX_BINARY_FIELD_LENGTH} in length.
+ * </ul>
  */
 public final class Lucene42DocValuesFormat extends DocValuesFormat {
+
+  /** Maximum length for each binary doc values field. */
+  public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
+  
   final float acceptableOverheadRatio;
   
   /** 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Fri Aug 16 18:56:23 2013
@@ -22,29 +22,43 @@ import java.util.Iterator;
 import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
 import org.apache.lucene.util.packed.PackedInts;
 
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
-
-
 /** Buffers up pending byte[] per doc, then flushes when
  *  segment flushes. */
 class BinaryDocValuesWriter extends DocValuesWriter {
 
-  private final ByteBlockPool pool;
+  /** Maximum length for a binary field; we set this to "a
+   *  bit" below Integer.MAX_VALUE because the exact max
+   *  allowed byte[] is JVM dependent, so we want to avoid
+   *  a case where a large value worked in one JVM but
+   *  failed later at search time with a different JVM.  */
+  private static final int MAX_LENGTH = Integer.MAX_VALUE-256;
+
+  // 32 KB block sizes for PagedBytes storage:
+  private final static int BLOCK_BITS = 15;
+
+  private final PagedBytes bytes;
+  private final DataOutput bytesOut;
+
+  private final Counter iwBytesUsed;
   private final AppendingDeltaPackedLongBuffer lengths;
   private final FieldInfo fieldInfo;
-  private int addedValues = 0;
+  private int addedValues;
+  private long bytesUsed;
 
   public BinaryDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     this.fieldInfo = fieldInfo;
-    this.pool = new ByteBlockPool(new DirectTrackingAllocator(iwBytesUsed));
+    this.bytes = new PagedBytes(BLOCK_BITS);
+    this.bytesOut = bytes.getDataOutput();
     this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
+    this.iwBytesUsed = iwBytesUsed;
   }
 
   public void addValue(int docID, BytesRef value) {
@@ -54,10 +68,10 @@ class BinaryDocValuesWriter extends DocV
     if (value == null) {
       throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\": null value not allowed");
     }
-    if (value.length > (BYTE_BLOCK_SIZE - 2)) {
-      throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" is too large, must be <= " + (BYTE_BLOCK_SIZE - 2));
+    if (value.length > MAX_LENGTH) {
+      throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" is too large, must be <= " + MAX_LENGTH);
     }
-    
+
     // Fill in any holes:
     while(addedValues < docID) {
       addedValues++;
@@ -65,7 +79,19 @@ class BinaryDocValuesWriter extends DocV
     }
     addedValues++;
     lengths.add(value.length);
-    pool.append(value);
+    try {
+      bytesOut.writeBytes(value.bytes, value.offset, value.length);
+    } catch (IOException ioe) {
+      // Should never happen!
+      throw new RuntimeException(ioe);
+    }
+    updateBytesUsed();
+  }
+
+  private void updateBytesUsed() {
+    final long newBytesUsed = lengths.ramBytesUsed() + bytes.ramBytesUsed();
+    iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
+    bytesUsed = newBytesUsed;
   }
 
   @Override
@@ -75,6 +101,7 @@ class BinaryDocValuesWriter extends DocV
   @Override
   public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
     final int maxDoc = state.segmentInfo.getDocCount();
+    bytes.freeze(false);
     dvConsumer.addBinaryField(fieldInfo,
                               new Iterable<BytesRef>() {
                                 @Override
@@ -92,10 +119,10 @@ class BinaryDocValuesWriter extends DocV
   private class BytesIterator implements Iterator<BytesRef> {
     final BytesRef value = new BytesRef();
     final AppendingDeltaPackedLongBuffer.Iterator lengthsIterator = lengths.iterator();
+    final DataInput bytesIterator = bytes.getDataInput();
     final int size = (int) lengths.size();
     final int maxDoc;
     int upto;
-    long byteOffset;
     
     BytesIterator(int maxDoc) {
       this.maxDoc = maxDoc;
@@ -115,8 +142,12 @@ class BinaryDocValuesWriter extends DocV
         int length = (int) lengthsIterator.next();
         value.grow(length);
         value.length = length;
-        pool.readBytes(byteOffset, value.bytes, value.offset, value.length);
-        byteOffset += length;
+        try {
+          bytesIterator.readBytes(value.bytes, value.offset, value.length);
+        } catch (IOException ioe) {
+          // Should never happen!
+          throw new RuntimeException(ioe);
+        }
       } else {
         // This is to handle last N documents not having
         // this DV field in the end of the segment:

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Fri Aug 16 18:56:23 2013
@@ -92,21 +92,22 @@ public final class FieldInfo {
      */
     NUMERIC,
     /**
-     * A per-document byte[].
+     * A per-document byte[].  Values may be larger than
+     * 32766 bytes, but different codecs may enforce their own limits.
      */
     BINARY,
     /** 
      * A pre-sorted byte[]. Fields with this type only store distinct byte values 
      * and store an additional offset pointer per document to dereference the shared 
      * byte[]. The stored byte[] is presorted and allows access via document id, 
-     * ordinal and by-value.
+     * ordinal and by-value.  Values must be <= 32766 bytes.
      */
     SORTED,
     /** 
      * A pre-sorted Set&lt;byte[]&gt;. Fields with this type only store distinct byte values 
      * and store additional offset pointers per document to dereference the shared 
      * byte[]s. The stored byte[] is presorted and allows access via document id, 
-     * ordinal and by-value.
+     * ordinal and by-value.  Values must be <= 32766 bytes.
      */
     SORTED_SET
   };

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java Fri Aug 16 18:56:23 2013
@@ -36,6 +36,7 @@ import org.apache.lucene.store.IndexInpu
 // other "shift/mask big arrays". there are too many of these classes!
 public final class PagedBytes {
   private final List<byte[]> blocks = new ArrayList<byte[]>();
+  // TODO: these are unused?
   private final List<Integer> blockEnd = new ArrayList<Integer>();
   private final int blockSize;
   private final int blockBits;
@@ -44,6 +45,7 @@ public final class PagedBytes {
   private boolean frozen;
   private int upto;
   private byte[] currentBlock;
+  private final long bytesUsedPerBlock;
 
   private static final byte[] EMPTY_BYTES = new byte[0];
 
@@ -77,13 +79,13 @@ public final class PagedBytes {
      * given length. Iff the slice spans across a block border this method will
      * allocate sufficient resources and copy the paged data.
      * <p>
-     * Slices spanning more than one block are not supported.
+     * Slices spanning more than two blocks are not supported.
      * </p>
      * @lucene.internal 
      **/
     public void fillSlice(BytesRef b, long start, int length) {
       assert length >= 0: "length=" + length;
-      assert length <= blockSize+1;
+      assert length <= blockSize+1: "length=" + length;
       final int index = (int) (start >> blockBits);
       final int offset = (int) (start & blockMask);
       b.length = length;
@@ -134,6 +136,7 @@ public final class PagedBytes {
     this.blockBits = blockBits;
     blockMask = blockSize-1;
     upto = blockSize;
+    bytesUsedPerBlock = blockSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + RamUsageEstimator.NUM_BYTES_OBJECT_REF;
   }
 
   /** Read this many bytes from in */
@@ -218,6 +221,11 @@ public final class PagedBytes {
     }
   }
 
+  /** Return approx RAM usage in bytes. */
+  public long ramBytesUsed() {
+    return (blocks.size() + (currentBlock != null ? 1 : 0)) * bytesUsedPerBlock;
+  }
+
   /** Copy bytes in, writing the length as a 1 or 2 byte
    *  vInt prefix. */
   // TODO: this really needs to be refactored into fieldcacheimpl

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java Fri Aug 16 18:56:23 2013
@@ -30,5 +30,11 @@ public class TestLucene40DocValuesFormat
   protected Codec getCodec() {
     return codec;
   }
-  
+
+  // LUCENE-4583: This codec should throw IAE on huge binary values:
+  @Override
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return false;
+  }
+
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java Fri Aug 16 18:56:23 2013
@@ -30,4 +30,9 @@ public class TestLucene42DocValuesFormat
   protected Codec getCodec() {
     return codec;
   }
+
+  @Override
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return false;
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java Fri Aug 16 18:56:23 2013
@@ -45,6 +45,7 @@ import org.apache.lucene.search.TermQuer
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * Basic tests of PerFieldDocValuesFormat
@@ -62,6 +63,11 @@ public class TestPerFieldDocValuesFormat
   protected Codec getCodec() {
     return codec;
   }
+
+  @Override
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return _TestUtil.fieldSupportsHugeBinaryDocValues(field);
+  }
   
   // just a simple trivial test
   // TODO: we should come up with a test that somehow checks that segment suffix

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesFormat.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesFormat.java Fri Aug 16 18:56:23 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util._TestUtil;
 
 /** Tests the codec configuration defined by LuceneTestCase randomly
  *  (typically a mix across different fields).
@@ -30,4 +31,9 @@ public class TestDocValuesFormat extends
   protected Codec getCodec() {
     return Codec.getDefault();
   }
+
+  @Override
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return _TestUtil.fieldSupportsHugeBinaryDocValues(field);
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java Fri Aug 16 18:56:23 2013
@@ -328,31 +328,7 @@ public class TestDocValuesIndexing exten
     iwriter.close();
     directory.close();
   }
-  
-  public void testTooLargeBytes() throws IOException {
-    Analyzer analyzer = new MockAnalyzer(random());
-
-    Directory directory = newDirectory();
-    // we don't use RandomIndexWriter because it might add more docvalues than we expect !!!!1
-    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
-    iwc.setMergePolicy(newLogMergePolicy());
-    IndexWriter iwriter = new IndexWriter(directory, iwc);
-    Document doc = new Document();
-    byte bytes[] = new byte[100000];
-    BytesRef b = new BytesRef(bytes);
-    random().nextBytes(bytes);
-    doc.add(new BinaryDocValuesField("dv", b));
-    try {
-      iwriter.addDocument(doc);
-      fail("did not get expected exception");
-    } catch (IllegalArgumentException expected) {
-      // expected
-    }
-    iwriter.close();
 
-    directory.close();
-  }
-  
   public void testTooLargeSortedBytes() throws IOException {
     Analyzer analyzer = new MockAnalyzer(random());
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestPagedBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestPagedBytes.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestPagedBytes.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestPagedBytes.java Fri Aug 16 18:56:23 2013
@@ -22,6 +22,7 @@ import java.util.*;
 
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -30,6 +31,9 @@ import org.junit.Ignore;
 
 public class TestPagedBytes extends LuceneTestCase {
 
+  // Writes random byte/s to "normal" file in dir, then
+  // copies into PagedBytes and verifies with
+  // PagedBytes.Reader: 
   public void testDataInputOutput() throws Exception {
     Random random = random();
     for(int iter=0;iter<5*RANDOM_MULTIPLIER;iter++) {
@@ -90,6 +94,60 @@ public class TestPagedBytes extends Luce
     }
   }
 
+  // Writes random byte/s into PagedBytes via
+  // .getDataOutput(), then verifies with
+  // PagedBytes.getDataInput(): 
+  public void testDataInputOutput2() throws Exception {
+    Random random = random();
+    for(int iter=0;iter<5*RANDOM_MULTIPLIER;iter++) {
+      final int blockBits = _TestUtil.nextInt(random, 1, 20);
+      final int blockSize = 1 << blockBits;
+      final PagedBytes p = new PagedBytes(blockBits);
+      final DataOutput out = p.getDataOutput();
+      final int numBytes = random().nextInt(10000000);
+
+      final byte[] answer = new byte[numBytes];
+      random().nextBytes(answer);
+      int written = 0;
+      while(written < numBytes) {
+        if (random().nextInt(10) == 7) {
+          out.writeByte(answer[written++]);
+        } else {
+          int chunk = Math.min(random().nextInt(1000), numBytes - written);
+          out.writeBytes(answer, written, chunk);
+          written += chunk;
+        }
+      }
+
+      final PagedBytes.Reader reader = p.freeze(random.nextBoolean());
+
+      final DataInput in = p.getDataInput();
+
+      final byte[] verify = new byte[numBytes];
+      int read = 0;
+      while(read < numBytes) {
+        if (random().nextInt(10) == 7) {
+          verify[read++] = in.readByte();
+        } else {
+          int chunk = Math.min(random().nextInt(1000), numBytes - read);
+          in.readBytes(verify, read, chunk);
+          read += chunk;
+        }
+      }
+      assertTrue(Arrays.equals(answer, verify));
+
+      final BytesRef slice = new BytesRef();
+      for(int iter2=0;iter2<100;iter2++) {
+        final int pos = random.nextInt(numBytes-1);
+        final int len = random.nextInt(Math.min(blockSize+1, numBytes - pos));
+        reader.fillSlice(slice, pos, len);
+        for(int byteUpto=0;byteUpto<len;byteUpto++) {
+          assertEquals(answer[pos + byteUpto], slice.bytes[slice.offset + byteUpto]);
+        }
+      }
+    }
+  }
+
   @Ignore // memory hole
   public void testOverflow() throws IOException {
     BaseDirectoryWrapper dir = newFSDirectory(_TestUtil.getTempDir("testOverflow"));
@@ -126,4 +184,5 @@ public class TestPagedBytes extends Luce
     in.close();
     dir.close();
   }
+
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java Fri Aug 16 18:56:23 2013
@@ -22,13 +22,16 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.FacetTestUtils;
+import org.apache.lucene.facet.codecs.facet42.Facet42Codec;
 import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.params.FacetIndexingParams;
@@ -48,6 +51,8 @@ import org.apache.lucene.search.similari
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util._TestUtil;
 
 public class TestDemoFacets extends FacetTestCase {
 
@@ -248,4 +253,60 @@ public class TestDemoFacets extends Face
     dir.close();
     taxoDir.close();
   }
+  
+  // LUCENE-4583: make sure if we require > 32 KB for one
+  // document, we don't hit exc when using Facet42DocValuesFormat
+  public void testManyFacetsInOneDocument() throws Exception {
+    Directory dir = newDirectory();
+    Directory taxoDir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setCodec(new Facet42Codec());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
+
+    FacetFields facetFields = new FacetFields(taxoWriter);
+
+    int numLabels = _TestUtil.nextInt(random(), 40000, 100000);
+
+    Document doc = new Document();
+    doc.add(newTextField("field", "text", Field.Store.NO));
+    List<CategoryPath> paths = new ArrayList<CategoryPath>();
+    for(int i=0;i<numLabels;i++) {
+      paths.add(new CategoryPath("dim", "" + i));
+    }
+    facetFields.addFields(doc, paths);
+    writer.addDocument(doc);
+
+    // NRT open
+    IndexSearcher searcher = newSearcher(writer.getReader());
+    writer.close();
+
+    // NRT open
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
+    taxoWriter.close();
+    
+    FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("dim"), Integer.MAX_VALUE));
+
+    // Aggregate the facet counts:
+    FacetsCollector c = FacetsCollector.create(fsp, searcher.getIndexReader(), taxoReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query, and use MultiCollector to
+    // wrap collecting the "normal" hits and also facets:
+    searcher.search(new MatchAllDocsQuery(), c);
+    List<FacetResult> results = c.getFacetResults();
+    assertEquals(1, results.size());
+    FacetResultNode root = results.get(0).getFacetResultNode();
+    assertEquals(numLabels, root.subResults.size());
+    Set<String> allLabels = new HashSet<String>();
+    for(FacetResultNode childNode : root.subResults) {
+      assertEquals(2, childNode.label.length);
+      allLabels.add(childNode.label.components[1]);
+      assertEquals(1, (int) childNode.value);
+    }
+    assertEquals(numLabels, allLabels.size());
+
+    IOUtils.close(searcher.getIndexReader(), taxoReader, dir, taxoDir);
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java Fri Aug 16 18:56:23 2013
@@ -156,6 +156,9 @@ class Lucene40DocValuesWriter extends Do
     int minLength = Integer.MAX_VALUE;
     int maxLength = Integer.MIN_VALUE;
     for (BytesRef b : values) {
+      if (b.length > Lucene40DocValuesFormat.MAX_BINARY_FIELD_LENGTH) {
+        throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + Lucene40DocValuesFormat.MAX_BINARY_FIELD_LENGTH);
+      }
       minLength = Math.min(minLength, b.length);
       maxLength = Math.max(maxLength, b.length);
       if (uniqueValues != null) {

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Fri Aug 16 18:56:23 2013
@@ -17,21 +17,20 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -57,6 +56,8 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
+import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
+
 /**
  * Abstract class to do basic tests for a docvalues format.
  * NOTE: This test focuses on the docvalues impl, nothing else.
@@ -2198,4 +2199,172 @@ public abstract class BaseDocValuesForma
     }
   }
 
+  // LUCENE-4853
+  public void testHugeBinaryValues() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    // FSDirectory because SimpleText will consume gobbs of
+    // space when storing big binary values:
+    Directory d = newFSDirectory(_TestUtil.getTempDir("hugeBinaryValues"));
+    boolean doFixed = random().nextBoolean();
+    int numDocs;
+    int fixedLength = 0;
+    if (doFixed) {
+      // Sometimes make all values fixed length since some
+      // codecs have different code paths for this:
+      numDocs = _TestUtil.nextInt(random(), 10, 20);
+      fixedLength = _TestUtil.nextInt(random(), 65537, 256*1024);
+    } else {
+      numDocs = _TestUtil.nextInt(random(), 100, 200);
+    }
+    IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
+    List<byte[]> docBytes = new ArrayList<byte[]>();
+    long totalBytes = 0;
+    for(int docID=0;docID<numDocs;docID++) {
+      // we don't use RandomIndexWriter because it might add
+      // more docvalues than we expect !!!!
+
+      // Must be > 64KB in size to ensure more than 2 pages in
+      // PagedBytes would be needed:
+      int numBytes;
+      if (doFixed) {
+        numBytes = fixedLength;
+      } else if (docID == 0 || random().nextInt(5) == 3) {
+        numBytes = _TestUtil.nextInt(random(), 65537, 3*1024*1024);
+      } else {
+        numBytes = _TestUtil.nextInt(random(), 1, 1024*1024);
+      }
+      totalBytes += numBytes;
+      if (totalBytes > 5 * 1024*1024) {
+        break;
+      }
+      byte[] bytes = new byte[numBytes];
+      random().nextBytes(bytes);
+      docBytes.add(bytes);
+      Document doc = new Document();      
+      BytesRef b = new BytesRef(bytes);
+      b.length = bytes.length;
+      doc.add(new BinaryDocValuesField("field", b));
+      doc.add(new StringField("id", ""+docID, Field.Store.YES));
+      try {
+        w.addDocument(doc);
+      } catch (IllegalArgumentException iae) {
+        if (iae.getMessage().indexOf("is too large") == -1) {
+          throw iae;
+        } else {
+          // OK: some codecs can't handle binary DV > 32K
+          assertFalse(codecAcceptsHugeBinaryValues("field"));
+          w.rollback();
+          d.close();
+          return;
+        }
+      }
+    }
+    
+    DirectoryReader r;
+    try {
+      r = w.getReader();
+    } catch (IllegalArgumentException iae) {
+      if (iae.getMessage().indexOf("is too large") == -1) {
+        throw iae;
+      } else {
+        assertFalse(codecAcceptsHugeBinaryValues("field"));
+
+        // OK: some codecs can't handle binary DV > 32K
+        w.rollback();
+        d.close();
+        return;
+      }
+    }
+    w.close();
+
+    AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+
+    BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field");
+    for(int docID=0;docID<docBytes.size();docID++) {
+      Document doc = ar.document(docID);
+      BytesRef bytes = new BytesRef();
+      s.get(docID, bytes);
+      byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
+      assertEquals(expected.length, bytes.length);
+      assertEquals(new BytesRef(expected), bytes);
+    }
+
+    assertTrue(codecAcceptsHugeBinaryValues("field"));
+
+    ar.close();
+    d.close();
+  }
+
+  public void testHugeBinaryValueLimit() throws Exception {
+    // We only test DVFormats that have a limit
+    assumeFalse("test requires codec with limits on max binary field length", codecAcceptsHugeBinaryValues("field"));
+    Analyzer analyzer = new MockAnalyzer(random());
+    // FSDirectory because SimpleText will consume gobbs of
+    // space when storing big binary values:
+    Directory d = newFSDirectory(_TestUtil.getTempDir("hugeBinaryValues"));
+    boolean doFixed = random().nextBoolean();
+    int numDocs;
+    int fixedLength = 0;
+    if (doFixed) {
+      // Sometimes make all values fixed length since some
+      // codecs have different code paths for this:
+      numDocs = _TestUtil.nextInt(random(), 10, 20);
+      fixedLength = Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH;
+    } else {
+      numDocs = _TestUtil.nextInt(random(), 100, 200);
+    }
+    IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
+    List<byte[]> docBytes = new ArrayList<byte[]>();
+    long totalBytes = 0;
+    for(int docID=0;docID<numDocs;docID++) {
+      // we don't use RandomIndexWriter because it might add
+      // more docvalues than we expect !!!!
+
+      // Must be > 64KB in size to ensure more than 2 pages in
+      // PagedBytes would be needed:
+      int numBytes;
+      if (doFixed) {
+        numBytes = fixedLength;
+      } else if (docID == 0 || random().nextInt(5) == 3) {
+        numBytes = Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH;
+      } else {
+        numBytes = _TestUtil.nextInt(random(), 1, Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH);
+      }
+      totalBytes += numBytes;
+      if (totalBytes > 5 * 1024*1024) {
+        break;
+      }
+      byte[] bytes = new byte[numBytes];
+      random().nextBytes(bytes);
+      docBytes.add(bytes);
+      Document doc = new Document();      
+      BytesRef b = new BytesRef(bytes);
+      b.length = bytes.length;
+      doc.add(new BinaryDocValuesField("field", b));
+      doc.add(new StringField("id", ""+docID, Field.Store.YES));
+      w.addDocument(doc);
+    }
+    
+    DirectoryReader r = w.getReader();
+    w.close();
+
+    AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+
+    BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field");
+    for(int docID=0;docID<docBytes.size();docID++) {
+      Document doc = ar.document(docID);
+      BytesRef bytes = new BytesRef();
+      s.get(docID, bytes);
+      byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
+      assertEquals(expected.length, bytes.length);
+      assertEquals(new BytesRef(expected), bytes);
+    }
+
+    ar.close();
+    d.close();
+  }
+
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return true;
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java?rev=1514848&r1=1514847&r2=1514848&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java Fri Aug 16 18:56:23 2013
@@ -45,6 +45,7 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
@@ -53,12 +54,12 @@ import org.apache.lucene.document.Numeri
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.CheckIndex;
 import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
 import org.apache.lucene.index.CheckIndex.Status.FieldNormStatus;
 import org.apache.lucene.index.CheckIndex.Status.StoredFieldStatus;
 import org.apache.lucene.index.CheckIndex.Status.TermIndexStatus;
 import org.apache.lucene.index.CheckIndex.Status.TermVectorStatus;
+import org.apache.lucene.index.CheckIndex;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
@@ -732,6 +733,25 @@ public class _TestUtil {
       return p.getName();
     }
   }
+  public static String getDocValuesFormat(String field) {
+    return getDocValuesFormat(Codec.getDefault(), field);
+  }
+  
+  public static String getDocValuesFormat(Codec codec, String field) {
+    DocValuesFormat f = codec.docValuesFormat();
+    if (f instanceof PerFieldDocValuesFormat) {
+      return ((PerFieldDocValuesFormat) f).getDocValuesFormatForField(field).getName();
+    } else {
+      return f.getName();
+    }
+  }
+
+  public static boolean fieldSupportsHugeBinaryDocValues(String field) {
+    String dvFormat = getDocValuesFormat(field);
+    return dvFormat.equals("CheapBastard") ||
+      dvFormat.equals("Disk") ||
+      dvFormat.equals("SimpleText");
+  }
 
   public static boolean anyFilesExceptWriteLock(Directory dir) throws IOException {
     String[] files = dir.listAll();