You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2014/08/13 11:36:59 UTC

svn commit: r1617695 [5/9] - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/analysis/ lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/ lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ lucene/analysis/comm...

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java Wed Aug 13 09:36:54 2014
@@ -99,16 +99,15 @@ public final class BytesRefArray {
    * @param index the elements index to retrieve 
    * @return the <i>n'th</i> element of this {@link BytesRefArray}
    */
-  public BytesRef get(BytesRef spare, int index) {
+  public BytesRef get(BytesRefBuilder spare, int index) {
     if (lastElement > index) {
       int offset = offsets[index];
       int length = index == lastElement - 1 ? currentOffset - offset
           : offsets[index + 1] - offset;
-      assert spare.offset == 0;
       spare.grow(length);
-      spare.length = length;
-      pool.readBytes(offset, spare.bytes, spare.offset, spare.length);
-      return spare;
+      spare.setLength(length);
+      pool.readBytes(offset, spare.bytes(), 0, spare.length());
+      return spare.get();
     }
     throw new IndexOutOfBoundsException("index " + index
         + " must be less than the size: " + lastElement);
@@ -137,7 +136,7 @@ public final class BytesRefArray {
       @Override
       protected void setPivot(int i) {
         final int index = orderedEntries[i];
-        get(pivot, index);
+        pivot = get(pivotBuilder, index);
       }
       
       @Override
@@ -145,9 +144,11 @@ public final class BytesRefArray {
         final int index = orderedEntries[j];
         return comp.compare(pivot, get(scratch2, index));
       }
-      
-      private final BytesRef pivot = new BytesRef(), scratch1 = new BytesRef(),
-          scratch2 = new BytesRef();
+
+      private BytesRef pivot;
+      private final BytesRefBuilder pivotBuilder = new BytesRefBuilder(),
+          scratch1 = new BytesRefBuilder(),
+          scratch2 = new BytesRefBuilder();
     }.sort(0, size());
     return orderedEntries;
   }
@@ -174,7 +175,7 @@ public final class BytesRefArray {
    * </p>
    */
   public BytesRefIterator iterator(final Comparator<BytesRef> comp) {
-    final BytesRef spare = new BytesRef();
+    final BytesRefBuilder spare = new BytesRefBuilder();
     final int size = size();
     final int[] indices = comp == null ? null : sort(comp);
     return new BytesRefIterator() {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/CharsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/CharsRef.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/CharsRef.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/CharsRef.java Wed Aug 13 09:36:54 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.util.Arrays;
 import java.util.Comparator;
 
 /**
@@ -147,55 +148,6 @@ public final class CharsRef implements C
     // One is a prefix of the other, or, they are equal:
     return this.length - other.length;
   }
-  
-  /**
-   * Copies the given {@link CharsRef} referenced content into this instance.
-   * 
-   * @param other
-   *          the {@link CharsRef} to copy
-   */
-  public void copyChars(CharsRef other) {
-    copyChars(other.chars, other.offset, other.length);
-  }
-
-  /** 
-   * Used to grow the reference array. 
-   * 
-   * In general this should not be used as it does not take the offset into account.
-   * @lucene.internal */
-  public void grow(int newLength) {
-    assert offset == 0;
-    if (chars.length < newLength) {
-      chars = ArrayUtil.grow(chars, newLength);
-    }
-  }
-
-  /**
-   * Copies the given array into this CharsRef.
-   */
-  public void copyChars(char[] otherChars, int otherOffset, int otherLength) {
-    if (chars.length - offset < otherLength) {
-      chars = new char[otherLength];
-      offset = 0;
-    }
-    System.arraycopy(otherChars, otherOffset, chars, offset, otherLength);
-    length = otherLength;
-  }
-
-  /**
-   * Appends the given array to this CharsRef
-   */
-  public void append(char[] otherChars, int otherOffset, int otherLength) {
-    int newLen = length + otherLength;
-    if (chars.length - offset < newLen) {
-      char[] newChars = new char[newLen];
-      System.arraycopy(chars, offset, newChars, 0, length);
-      offset = 0;
-      chars = newChars;
-    }
-    System.arraycopy(otherChars, otherOffset, chars, length+offset, otherLength);
-    length = newLen;
-  }
 
   @Override
   public String toString() {
@@ -292,9 +244,7 @@ public final class CharsRef implements C
    * and an offset of zero.
    */
   public static CharsRef deepCopyOf(CharsRef other) {
-    CharsRef clone = new CharsRef();
-    clone.copyChars(other);
-    return clone;
+    return new CharsRef(Arrays.copyOfRange(other.chars, other.offset, other.offset + other.length), 0, other.length);
   }
   
   /** 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/IntsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/IntsRef.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/IntsRef.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/IntsRef.java Wed Aug 13 09:36:54 2014
@@ -1,5 +1,7 @@
 package org.apache.lucene.util;
 
+import java.util.Arrays;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -132,27 +134,6 @@ public final class IntsRef implements Co
     return this.length - other.length;
   }
 
-  public void copyInts(IntsRef other) {
-    if (ints.length - offset < other.length) {
-      ints = new int[other.length];
-      offset = 0;
-    }
-    System.arraycopy(other.ints, other.offset, ints, offset, other.length);
-    length = other.length;
-  }
-
-  /** 
-   * Used to grow the reference array. 
-   * 
-   * In general this should not be used as it does not take the offset into account.
-   * @lucene.internal */
-  public void grow(int newLength) {
-    assert offset == 0;
-    if (ints.length < newLength) {
-      ints = ArrayUtil.grow(ints, newLength);
-    }
-  }
-
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
@@ -176,9 +157,7 @@ public final class IntsRef implements Co
    * and an offset of zero.
    */
   public static IntsRef deepCopyOf(IntsRef other) {
-    IntsRef clone = new IntsRef();
-    clone.copyInts(other);
-    return clone;
+    return new IntsRef(Arrays.copyOfRange(other.ints, other.offset, other.offset + other.length), 0, other.length);
   }
   
   /** 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongsRef.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongsRef.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongsRef.java Wed Aug 13 09:36:54 2014
@@ -1,5 +1,7 @@
 package org.apache.lucene.util;
 
+import java.util.Arrays;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -131,27 +133,6 @@ public final class LongsRef implements C
     return this.length - other.length;
   }
 
-  public void copyLongs(LongsRef other) {
-    if (longs.length - offset < other.length) {
-      longs = new long[other.length];
-      offset = 0;
-    }
-    System.arraycopy(other.longs, other.offset, longs, offset, other.length);
-    length = other.length;
-  }
-
-  /** 
-   * Used to grow the reference array. 
-   * 
-   * In general this should not be used as it does not take the offset into account.
-   * @lucene.internal */
-  public void grow(int newLength) {
-    assert offset == 0;
-    if (longs.length < newLength) {
-      longs = ArrayUtil.grow(longs, newLength);
-    }
-  }
-
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
@@ -168,16 +149,14 @@ public final class LongsRef implements C
   }
   
   /**
-   * Creates a new IntsRef that points to a copy of the longs from 
+   * Creates a new LongsRef that points to a copy of the longs from 
    * <code>other</code>
    * <p>
    * The returned IntsRef will have a length of other.length
    * and an offset of zero.
    */
   public static LongsRef deepCopyOf(LongsRef other) {
-    LongsRef clone = new LongsRef();
-    clone.copyLongs(other);
-    return clone;
+    return new LongsRef(Arrays.copyOfRange(other.longs, other.offset, other.offset + other.length), 0, other.length);
   }
   
   /** 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java Wed Aug 13 09:36:54 2014
@@ -118,7 +118,7 @@ public final class NumericUtils {
    * @param shift how many bits to strip from the right
    * @param bytes will contain the encoded value
    */
-  public static void longToPrefixCoded(final long val, final int shift, final BytesRef bytes) {
+  public static void longToPrefixCoded(final long val, final int shift, final BytesRefBuilder bytes) {
     longToPrefixCodedBytes(val, shift, bytes);
   }
 
@@ -130,7 +130,7 @@ public final class NumericUtils {
    * @param shift how many bits to strip from the right
    * @param bytes will contain the encoded value
    */
-  public static void intToPrefixCoded(final int val, final int shift, final BytesRef bytes) {
+  public static void intToPrefixCoded(final int val, final int shift, final BytesRefBuilder bytes) {
     intToPrefixCodedBytes(val, shift, bytes);
   }
 
@@ -142,22 +142,19 @@ public final class NumericUtils {
    * @param shift how many bits to strip from the right
    * @param bytes will contain the encoded value
    */
-  public static void longToPrefixCodedBytes(final long val, final int shift, final BytesRef bytes) {
+  public static void longToPrefixCodedBytes(final long val, final int shift, final BytesRefBuilder bytes) {
     if ((shift & ~0x3f) != 0)  // ensure shift is 0..63
       throw new IllegalArgumentException("Illegal shift value, must be 0..63");
     int nChars = (((63-shift)*37)>>8) + 1;    // i/7 is the same as (i*37)>>8 for i in 0..63
-    bytes.offset = 0;
-    bytes.length = nChars+1;   // one extra for the byte that contains the shift info
-    if (bytes.bytes.length < bytes.length) {
-      bytes.bytes = new byte[NumericUtils.BUF_SIZE_LONG];  // use the max
-    }
-    bytes.bytes[0] = (byte)(SHIFT_START_LONG + shift);
+    bytes.setLength(nChars+1);   // one extra for the byte that contains the shift info
+    bytes.grow(BUF_SIZE_LONG);
+    bytes.setByteAt(0, (byte)(SHIFT_START_LONG + shift));
     long sortableBits = val ^ 0x8000000000000000L;
     sortableBits >>>= shift;
     while (nChars > 0) {
       // Store 7 bits per byte for compatibility
       // with UTF-8 encoding of terms
-      bytes.bytes[nChars--] = (byte)(sortableBits & 0x7f);
+      bytes.setByteAt(nChars--, (byte)(sortableBits & 0x7f));
       sortableBits >>>= 7;
     }
   }
@@ -171,22 +168,19 @@ public final class NumericUtils {
    * @param shift how many bits to strip from the right
    * @param bytes will contain the encoded value
    */
-  public static void intToPrefixCodedBytes(final int val, final int shift, final BytesRef bytes) {
+  public static void intToPrefixCodedBytes(final int val, final int shift, final BytesRefBuilder bytes) {
     if ((shift & ~0x1f) != 0)  // ensure shift is 0..31
       throw new IllegalArgumentException("Illegal shift value, must be 0..31");
     int nChars = (((31-shift)*37)>>8) + 1;    // i/7 is the same as (i*37)>>8 for i in 0..63
-    bytes.offset = 0;
-    bytes.length = nChars+1;   // one extra for the byte that contains the shift info
-    if (bytes.bytes.length < bytes.length) {
-      bytes.bytes = new byte[NumericUtils.BUF_SIZE_LONG];  // use the max
-    }
-    bytes.bytes[0] = (byte)(SHIFT_START_INT + shift);
+    bytes.setLength(nChars+1);   // one extra for the byte that contains the shift info
+    bytes.grow(NumericUtils.BUF_SIZE_LONG);  // use the max
+    bytes.setByteAt(0, (byte)(SHIFT_START_INT + shift));
     int sortableBits = val ^ 0x80000000;
     sortableBits >>>= shift;
     while (nChars > 0) {
       // Store 7 bits per byte for compatibility
       // with UTF-8 encoding of terms
-      bytes.bytes[nChars--] = (byte)(sortableBits & 0x7f);
+      bytes.setByteAt(nChars--, (byte)(sortableBits & 0x7f));
       sortableBits >>>= 7;
     }
   }
@@ -430,10 +424,10 @@ public final class NumericUtils {
      * You can use this for e.g. debugging purposes (print out range bounds).
      */
     public void addRange(final long min, final long max, final int shift) {
-      final BytesRef minBytes = new BytesRef(BUF_SIZE_LONG), maxBytes = new BytesRef(BUF_SIZE_LONG);
+      final BytesRefBuilder minBytes = new BytesRefBuilder(), maxBytes = new BytesRefBuilder();
       longToPrefixCodedBytes(min, shift, minBytes);
       longToPrefixCodedBytes(max, shift, maxBytes);
-      addRange(minBytes, maxBytes);
+      addRange(minBytes.get(), maxBytes.get());
     }
   
   }
@@ -459,10 +453,10 @@ public final class NumericUtils {
      * You can use this for e.g. debugging purposes (print out range bounds).
      */
     public void addRange(final int min, final int max, final int shift) {
-      final BytesRef minBytes = new BytesRef(BUF_SIZE_INT), maxBytes = new BytesRef(BUF_SIZE_INT);
+      final BytesRefBuilder minBytes = new BytesRefBuilder(), maxBytes = new BytesRefBuilder();
       intToPrefixCodedBytes(min, shift, minBytes);
       intToPrefixCodedBytes(max, shift, maxBytes);
-      addRange(minBytes, maxBytes);
+      addRange(minBytes.get(), maxBytes.get());
     }
   
   }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java Wed Aug 13 09:36:54 2014
@@ -359,7 +359,7 @@ public final class OfflineSorter {
     PriorityQueue<FileAndTop> queue = new PriorityQueue<FileAndTop>(merges.size()) {
       @Override
       protected boolean lessThan(FileAndTop a, FileAndTop b) {
-        return comparator.compare(a.current, b.current) < 0;
+        return comparator.compare(a.current.get(), b.current.get()) < 0;
       }
     };
 
@@ -380,7 +380,7 @@ public final class OfflineSorter {
       // so it shouldn't make much of a difference (didn't check).
       FileAndTop top;
       while ((top = queue.top()) != null) {
-        out.write(top.current);
+        out.write(top.current.bytes(), 0, top.current.length());
         if (!streams[top.fd].read(top.current)) {
           queue.pop();
         } else {
@@ -420,11 +420,12 @@ public final class OfflineSorter {
 
   static class FileAndTop {
     final int fd;
-    final BytesRef current;
+    final BytesRefBuilder current;
 
-    FileAndTop(int fd, byte [] firstLine) {
+    FileAndTop(int fd, byte[] firstLine) {
       this.fd = fd;
-      this.current = new BytesRef(firstLine);
+      this.current = new BytesRefBuilder();
+      this.current.copyBytes(firstLine, 0, firstLine.length);
     }
   }
 
@@ -519,7 +520,7 @@ public final class OfflineSorter {
      * the header of the next sequence. Returns <code>true</code> otherwise.
      * @throws EOFException if the file ends before the full sequence is read.
      */
-    public boolean read(BytesRef ref) throws IOException {
+    public boolean read(BytesRefBuilder ref) throws IOException {
       short length;
       try {
         length = is.readShort();
@@ -528,16 +529,15 @@ public final class OfflineSorter {
       }
 
       ref.grow(length);
-      ref.offset = 0;
-      ref.length = length;
-      is.readFully(ref.bytes, 0, length);
+      ref.setLength(length);
+      is.readFully(ref.bytes(), 0, length);
       return true;
     }
 
     /**
      * Reads the next entry and returns it if successful.
      * 
-     * @see #read(BytesRef)
+     * @see #read(BytesRefBuilder)
      * 
      * @return Returns <code>null</code> if EOF occurred before the next entry
      * could be read.

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java Wed Aug 13 09:36:54 2014
@@ -1,6 +1,5 @@
 package org.apache.lucene.util;
 
-import java.nio.charset.StandardCharsets;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -123,21 +122,18 @@ public final class UnicodeUtil {
     Character.MIN_SUPPLEMENTARY_CODE_POINT - 
     (UNI_SUR_HIGH_START << HALF_SHIFT) - UNI_SUR_LOW_START;
 
+  /** Maximum number of UTF8 bytes per UTF16 character. */
+  public static final int MAX_UTF8_BYTES_PER_CHAR = 4;
+
   /** Encode characters from a char[] source, starting at
-   *  offset for length chars. After encoding, result.offset will always be 0.
+   *  offset for length chars. It is the responsibility of the
+   *  caller to make sure that the destination array is large enough.
    */
-  // TODO: broken if incoming result.offset != 0
-  public static void UTF16toUTF8(final char[] source, final int offset, final int length, BytesRef result) {
+  public static int UTF16toUTF8(final char[] source, final int offset, final int length, byte[] out) {
 
     int upto = 0;
     int i = offset;
     final int end = offset + length;
-    byte[] out = result.bytes;
-    // Pre-allocate for worst case 4-for-1
-    final int maxLen = length * 4;
-    if (out.length < maxLen)
-      out = result.bytes = new byte[maxLen];
-    result.offset = 0;
 
     while(i < end) {
       
@@ -176,23 +172,17 @@ public final class UnicodeUtil {
       }
     }
     //assert matches(source, offset, length, out, upto);
-    result.length = upto;
+    return upto;
   }
 
   /** Encode characters from this String, starting at offset
-   *  for length characters. After encoding, result.offset will always be 0.
+   *  for length characters. It is the responsibility of the
+   *  caller to make sure that the destination array is large enough.
    */
   // TODO: broken if incoming result.offset != 0
-  public static void UTF16toUTF8(final CharSequence s, final int offset, final int length, BytesRef result) {
+  public static int UTF16toUTF8(final CharSequence s, final int offset, final int length, byte[] out) {
     final int end = offset + length;
 
-    byte[] out = result.bytes;
-    result.offset = 0;
-    // Pre-allocate for worst case 4-for-1
-    final int maxLen = length * 4;
-    if (out.length < maxLen)
-      out = result.bytes = new byte[maxLen];
-
     int upto = 0;
     for(int i=offset;i<end;i++) {
       final int code = (int) s.charAt(i);
@@ -230,7 +220,7 @@ public final class UnicodeUtil {
       }
     }
     //assert matches(s, offset, length, out, upto);
-    result.length = upto;
+    return upto;
   }
 
   // Only called from assert
@@ -405,21 +395,16 @@ public final class UnicodeUtil {
    * <p>This method assumes valid UTF8 input. This method 
    * <strong>does not perform</strong> full UTF8 validation, it will check only the 
    * first byte of each codepoint (for multi-byte sequences any bytes after 
-   * the head are skipped).  
+   * the head are skipped). It is the responsibility of the caller to make sure
+   * that the destination array is large enough.
    * 
    * @throws IllegalArgumentException If invalid codepoint header byte occurs or the 
    *    content is prematurely truncated.
    */
-  public static void UTF8toUTF32(final BytesRef utf8, final IntsRef utf32) {
-    // TODO: broken if incoming result.offset != 0
-    // pre-alloc for worst case
+  public static int UTF8toUTF32(final BytesRef utf8, final int[] ints) {
     // TODO: ints cannot be null, should be an assert
-    if (utf32.ints == null || utf32.ints.length < utf8.length) {
-      utf32.ints = new int[utf8.length];
-    }
     int utf32Count = 0;
     int utf8Upto = utf8.offset;
-    final int[] ints = utf32.ints;
     final byte[] bytes = utf8.bytes;
     final int utf8Limit = utf8.offset + utf8.length;
     while(utf8Upto < utf8Limit) {
@@ -453,8 +438,7 @@ public final class UnicodeUtil {
       ints[utf32Count++] = v;
     }
     
-    utf32.offset = 0;
-    utf32.length = utf32Count;
+    return utf32Count;
   }
 
   /** Shift value for lead surrogate to form a supplementary character. */
@@ -545,17 +529,16 @@ public final class UnicodeUtil {
   }
   
   /**
-   * Interprets the given byte array as UTF-8 and converts to UTF-16. The {@link CharsRef} will be extended if 
-   * it doesn't provide enough space to hold the worst case of each byte becoming a UTF-16 codepoint.
+   * Interprets the given byte array as UTF-8 and converts to UTF-16. It is the
+   * responsibility of the caller to make sure that the destination array is large enough.
    * <p>
    * NOTE: Full characters are read, even if this reads past the length passed (and
    * can result in an ArrayOutOfBoundsException if invalid UTF-8 is passed).
    * Explicit checks for valid UTF-8 are not performed. 
    */
   // TODO: broken if chars.offset != 0
-  public static void UTF8toUTF16(byte[] utf8, int offset, int length, CharsRef chars) {
-    int out_offset = chars.offset = 0;
-    final char[] out = chars.chars =  ArrayUtil.grow(chars.chars, length);
+  public static int UTF8toUTF16(byte[] utf8, int offset, int length, char[] out) {
+    int out_offset = 0;
     final int limit = offset + length;
     while (offset < limit) {
       int b = utf8[offset++]&0xff;
@@ -580,15 +563,15 @@ public final class UnicodeUtil {
         }
       }
     }
-    chars.length = out_offset - chars.offset;
+    return out_offset;
   }
   
   /**
-   * Utility method for {@link #UTF8toUTF16(byte[], int, int, CharsRef)}
-   * @see #UTF8toUTF16(byte[], int, int, CharsRef)
+   * Utility method for {@link #UTF8toUTF16(byte[], int, int, char[])}
+   * @see #UTF8toUTF16(byte[], int, int, char[])
    */
-  public static void UTF8toUTF16(BytesRef bytesRef, CharsRef chars) {
-    UTF8toUTF16(bytesRef.bytes, bytesRef.offset, bytesRef.length, chars);
+  public static int UTF8toUTF16(BytesRef bytesRef, char[] chars) {
+    return UTF8toUTF16(bytesRef.bytes, bytesRef.offset, bytesRef.length, chars);
   }
 
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Wed Aug 13 09:36:54 2014
@@ -26,6 +26,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.search.PrefixTermsEnum;
 import org.apache.lucene.index.SingleTermsEnum;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * Immutable class holding compiled details for a given
@@ -195,7 +196,7 @@ public class CompiledAutomaton {
   
   //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
 
-  private BytesRef addTail(int state, BytesRef term, int idx, int leadLabel) {
+  private BytesRef addTail(int state, BytesRefBuilder term, int idx, int leadLabel) {
     //System.out.println("addTail state=" + state + " term=" + term.utf8ToString() + " idx=" + idx + " leadLabel=" + (char) leadLabel);
     //System.out.println(automaton.toDot());
     // Find biggest transition that's < label
@@ -225,11 +226,9 @@ public class CompiledAutomaton {
       floorLabel = transition.max;
     }
     //System.out.println("  floorLabel=" + (char) floorLabel);
-    if (idx >= term.bytes.length) {
-      term.grow(1+idx);
-    }
+    term.grow(1+idx);
     //if (DEBUG) System.out.println("  add floorLabel=" + (char) floorLabel + " idx=" + idx);
-    term.bytes[idx] = (byte) floorLabel;
+    term.setByteAt(idx, (byte) floorLabel);
 
     state = transition.dest;
     //System.out.println("  dest: " + state);
@@ -241,20 +240,18 @@ public class CompiledAutomaton {
       if (numTransitions == 0) {
         //System.out.println("state=" + state + " 0 trans");
         assert runAutomaton.isAccept(state);
-        term.length = idx;
+        term.setLength(idx);
         //if (DEBUG) System.out.println("  return " + term.utf8ToString());
-        return term;
+        return term.get();
       } else {
         // We are pushing "top" -- so get last label of
         // last transition:
         //System.out.println("get state=" + state + " numTrans=" + numTransitions);
         automaton.getTransition(state, numTransitions-1, transition);
-        if (idx >= term.bytes.length) {
-          term.grow(1+idx);
-        }
+        term.grow(1+idx);
         //if (DEBUG) System.out.println("  push maxLabel=" + (char) lastTransition.max + " idx=" + idx);
         //System.out.println("  add trans dest=" + scratch.dest + " label=" + (char) scratch.max);
-        term.bytes[idx] = (byte) transition.max;
+        term.setByteAt(idx, (byte) transition.max);
         state = transition.dest;
         idx++;
       }
@@ -289,13 +286,12 @@ public class CompiledAutomaton {
   /** Finds largest term accepted by this Automaton, that's
    *  <= the provided input term.  The result is placed in
    *  output; it's fine for output and input to point to
-   *  the same BytesRef.  The returned result is either the
+   *  the same bytes.  The returned result is either the
    *  provided output, or null if there is no floor term
    *  (ie, the provided input term is before the first term
    *  accepted by this Automaton). */
-  public BytesRef floor(BytesRef input, BytesRef output) {
+  public BytesRef floor(BytesRef input, BytesRefBuilder output) {
 
-    output.offset = 0;
     //if (DEBUG) System.out.println("CA.floor input=" + input.utf8ToString());
 
     int state = runAutomaton.getInitialState();
@@ -303,8 +299,8 @@ public class CompiledAutomaton {
     // Special case empty string:
     if (input.length == 0) {
       if (runAutomaton.isAccept(state)) {
-        output.length = 0;
-        return output;
+        output.clear();
+        return output.get();
       } else {
         return null;
       }
@@ -321,13 +317,11 @@ public class CompiledAutomaton {
       if (idx == input.length-1) {
         if (nextState != -1 && runAutomaton.isAccept(nextState)) {
           // Input string is accepted
-          if (idx >= output.bytes.length) {
-            output.grow(1+idx);
-          }
-          output.bytes[idx] = (byte) label;
-          output.length = input.length;
+          output.grow(1+idx);
+          output.setByteAt(idx, (byte) label);
+          output.setLength(input.length);
           //if (DEBUG) System.out.println("  input is accepted; return term=" + output.utf8ToString());
-          return output;
+          return output.get();
         } else {
           nextState = -1;
         }
@@ -341,18 +335,18 @@ public class CompiledAutomaton {
           int numTransitions = automaton.getNumTransitions(state);
           if (numTransitions == 0) {
             assert runAutomaton.isAccept(state);
-            output.length = idx;
+            output.setLength(idx);
             //if (DEBUG) System.out.println("  return " + output.utf8ToString());
-            return output;
+            return output.get();
           } else {
             automaton.getTransition(state, 0, transition);
 
             if (label-1 < transition.min) {
 
               if (runAutomaton.isAccept(state)) {
-                output.length = idx;
+                output.setLength(idx);
                 //if (DEBUG) System.out.println("  return " + output.utf8ToString());
-                return output;
+                return output.get();
               }
               // pop
               if (stack.size() == 0) {
@@ -376,10 +370,8 @@ public class CompiledAutomaton {
         return addTail(state, output, idx, label);
         
       } else {
-        if (idx >= output.bytes.length) {
-          output.grow(1+idx);
-        }
-        output.bytes[idx] = (byte) label;
+        output.grow(1+idx);
+        output.setByteAt(idx, (byte) label);
         stack.add(state);
         state = nextState;
         idx++;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java Wed Aug 13 09:36:54 2014
@@ -19,8 +19,10 @@ package org.apache.lucene.util.automaton
 
 import java.util.*;
 
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.UnicodeUtil;
 
 /**
@@ -277,10 +279,14 @@ final class DaciukMihovAutomatonBuilder 
   public static Automaton build(Collection<BytesRef> input) {
     final DaciukMihovAutomatonBuilder builder = new DaciukMihovAutomatonBuilder();
     
-    CharsRef scratch = new CharsRef();
+    char[] chars = new char[0];
+    CharsRef ref = new CharsRef();
     for (BytesRef b : input) {
-      UnicodeUtil.UTF8toUTF16(b, scratch);
-      builder.add(scratch);
+      chars = ArrayUtil.grow(chars, b.length);
+      final int len = UnicodeUtil.UTF8toUTF16(b, chars);
+      ref.chars = chars;
+      ref.length = len;
+      builder.add(ref);
     }
     
     Automaton.Builder a = new Automaton.Builder();

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java Wed Aug 13 09:36:54 2014
@@ -42,7 +42,9 @@ import java.util.Set;
 
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /**
@@ -1073,7 +1075,7 @@ final public class Operations {
    * @return common prefix
    */
   public static BytesRef getCommonPrefixBytesRef(Automaton a) {
-    BytesRef ref = new BytesRef(10);
+    BytesRefBuilder builder = new BytesRefBuilder();
     HashSet<Integer> visited = new HashSet<>();
     int s = 0;
     boolean done;
@@ -1084,15 +1086,14 @@ final public class Operations {
       if (a.isAccept(s) == false && a.getNumTransitions(s) == 1) {
         a.getTransition(s, 0, t);
         if (t.min == t.max && !visited.contains(t.dest)) {
-          ref.grow(++ref.length);
-          ref.bytes[ref.length - 1] = (byte) t.min;
+          builder.append((byte) t.min);
           s = t.dest;
           done = false;
         }
       }
     } while (!done);
 
-    return ref;
+    return builder.get();
   }
 
   /**
@@ -1271,23 +1272,23 @@ final public class Operations {
       PathNode root = getNode(nodes, 0);
       root.resetState(a, 0);
 
-      IntsRef string = new IntsRef(1);
-      string.length = 1;
+      IntsRefBuilder string = new IntsRefBuilder();
+      string.append(0);
 
-      while (string.length > 0) {
+      while (string.length() > 0) {
 
-        PathNode node = nodes[string.length-1];
+        PathNode node = nodes[string.length()-1];
 
         // Get next label leaving the current node:
         int label = node.nextLabel(a);
 
         if (label != -1) {
-          string.ints[string.length-1] = label;
+          string.setIntAt(string.length()-1, label);
 
           if (a.isAccept(node.to)) {
             // This transition leads to an accept state,
             // so we save the current string:
-            results.add(IntsRef.deepCopyOf(string));
+            results.add(string.toIntsRef());
             if (results.size() == limit) {
               break;
             }
@@ -1302,21 +1303,21 @@ final public class Operations {
             pathStates.set(node.to);
 
             // Push node onto stack:
-            if (nodes.length == string.length) {
+            if (nodes.length == string.length()) {
               PathNode[] newNodes = new PathNode[ArrayUtil.oversize(nodes.length+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
               System.arraycopy(nodes, 0, newNodes, 0, nodes.length);
               nodes = newNodes;
             }
-            getNode(nodes, string.length).resetState(a, node.to);
-            string.length++;
-            string.grow(string.length);
+            getNode(nodes, string.length()).resetState(a, node.to);
+            string.setLength(string.length() + 1);
+            string.grow(string.length());
           }
         } else {
           // No more transitions leaving this state,
           // pop/return back to previous state:
           assert pathStates.get(node.state);
           pathStates.clear(node.state);
-          string.length--;
+          string.setLength(string.length() - 1);
         }
       }
     }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java Wed Aug 13 09:36:54 2014
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.FST.INPUT_TYPE; // javadoc
 import org.apache.lucene.util.packed.PackedInts;
@@ -68,7 +69,7 @@ public class Builder<T> {
   private final boolean doShareNonSingletonNodes;
   private final int shareMaxTailLength;
 
-  private final IntsRef lastInput = new IntsRef();
+  private final IntsRefBuilder lastInput = new IntsRefBuilder();
   
   // for packing
   private final boolean doPackFST;
@@ -202,7 +203,7 @@ public class Builder<T> {
   private void freezeTail(int prefixLenPlus1) throws IOException {
     //System.out.println("  compileTail " + prefixLenPlus1);
     final int downTo = Math.max(1, prefixLenPlus1);
-    for(int idx=lastInput.length; idx >= downTo; idx--) {
+    for(int idx=lastInput.length(); idx >= downTo; idx--) {
 
       boolean doPrune = false;
       boolean doCompile = false;
@@ -254,11 +255,11 @@ public class Builder<T> {
       if (doPrune) {
         // this node doesn't make it -- deref it
         node.clear();
-        parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
+        parent.deleteLast(lastInput.intAt(idx-1), node);
       } else {
 
         if (minSuffixCount2 != 0) {
-          compileAllTargets(node, lastInput.length-idx);
+          compileAllTargets(node, lastInput.length()-idx);
         }
         final T nextFinalOutput = node.output;
 
@@ -273,14 +274,14 @@ public class Builder<T> {
           // this node makes it and we now compile it.  first,
           // compile any targets that were previously
           // undecided:
-          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                             compileNode(node, 1+lastInput.length-idx),
+          parent.replaceLast(lastInput.intAt(idx-1),
+                             compileNode(node, 1+lastInput.length()-idx),
                              nextFinalOutput,
                              isFinal);
         } else {
           // replaceLast just to install
           // nextFinalOutput/isFinal onto the arc
-          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+          parent.replaceLast(lastInput.intAt(idx-1),
                              node,
                              nextFinalOutput,
                              isFinal);
@@ -334,7 +335,7 @@ public class Builder<T> {
       output = NO_OUTPUT;
     }
 
-    assert lastInput.length == 0 || input.compareTo(lastInput) >= 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input;
+    assert lastInput.length() == 0 || input.compareTo(lastInput.get()) >= 0: "inputs are added out of order lastInput=" + lastInput.get() + " vs input=" + input;
     assert validOutput(output);
 
     //System.out.println("\nadd: " + input);
@@ -353,11 +354,11 @@ public class Builder<T> {
     // compare shared prefix length
     int pos1 = 0;
     int pos2 = input.offset;
-    final int pos1Stop = Math.min(lastInput.length, input.length);
+    final int pos1Stop = Math.min(lastInput.length(), input.length);
     while(true) {
       frontier[pos1].inputCount++;
       //System.out.println("  incr " + pos1 + " ct=" + frontier[pos1].inputCount + " n=" + frontier[pos1]);
-      if (pos1 >= pos1Stop || lastInput.ints[pos1] != input.ints[pos2]) {
+      if (pos1 >= pos1Stop || lastInput.intAt(pos1) != input.ints[pos2]) {
         break;
       }
       pos1++;
@@ -387,7 +388,7 @@ public class Builder<T> {
     }
 
     final UnCompiledNode<T> lastNode = frontier[input.length];
-    if (lastInput.length != input.length || prefixLenPlus1 != input.length + 1) {
+    if (lastInput.length() != input.length || prefixLenPlus1 != input.length + 1) {
       lastNode.isFinal = true;
       lastNode.output = NO_OUTPUT;
     }
@@ -419,7 +420,7 @@ public class Builder<T> {
       assert validOutput(output);
     }
 
-    if (lastInput.length == input.length && prefixLenPlus1 == 1+input.length) {
+    if (lastInput.length() == input.length && prefixLenPlus1 == 1+input.length) {
       // same input more than 1 time in a row, mapping to
       // multiple outputs
       lastNode.output = fst.outputs.merge(lastNode.output, output);
@@ -456,11 +457,11 @@ public class Builder<T> {
       }
     } else {
       if (minSuffixCount2 != 0) {
-        compileAllTargets(root, lastInput.length);
+        compileAllTargets(root, lastInput.length());
       }
     }
     //if (DEBUG) System.out.println("  builder.finish root.isFinal=" + root.isFinal + " root.output=" + root.output);
-    fst.finish(compileNode(root, lastInput.length).node);
+    fst.finish(compileNode(root, lastInput.length()).node);
 
     if (doPackFST) {
       return fst.pack(3, Math.max(10, (int) (fst.getNodeCount()/4)), acceptableOverheadRatio);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Wed Aug 13 09:36:54 2014
@@ -18,7 +18,9 @@ package org.apache.lucene.util.fst;
  */
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.FST.Arc;
 import org.apache.lucene.util.fst.FST.BytesReader;
 
@@ -112,8 +114,7 @@ public final class Util {
     
     FST.Arc<Long> scratchArc = new FST.Arc<>();
 
-    final IntsRef result = new IntsRef();
-    
+    final IntsRefBuilder result = new IntsRefBuilder();
     return getByOutput(fst, targetOutput, in, arc, scratchArc, result);
   }
     
@@ -121,7 +122,7 @@ public final class Util {
    * Expert: like {@link Util#getByOutput(FST, long)} except reusing 
    * BytesReader, initial and scratch Arc, and result.
    */
-  public static IntsRef getByOutput(FST<Long> fst, long targetOutput, BytesReader in, Arc<Long> arc, Arc<Long> scratchArc, IntsRef result) throws IOException {
+  public static IntsRef getByOutput(FST<Long> fst, long targetOutput, BytesReader in, Arc<Long> arc, Arc<Long> scratchArc, IntsRefBuilder result) throws IOException {
     long output = arc.output;
     int upto = 0;
 
@@ -133,9 +134,9 @@ public final class Util {
         final long finalOutput = output + arc.nextFinalOutput;
         //System.out.println("  isFinal finalOutput=" + finalOutput);
         if (finalOutput == targetOutput) {
-          result.length = upto;
+          result.setLength(upto);
           //System.out.println("    found!");
-          return result;
+          return result.get();
         } else if (finalOutput > targetOutput) {
           //System.out.println("    not found!");
           return null;
@@ -144,9 +145,7 @@ public final class Util {
 
       if (FST.targetHasArcs(arc)) {
         //System.out.println("  targetHasArcs");
-        if (result.ints.length == upto) {
-          result.grow(1+upto);
-        }
+        result.grow(1+upto);
         
         fst.readFirstRealTargetArc(arc.target, arc, in);
 
@@ -190,7 +189,7 @@ public final class Util {
           }
 
           fst.readNextRealArc(arc, in);
-          result.ints[upto++] = arc.label;
+          result.setIntAt(upto++, arc.label);
           output += arc.output;
 
         } else {
@@ -208,7 +207,7 @@ public final class Util {
               // Recurse on this arc:
               //System.out.println("  match!  break");
               output = minArcOutput;
-              result.ints[upto++] = arc.label;
+              result.setIntAt(upto++, arc.label);
               break;
             } else if (minArcOutput > targetOutput) {
               if (prevArc == null) {
@@ -217,7 +216,7 @@ public final class Util {
               } else {
                 // Recurse on previous arc:
                 arc.copyFrom(prevArc);
-                result.ints[upto++] = arc.label;
+                result.setIntAt(upto++, arc.label);
                 output += arc.output;
                 //System.out.println("    recurse prev label=" + (char) arc.label + " output=" + output);
                 break;
@@ -226,7 +225,7 @@ public final class Util {
               // Recurse on this arc:
               output = minArcOutput;
               //System.out.println("    recurse last label=" + (char) arc.label + " output=" + output);
-              result.ints[upto++] = arc.label;
+              result.setIntAt(upto++, arc.label);
               break;
             } else {
               // Read next arc in this node:
@@ -251,10 +250,10 @@ public final class Util {
   public static class FSTPath<T> {
     public FST.Arc<T> arc;
     public T cost;
-    public final IntsRef input;
+    public final IntsRefBuilder input;
 
     /** Sole constructor */
-    public FSTPath(T cost, FST.Arc<T> arc, IntsRef input) {
+    public FSTPath(T cost, FST.Arc<T> arc, IntsRefBuilder input) {
       this.arc = new FST.Arc<T>().copyFrom(arc);
       this.cost = cost;
       this.input = input;
@@ -278,7 +277,7 @@ public final class Util {
     public int compare(FSTPath<T> a, FSTPath<T> b) {
       int cmp = comparator.compare(a.cost, b.cost);
       if (cmp == 0) {
-        return a.input.compareTo(b.input);
+        return a.input.get().compareTo(b.input.get());
       } else {
         return cmp;
       }
@@ -333,10 +332,9 @@ public final class Util {
           return;
         } else if (comp == 0) {
           // Tie break by alpha sort on the input:
-          path.input.grow(path.input.length+1);
-          path.input.ints[path.input.length++] = path.arc.label;
-          final int cmp = bottom.input.compareTo(path.input);
-          path.input.length--;
+          path.input.append(path.arc.label);
+          final int cmp = bottom.input.get().compareTo(path.input.get());
+          path.input.setLength(path.input.length() - 1);
 
           // We should never see dups:
           assert cmp != 0;
@@ -353,10 +351,9 @@ public final class Util {
 
       // copy over the current input to the new input
       // and add the arc.label to the end
-      IntsRef newInput = new IntsRef(path.input.length+1);     
-      System.arraycopy(path.input.ints, 0, newInput.ints, 0, path.input.length);
-      newInput.ints[path.input.length] = path.arc.label;
-      newInput.length = path.input.length+1;
+      IntsRefBuilder newInput = new IntsRefBuilder();
+      newInput.copyInts(path.input.get());
+      newInput.append(path.arc.label);
       final FSTPath<T> newPath = new FSTPath<>(cost, path.arc, newInput);
 
       queue.add(newPath);
@@ -368,7 +365,7 @@ public final class Util {
 
     /** Adds all leaving arcs, including 'finished' arc, if
      *  the node is final, from this node into the queue.  */
-    public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRef input) throws IOException {
+    public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRefBuilder input) throws IOException {
 
       // De-dup NO_OUTPUT since it must be a singleton:
       if (startOutput.equals(fst.outputs.getNoOutput())) {
@@ -434,8 +431,8 @@ public final class Util {
         if (path.arc.label == FST.END_LABEL) {
           //System.out.println("    empty string!  cost=" + path.cost);
           // Empty string!
-          path.input.length--;
-          results.add(new Result<>(path.input, path.cost));
+          path.input.setLength(path.input.length() - 1);
+          results.add(new Result<>(path.input.get(), path.cost));
           continue;
         }
 
@@ -497,17 +494,15 @@ public final class Util {
             // Add final output:
             //System.out.println("    done!: " + path);
             T finalOutput = fst.outputs.add(path.cost, path.arc.output);
-            if (acceptResult(path.input, finalOutput)) {
+            if (acceptResult(path.input.get(), finalOutput)) {
               //System.out.println("    add result: " + path);
-              results.add(new Result<>(path.input, finalOutput));
+              results.add(new Result<>(path.input.get(), finalOutput));
             } else {
               rejectCount++;
             }
             break;
           } else {
-            path.input.grow(1+path.input.length);
-            path.input.ints[path.input.length] = path.arc.label;
-            path.input.length++;
+            path.input.append(path.arc.label);
             path.cost = fst.outputs.add(path.cost, path.arc.output);
           }
         }
@@ -571,7 +566,7 @@ public final class Util {
 
     // since this search is initialized with a single start node 
     // it is okay to start with an empty input path here
-    searcher.addStartPaths(fromNode, startOutput, allowEmptyString, new IntsRef());
+    searcher.addStartPaths(fromNode, startOutput, allowEmptyString, new IntsRefBuilder());
     return searcher.search();
   } 
 
@@ -817,76 +812,74 @@ public final class Util {
 
   /** Just maps each UTF16 unit (char) to the ints in an
    *  IntsRef. */
-  public static IntsRef toUTF16(CharSequence s, IntsRef scratch) {
+  public static IntsRef toUTF16(CharSequence s, IntsRefBuilder scratch) {
     final int charLimit = s.length();
-    scratch.offset = 0;
-    scratch.length = charLimit;
+    scratch.setLength(charLimit);
     scratch.grow(charLimit);
     for (int idx = 0; idx < charLimit; idx++) {
-      scratch.ints[idx] = (int) s.charAt(idx);
+      scratch.setIntAt(idx, (int) s.charAt(idx));
     }
-    return scratch;
+    return scratch.get();
   }    
 
   /** Decodes the Unicode codepoints from the provided
    *  CharSequence and places them in the provided scratch
    *  IntsRef, which must not be null, returning it. */
-  public static IntsRef toUTF32(CharSequence s, IntsRef scratch) {
+  public static IntsRef toUTF32(CharSequence s, IntsRefBuilder scratch) {
     int charIdx = 0;
     int intIdx = 0;
     final int charLimit = s.length();
     while(charIdx < charLimit) {
       scratch.grow(intIdx+1);
       final int utf32 = Character.codePointAt(s, charIdx);
-      scratch.ints[intIdx] = utf32;
+      scratch.setIntAt(intIdx, utf32);
       charIdx += Character.charCount(utf32);
       intIdx++;
     }
-    scratch.length = intIdx;
-    return scratch;
+    scratch.setLength(intIdx);
+    return scratch.get();
   }
 
   /** Decodes the Unicode codepoints from the provided
    *  char[] and places them in the provided scratch
    *  IntsRef, which must not be null, returning it. */
-  public static IntsRef toUTF32(char[] s, int offset, int length, IntsRef scratch) {
+  public static IntsRef toUTF32(char[] s, int offset, int length, IntsRefBuilder scratch) {
     int charIdx = offset;
     int intIdx = 0;
     final int charLimit = offset + length;
     while(charIdx < charLimit) {
       scratch.grow(intIdx+1);
       final int utf32 = Character.codePointAt(s, charIdx, charLimit);
-      scratch.ints[intIdx] = utf32;
+      scratch.setIntAt(intIdx, utf32);
       charIdx += Character.charCount(utf32);
       intIdx++;
     }
-    scratch.length = intIdx;
-    return scratch;
+    scratch.setLength(intIdx);
+    return scratch.get();
   }
 
   /** Just takes unsigned byte values from the BytesRef and
    *  converts into an IntsRef. */
-  public static IntsRef toIntsRef(BytesRef input, IntsRef scratch) {
-    scratch.grow(input.length);
+  public static IntsRef toIntsRef(BytesRef input, IntsRefBuilder scratch) {
+    scratch.clear();
     for(int i=0;i<input.length;i++) {
-      scratch.ints[i] = input.bytes[i+input.offset] & 0xFF;
+      scratch.append(input.bytes[i+input.offset] & 0xFF);
     }
-    scratch.length = input.length;
-    return scratch;
+    return scratch.get();
   }
 
   /** Just converts IntsRef to BytesRef; you must ensure the
    *  int values fit into a byte. */
-  public static BytesRef toBytesRef(IntsRef input, BytesRef scratch) {
+  public static BytesRef toBytesRef(IntsRef input, BytesRefBuilder scratch) {
     scratch.grow(input.length);
     for(int i=0;i<input.length;i++) {
       int value = input.ints[i+input.offset];
       // NOTE: we allow -128 to 255
       assert value >= Byte.MIN_VALUE && value <= 255: "value " + value + " doesn't fit into byte";
-      scratch.bytes[i] = (byte) value;
+      scratch.setByteAt(i, (byte) value);
     }
-    scratch.length = input.length;
-    return scratch;
+    scratch.setLength(input.length);
+    return scratch.get();
   }
 
   // Uncomment for debugging:

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/mutable/MutableValueStr.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/mutable/MutableValueStr.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/mutable/MutableValueStr.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/mutable/MutableValueStr.java Wed Aug 13 09:36:54 2014
@@ -17,6 +17,7 @@
 package org.apache.lucene.util.mutable;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * {@link MutableValue} implementation of type {@link String}.
@@ -25,12 +26,12 @@ import org.apache.lucene.util.BytesRef;
  * have a <code>value</code> with a length set to 0.
  */
 public class MutableValueStr extends MutableValue {
-  public BytesRef value = new BytesRef();
+  public BytesRefBuilder value = new BytesRefBuilder();
 
   @Override
   public Object toObject() {
-    assert exists || 0 == value.length;
-    return exists ? value.utf8ToString() : null;
+    assert exists || 0 == value.length();
+    return exists ? value.get().utf8ToString() : null;
   }
 
   @Override
@@ -50,16 +51,16 @@ public class MutableValueStr extends Mut
 
   @Override
   public boolean equalsSameType(Object other) {
-    assert exists || 0 == value.length;
+    assert exists || 0 == value.length();
     MutableValueStr b = (MutableValueStr)other;
-    return value.equals(b.value) && exists == b.exists;
+    return value.get().equals(b.value.get()) && exists == b.exists;
   }
 
   @Override
   public int compareSameType(Object other) {
-    assert exists || 0 == value.length;
+    assert exists || 0 == value.length();
     MutableValueStr b = (MutableValueStr)other;
-    int c = value.compareTo(b.value);
+    int c = value.get().compareTo(b.value.get());
     if (c != 0) return c;
     if (exists == b.exists) return 0;
     return exists ? 1 : -1;
@@ -68,7 +69,7 @@ public class MutableValueStr extends Mut
 
   @Override
   public int hashCode() {
-    assert exists || 0 == value.length;
-    return value.hashCode();
+    assert exists || 0 == value.length();
+    return value.get().hashCode();
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestSurrogates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestSurrogates.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestSurrogates.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestSurrogates.java Wed Aug 13 09:36:54 2014
@@ -111,7 +111,7 @@ public class TestSurrogates extends Luce
         assertNotNull(terms);
         TermsEnum termsEnum = terms.iterator(null);
         BytesRef text;
-        BytesRef lastText = null;
+        BytesRefBuilder lastText = null;
         while((text = termsEnum.next()) != null) {
           Term exp = fieldTerms.get(termCount);
           if (VERBOSE) {
@@ -120,9 +120,10 @@ public class TestSurrogates extends Luce
             System.out.println();
           }
           if (lastText == null) {
-            lastText = BytesRef.deepCopyOf(text);
+            lastText = new BytesRefBuilder();
+            lastText.copyBytes(text);
           } else {
-            assertTrue(lastText.compareTo(text) < 0);
+            assertTrue(lastText.get().compareTo(text) < 0);
             lastText.copyBytes(text);
           }
           assertEquals(exp.field(), field);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java Wed Aug 13 09:36:54 2014
@@ -26,6 +26,7 @@ import java.util.concurrent.locks.Reentr
 import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ThreadInterruptedException;
 
@@ -73,10 +74,10 @@ public class TestDocumentsWriterDeleteQu
     assertEquals(uniqueValues, bd1.terms.keySet());
     assertEquals(uniqueValues, bd2.terms.keySet());
     HashSet<Term> frozenSet = new HashSet<>();
+    BytesRefBuilder bytesRef = new BytesRefBuilder();
     for (Term t : queue.freezeGlobalBuffer(null).termsIterable()) {
-      BytesRef bytesRef = new BytesRef();
       bytesRef.copyBytes(t.bytes);
-      frozenSet.add(new Term(t.field, bytesRef));
+      frozenSet.add(new Term(t.field, bytesRef.toBytesRef()));
     }
     assertEquals(uniqueValues, frozenSet);
     assertEquals("num deletes must be 0 after freeze", 0, queue
@@ -202,10 +203,10 @@ public class TestDocumentsWriterDeleteQu
     }
     queue.tryApplyGlobalSlice();
     Set<Term> frozenSet = new HashSet<>();
+    BytesRefBuilder builder = new BytesRefBuilder();
     for (Term t : queue.freezeGlobalBuffer(null).termsIterable()) {
-      BytesRef bytesRef = new BytesRef();
-      bytesRef.copyBytes(t.bytes);
-      frozenSet.add(new Term(t.field, bytesRef));
+      builder.copyBytes(t.bytes);
+      frozenSet.add(new Term(t.field, builder.toBytesRef()));
     }
     assertEquals("num deletes must be 0 after freeze", 0, queue
         .numGlobalTermDeletes());

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java Wed Aug 13 09:36:54 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.nio.CharBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -29,7 +30,8 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -136,7 +138,7 @@ public class TestIndexWriterUnicode exte
   private void checkTermsOrder(IndexReader r, Set<String> allTerms, boolean isTop) throws IOException {
     TermsEnum terms = MultiFields.getFields(r).terms("f").iterator(null);
 
-    BytesRef last = new BytesRef();
+    BytesRefBuilder last = new BytesRefBuilder();
 
     Set<String> seenTerms = new HashSet<>();
 
@@ -146,7 +148,7 @@ public class TestIndexWriterUnicode exte
         break;
       }
 
-      assertTrue(last.compareTo(term) < 0);
+      assertTrue(last.get().compareTo(term) < 0);
       last.copyBytes(term);
 
       final String s = term.utf8ToString();
@@ -173,14 +175,13 @@ public class TestIndexWriterUnicode exte
     char[] buffer = new char[20];
     char[] expected = new char[20];
 
-    BytesRef utf8 = new BytesRef(20);
-    CharsRef utf16 = new CharsRef(20);
+    CharsRefBuilder utf16 = new CharsRefBuilder();
 
     int num = atLeast(100000);
     for (int iter = 0; iter < num; iter++) {
       boolean hasIllegal = fillUnicode(buffer, expected, 0, 20);
 
-      UnicodeUtil.UTF16toUTF8(buffer, 0, 20, utf8);
+      BytesRef utf8 = new BytesRef(CharBuffer.wrap(buffer, 0, 20));
       if (!hasIllegal) {
         byte[] b = new String(buffer, 0, 20).getBytes(StandardCharsets.UTF_8);
         assertEquals(b.length, utf8.length);
@@ -188,18 +189,17 @@ public class TestIndexWriterUnicode exte
           assertEquals(b[i], utf8.bytes[i]);
       }
 
-      UnicodeUtil.UTF8toUTF16(utf8.bytes, 0, utf8.length, utf16);
-      assertEquals(utf16.length, 20);
+      utf16.copyUTF8Bytes(utf8.bytes, 0, utf8.length);
+      assertEquals(utf16.length(), 20);
       for(int i=0;i<20;i++)
-        assertEquals(expected[i], utf16.chars[i]);
+        assertEquals(expected[i], utf16.charAt(i));
     }
   }
 
   // LUCENE-510
   public void testAllUnicodeChars() throws Throwable {
 
-    BytesRef utf8 = new BytesRef(10);
-    CharsRef utf16 = new CharsRef(10);
+    CharsRefBuilder utf16 = new CharsRefBuilder();
     char[] chars = new char[2];
     for(int ch=0;ch<0x0010FFFF;ch++) {
 
@@ -215,14 +215,14 @@ public class TestIndexWriterUnicode exte
         chars[len++] = (char) (((ch-0x0010000) & 0x3FFL) + UnicodeUtil.UNI_SUR_LOW_START);
       }
 
-      UnicodeUtil.UTF16toUTF8(chars, 0, len, utf8);
+      BytesRef utf8 = new BytesRef(CharBuffer.wrap(chars, 0, len));
 
       String s1 = new String(chars, 0, len);
       String s2 = new String(utf8.bytes, 0, utf8.length, StandardCharsets.UTF_8);
       assertEquals("codepoint " + ch, s1, s2);
 
-      UnicodeUtil.UTF8toUTF16(utf8.bytes, 0, utf8.length, utf16);
-      assertEquals("codepoint " + ch, s1, new String(utf16.chars, 0, utf16.length));
+      utf16.copyUTF8Bytes(utf8.bytes, 0, utf8.length);
+      assertEquals("codepoint " + ch, s1, utf16.toString());
 
       byte[] b = s1.getBytes(StandardCharsets.UTF_8);
       assertEquals(utf8.length, b.length);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java Wed Aug 13 09:36:54 2014
@@ -75,8 +75,7 @@ public class TestMultiDocValues extends 
   public void testBinary() throws Exception {
     Directory dir = newDirectory();
     Document doc = new Document();
-    BytesRef ref = new BytesRef();
-    Field field = new BinaryDocValuesField("bytes", ref);
+    Field field = new BinaryDocValuesField("bytes", new BytesRef());
     doc.add(field);
     
     IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
@@ -85,7 +84,8 @@ public class TestMultiDocValues extends 
 
     int numDocs = atLeast(500);
     for (int i = 0; i < numDocs; i++) {
-      ref.copyChars(TestUtil.randomUnicodeString(random()));
+      BytesRef ref = new BytesRef(TestUtil.randomUnicodeString(random()));
+      field.setBytesValue(ref);
       iw.addDocument(doc);
       if (random().nextInt(17) == 0) {
         iw.commit();
@@ -112,8 +112,7 @@ public class TestMultiDocValues extends 
   public void testSorted() throws Exception {
     Directory dir = newDirectory();
     Document doc = new Document();
-    BytesRef ref = new BytesRef();
-    Field field = new SortedDocValuesField("bytes", ref);
+    Field field = new SortedDocValuesField("bytes", new BytesRef());
     doc.add(field);
     
     IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
@@ -122,7 +121,8 @@ public class TestMultiDocValues extends 
 
     int numDocs = atLeast(500);
     for (int i = 0; i < numDocs; i++) {
-      ref.copyChars(TestUtil.randomUnicodeString(random()));
+      BytesRef ref = new BytesRef(TestUtil.randomUnicodeString(random()));
+      field.setBytesValue(ref);
       if (defaultCodecSupportsDocsWithField() && random().nextInt(7) == 0) {
         iw.addDocument(new Document());
       }
@@ -157,8 +157,7 @@ public class TestMultiDocValues extends 
   public void testSortedWithLotsOfDups() throws Exception {
     Directory dir = newDirectory();
     Document doc = new Document();
-    BytesRef ref = new BytesRef();
-    Field field = new SortedDocValuesField("bytes", ref);
+    Field field = new SortedDocValuesField("bytes", new BytesRef());
     doc.add(field);
     
     IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
@@ -167,7 +166,8 @@ public class TestMultiDocValues extends 
 
     int numDocs = atLeast(500);
     for (int i = 0; i < numDocs; i++) {
-      ref.copyChars(TestUtil.randomSimpleString(random(), 2));
+      BytesRef ref = new BytesRef(TestUtil.randomSimpleString(random(), 2));
+      field.setBytesValue(ref);
       iw.addDocument(doc);
       if (random().nextInt(17) == 0) {
         iw.commit();

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java Wed Aug 13 09:36:54 2014
@@ -33,6 +33,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.TestNumericUtils; // NaN arrays
@@ -376,9 +377,12 @@ public class TestNumericRangeQuery32 ext
       if (lower>upper) {
         int a=lower; lower=upper; upper=a;
       }
-      final BytesRef lowerBytes = new BytesRef(NumericUtils.BUF_SIZE_INT), upperBytes = new BytesRef(NumericUtils.BUF_SIZE_INT);
-      NumericUtils.intToPrefixCodedBytes(lower, 0, lowerBytes);
-      NumericUtils.intToPrefixCodedBytes(upper, 0, upperBytes);
+      final BytesRef lowerBytes, upperBytes;
+      BytesRefBuilder b = new BytesRefBuilder();
+      NumericUtils.intToPrefixCodedBytes(lower, 0, b);
+      lowerBytes = b.toBytesRef();
+      NumericUtils.intToPrefixCodedBytes(upper, 0, b);
+      upperBytes = b.toBytesRef();
 
       // test inclusive range
       NumericRangeQuery<Integer> tq=NumericRangeQuery.newIntRange(field, precisionStep, lower, upper, true, true);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java Wed Aug 13 09:36:54 2014
@@ -33,6 +33,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.TestNumericUtils; // NaN arrays
@@ -404,9 +405,12 @@ public class TestNumericRangeQuery64 ext
       if (lower>upper) {
         long a=lower; lower=upper; upper=a;
       }
-      final BytesRef lowerBytes = new BytesRef(NumericUtils.BUF_SIZE_LONG), upperBytes = new BytesRef(NumericUtils.BUF_SIZE_LONG);
-      NumericUtils.longToPrefixCodedBytes(lower, 0, lowerBytes);
-      NumericUtils.longToPrefixCodedBytes(upper, 0, upperBytes);
+      final BytesRef lowerBytes, upperBytes;
+      BytesRefBuilder b = new BytesRefBuilder();
+      NumericUtils.longToPrefixCodedBytes(lower, 0, b);
+      lowerBytes = b.toBytesRef();
+      NumericUtils.longToPrefixCodedBytes(upper, 0, b);
+      upperBytes = b.toBytesRef();
       
       // test inclusive range
       NumericRangeQuery<Long> tq=NumericRangeQuery.newLongRange(field, precisionStep, lower, upper, true, true);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java Wed Aug 13 09:36:54 2014
@@ -37,6 +37,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.UnicodeUtil;
@@ -115,7 +116,7 @@ public class TestRegexpRandom2 extends L
 
     private class SimpleAutomatonTermsEnum extends FilteredTermsEnum {
       CharacterRunAutomaton runAutomaton = new CharacterRunAutomaton(automaton);
-      CharsRef utf16 = new CharsRef(10);
+      CharsRefBuilder utf16 = new CharsRefBuilder();
 
       private SimpleAutomatonTermsEnum(TermsEnum tenum) {
         super(tenum);
@@ -124,8 +125,8 @@ public class TestRegexpRandom2 extends L
       
       @Override
       protected AcceptStatus accept(BytesRef term) throws IOException {
-        UnicodeUtil.UTF8toUTF16(term.bytes, term.offset, term.length, utf16);
-        return runAutomaton.run(utf16.chars, 0, utf16.length) ? 
+        utf16.copyUTF8Bytes(term.bytes, term.offset, term.length);
+        return runAutomaton.run(utf16.chars(), 0, utf16.length()) ? 
             AcceptStatus.YES : AcceptStatus.NO;
       }
     }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java Wed Aug 13 09:36:54 2014
@@ -32,22 +32,22 @@ public class TestByteBlockPool extends L
       List<BytesRef> list = new ArrayList<>();
       int maxLength = atLeast(500);
       final int numValues = atLeast(100);
-      BytesRef ref = new BytesRef();
+      BytesRefBuilder ref = new BytesRefBuilder();
       for (int i = 0; i < numValues; i++) {
         final String value = TestUtil.randomRealisticUnicodeString(random(),
             maxLength);
         list.add(new BytesRef(value));
         ref.copyChars(value);
-        pool.append(ref);
+        pool.append(ref.get());
       }
       // verify
       long position = 0;
       for (BytesRef expected : list) {
         ref.grow(expected.length);
-        ref.length = expected.length;
-        pool.readBytes(position, ref.bytes, ref.offset, ref.length);
-        assertEquals(expected, ref);
-        position += ref.length;
+        ref.setLength(expected.length);
+        pool.readBytes(position, ref.bytes(), 0, ref.length());
+        assertEquals(expected, ref.get());
+        position += ref.length();
       }
       pool.reset(random().nextBoolean(), reuseFirst);
       if (reuseFirst) {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRef.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRef.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRef.java Wed Aug 13 09:36:54 2014
@@ -48,20 +48,4 @@ public class TestBytesRef extends Lucene
     // only for 4.x
     assertEquals("\uFFFF", new BytesRef("\uFFFF").utf8ToString());
   }
-  
-  // LUCENE-3590, AIOOBE if you append to a bytesref with offset != 0
-  public void testAppend() {
-    byte bytes[] = new byte[] { (byte)'a', (byte)'b', (byte)'c', (byte)'d' };
-    BytesRef b = new BytesRef(bytes, 1, 3); // bcd
-    b.append(new BytesRef("e"));
-    assertEquals("bcde", b.utf8ToString());
-  }
-  
-  // LUCENE-3590, AIOOBE if you copy to a bytesref with offset != 0
-  public void testCopyBytes() {
-    byte bytes[] = new byte[] { (byte)'a', (byte)'b', (byte)'c', (byte)'d' };
-    BytesRef b = new BytesRef(bytes, 1, 3); // bcd
-    b.copyBytes(new BytesRef("bcde"));
-    assertEquals("bcde", b.utf8ToString());
-  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java Wed Aug 13 09:36:54 2014
@@ -38,19 +38,19 @@ public class TestBytesRefArray extends L
         stringList.clear();
       }
       int entries = atLeast(500);
-      BytesRef spare = new BytesRef();
+      BytesRefBuilder spare = new BytesRefBuilder();
       int initSize = list.size();
       for (int i = 0; i < entries; i++) {
         String randomRealisticUnicodeString = TestUtil
             .randomRealisticUnicodeString(random);
         spare.copyChars(randomRealisticUnicodeString);
-        assertEquals(i+initSize, list.append(spare));
+        assertEquals(i+initSize, list.append(spare.get()));
         stringList.add(randomRealisticUnicodeString);
       }
       for (int i = 0; i < entries; i++) {
         assertNotNull(list.get(spare, i));
         assertEquals("entry " + i + " doesn't match", stringList.get(i),
-            spare.utf8ToString());
+            spare.get().utf8ToString());
       }
       
       // check random
@@ -58,7 +58,7 @@ public class TestBytesRefArray extends L
         int e = random.nextInt(entries);
         assertNotNull(list.get(spare, e));
         assertEquals("entry " + i + " doesn't match", stringList.get(e),
-            spare.utf8ToString());
+            spare.get().utf8ToString());
       }
       for (int i = 0; i < 2; i++) {
         
@@ -81,13 +81,13 @@ public class TestBytesRefArray extends L
         stringList.clear();
       }
       int entries = atLeast(500);
-      BytesRef spare = new BytesRef();
+      BytesRefBuilder spare = new BytesRefBuilder();
       final int initSize = list.size();
       for (int i = 0; i < entries; i++) {
         String randomRealisticUnicodeString = TestUtil
             .randomRealisticUnicodeString(random);
         spare.copyChars(randomRealisticUnicodeString);
-        assertEquals(initSize + i, list.append(spare));
+        assertEquals(initSize + i, list.append(spare.get()));
         stringList.add(randomRealisticUnicodeString);
       }
       
@@ -95,9 +95,10 @@ public class TestBytesRefArray extends L
       BytesRefIterator iter = list.iterator(BytesRef
           .getUTF8SortedAsUTF16Comparator());
       int i = 0;
-      while ((spare = iter.next()) != null) {
+      BytesRef next;
+      while ((next = iter.next()) != null) {
         assertEquals("entry " + i + " doesn't match", stringList.get(i),
-            spare.utf8ToString());
+            next.utf8ToString());
         i++;
       }
       assertNull(iter.next());

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java Wed Aug 13 09:36:54 2014
@@ -59,7 +59,7 @@ public class TestBytesRefHash extends Lu
    */
   @Test
   public void testSize() {
-    BytesRef ref = new BytesRef();
+    BytesRefBuilder ref = new BytesRefBuilder();
     int num = atLeast(2);
     for (int j = 0; j < num; j++) {
       final int mod = 1+random().nextInt(39);
@@ -70,7 +70,7 @@ public class TestBytesRefHash extends Lu
         } while (str.length() == 0);
         ref.copyChars(str);
         int count = hash.size();
-        int key = hash.add(ref);
+        int key = hash.add(ref.get());
         if (key < 0)
           assertEquals(hash.size(), count);
         else
@@ -91,7 +91,7 @@ public class TestBytesRefHash extends Lu
    */
   @Test
   public void testGet() {
-    BytesRef ref = new BytesRef();
+    BytesRefBuilder ref = new BytesRefBuilder();
     BytesRef scratch = new BytesRef();
     int num = atLeast(2);
     for (int j = 0; j < num; j++) {
@@ -104,7 +104,7 @@ public class TestBytesRefHash extends Lu
         } while (str.length() == 0);
         ref.copyChars(str);
         int count = hash.size();
-        int key = hash.add(ref);
+        int key = hash.add(ref.get());
         if (key >= 0) {
           assertNull(strings.put(str, Integer.valueOf(key)));
           assertEquals(uniqueCount, key);
@@ -117,7 +117,7 @@ public class TestBytesRefHash extends Lu
       }
       for (Entry<String, Integer> entry : strings.entrySet()) {
         ref.copyChars(entry.getKey());
-        assertEquals(ref, hash.get(entry.getValue().intValue(), scratch));
+        assertEquals(ref.get(), hash.get(entry.getValue().intValue(), scratch));
       }
       hash.clear();
       assertEquals(0, hash.size());
@@ -130,7 +130,7 @@ public class TestBytesRefHash extends Lu
    */
   @Test
   public void testCompact() {
-    BytesRef ref = new BytesRef();
+    BytesRefBuilder ref = new BytesRefBuilder();
     int num = atLeast(2);
     for (int j = 0; j < num; j++) {
       int numEntries = 0;
@@ -142,7 +142,7 @@ public class TestBytesRefHash extends Lu
           str = TestUtil.randomRealisticUnicodeString(random(), 1000);
         } while (str.length() == 0);
         ref.copyChars(str);
-        final int key = hash.add(ref);
+        final int key = hash.add(ref.get());
         if (key < 0) {
           assertTrue(bits.get((-key)-1));
         } else {
@@ -172,7 +172,7 @@ public class TestBytesRefHash extends Lu
    */
   @Test
   public void testSort() {
-    BytesRef ref = new BytesRef();
+    BytesRefBuilder ref = new BytesRefBuilder();
     int num = atLeast(2);
     for (int j = 0; j < num; j++) {
       SortedSet<String> strings = new TreeSet<>();
@@ -182,7 +182,7 @@ public class TestBytesRefHash extends Lu
           str = TestUtil.randomRealisticUnicodeString(random(), 1000);
         } while (str.length() == 0);
         ref.copyChars(str);
-        hash.add(ref);
+        hash.add(ref.get());
         strings.add(str);
       }
       // We use the UTF-16 comparator here, because we need to be able to
@@ -193,7 +193,7 @@ public class TestBytesRefHash extends Lu
       BytesRef scratch = new BytesRef();
       for (String string : strings) {
         ref.copyChars(string);
-        assertEquals(ref, hash.get(sort[i++], scratch));
+        assertEquals(ref.get(), hash.get(sort[i++], scratch));
       }
       hash.clear();
       assertEquals(0, hash.size());
@@ -209,7 +209,7 @@ public class TestBytesRefHash extends Lu
    */
   @Test
   public void testAdd() {
-    BytesRef ref = new BytesRef();
+    BytesRefBuilder ref = new BytesRefBuilder();
     BytesRef scratch = new BytesRef();
     int num = atLeast(2);
     for (int j = 0; j < num; j++) {
@@ -222,7 +222,7 @@ public class TestBytesRefHash extends Lu
         } while (str.length() == 0);
         ref.copyChars(str);
         int count = hash.size();
-        int key = hash.add(ref);
+        int key = hash.add(ref.get());
 
         if (key >=0) {
           assertTrue(strings.add(str));
@@ -246,7 +246,7 @@ public class TestBytesRefHash extends Lu
   
   @Test
   public void testFind() throws Exception {
-    BytesRef ref = new BytesRef();
+    BytesRefBuilder ref = new BytesRefBuilder();
     BytesRef scratch = new BytesRef();
     int num = atLeast(2);
     for (int j = 0; j < num; j++) {
@@ -259,14 +259,14 @@ public class TestBytesRefHash extends Lu
         } while (str.length() == 0);
         ref.copyChars(str);
         int count = hash.size();
-        int key = hash.find(ref); //hash.add(ref);
+        int key = hash.find(ref.get()); //hash.add(ref);
         if (key >= 0) { // string found in hash
           assertFalse(strings.add(str));
           assertTrue(key < count);
           assertEquals(str, hash.get(key, scratch).utf8ToString());
           assertEquals(count, hash.size());
         } else {
-          key = hash.add(ref);
+          key = hash.add(ref.get());
           assertTrue(strings.add(str));
           assertEquals(uniqueCount, key);
           assertEquals(hash.size(), count + 1);
@@ -308,7 +308,7 @@ public class TestBytesRefHash extends Lu
    */
   @Test
   public void testAddByPoolOffset() {
-    BytesRef ref = new BytesRef();
+    BytesRefBuilder ref = new BytesRefBuilder();
     BytesRef scratch = new BytesRef();
     BytesRefHash offsetHash = newHash(pool);
     int num = atLeast(2);
@@ -322,7 +322,7 @@ public class TestBytesRefHash extends Lu
         } while (str.length() == 0);
         ref.copyChars(str);
         int count = hash.size();
-        int key = hash.add(ref);
+        int key = hash.add(ref.get());
 
         if (key >= 0) {
           assertTrue(strings.add(str));
@@ -347,9 +347,9 @@ public class TestBytesRefHash extends Lu
       assertAllIn(strings, hash);
       for (String string : strings) {
         ref.copyChars(string);
-        int key = hash.add(ref);
+        int key = hash.add(ref.get());
         BytesRef bytesRef = offsetHash.get((-key)-1, scratch);
-        assertEquals(ref, bytesRef);
+        assertEquals(ref.get(), bytesRef);
       }
 
       hash.clear();
@@ -362,12 +362,12 @@ public class TestBytesRefHash extends Lu
   }
   
   private void assertAllIn(Set<String> strings, BytesRefHash hash) {
-    BytesRef ref = new BytesRef();
+    BytesRefBuilder ref = new BytesRefBuilder();
     BytesRef scratch = new BytesRef();
     int count = hash.size();
     for (String string : strings) {
       ref.copyChars(string);
-      int key  =  hash.add(ref); // add again to check duplicates
+      int key  =  hash.add(ref.get()); // add again to check duplicates
       assertEquals(string, hash.get((-key)-1, scratch).utf8ToString());
       assertEquals(count, hash.size());
       assertTrue("key: " + key + " count: " + count + " string: " + string,

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestCharsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestCharsRef.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestCharsRef.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/TestCharsRef.java Wed Aug 13 09:36:54 2014
@@ -40,7 +40,7 @@ public class TestCharsRef extends Lucene
   }
   
   public void testAppend() {
-    CharsRef ref = new CharsRef();
+    CharsRefBuilder ref = new CharsRefBuilder();
     StringBuilder builder = new StringBuilder();
     int numStrings = atLeast(10);
     for (int i = 0; i < numStrings; i++) {
@@ -51,13 +51,13 @@ public class TestCharsRef extends Lucene
       ref.append(charArray, offset, length);  
     }
     
-    assertEquals(builder.toString(), ref.toString());
+    assertEquals(builder.toString(), ref.get().toString());
   }
   
   public void testCopy() {
     int numIters = atLeast(10);
     for (int i = 0; i < numIters; i++) {
-      CharsRef ref = new CharsRef();
+      CharsRefBuilder ref = new CharsRefBuilder();
       char[] charArray = TestUtil.randomRealisticUnicodeString(random(), 1, 100).toCharArray();
       int offset = random().nextInt(charArray.length);
       int length = charArray.length - offset;
@@ -68,32 +68,6 @@ public class TestCharsRef extends Lucene
     
   }
   
-  // LUCENE-3590, AIOOBE if you append to a charsref with offset != 0
-  public void testAppendChars() {
-    char chars[] = new char[] { 'a', 'b', 'c', 'd' };
-    CharsRef c = new CharsRef(chars, 1, 3); // bcd
-    c.append(new char[] { 'e' }, 0, 1);
-    assertEquals("bcde", c.toString());
-  }
-  
-  // LUCENE-3590, AIOOBE if you copy to a charsref with offset != 0
-  public void testCopyChars() {
-    char chars[] = new char[] { 'a', 'b', 'c', 'd' };
-    CharsRef c = new CharsRef(chars, 1, 3); // bcd
-    char otherchars[] = new char[] { 'b', 'c', 'd', 'e' };
-    c.copyChars(otherchars, 0, 4);
-    assertEquals("bcde", c.toString());
-  }
-  
-  // LUCENE-3590, AIOOBE if you copy to a charsref with offset != 0
-  public void testCopyCharsRef() {
-    char chars[] = new char[] { 'a', 'b', 'c', 'd' };
-    CharsRef c = new CharsRef(chars, 1, 3); // bcd
-    char otherchars[] = new char[] { 'b', 'c', 'd', 'e' };
-    c.copyChars(new CharsRef(otherchars, 0, 4));
-    assertEquals("bcde", c.toString());
-  }
-  
   // LUCENE-3590: fix charsequence to fully obey interface
   public void testCharSequenceCharAt() {
     CharsRef c = new CharsRef("abc");