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 [7/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/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java (original)
+++ lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java Wed Aug 13 09:36:54 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
@@ -64,7 +65,7 @@ public final class IDVersionSegmentTerms
   // assert only:
   private boolean eof;
 
-  final BytesRef term = new BytesRef();
+  final BytesRefBuilder term = new BytesRefBuilder();
   private final FST.BytesReader fstReader;
 
   @SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc<Pair<BytesRef,Long>>[] arcs =
@@ -243,9 +244,7 @@ public final class IDVersionSegmentTerms
       throw new IllegalStateException("terms index was not loaded");
     }
 
-    if (term.bytes.length <= target.length) {
-      term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
-    }
+    term.grow(1 + target.length);
 
     assert clearEOF();
 
@@ -285,7 +284,7 @@ public final class IDVersionSegmentTerms
       targetUpto = 0;
 
       IDVersionSegmentTermsEnumFrame lastFrame = stack[0];
-      assert validIndexPrefix <= term.length: "validIndexPrefix=" + validIndexPrefix + " term.length=" + term.length + " seg=" + fr.parent.segment;
+      assert validIndexPrefix <= term.length(): "validIndexPrefix=" + validIndexPrefix + " term.length=" + term.length() + " seg=" + fr.parent.segment;
 
       final int targetLimit = Math.min(target.length, validIndexPrefix);
 
@@ -296,7 +295,7 @@ public final class IDVersionSegmentTerms
 
       // First compare up to valid seek frames:
       while (targetUpto < targetLimit) {
-        cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+        cmp = (term.byteAt(targetUpto)&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
         // if (DEBUG) {
         //    System.out.println("    cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")"   + " arc.output=" + arc.output + " output=" + output);
         // }
@@ -324,9 +323,9 @@ public final class IDVersionSegmentTerms
         // don't save arc/output/frame; we only do this
         // to find out if the target term is before,
         // equal or after the current term
-        final int targetLimit2 = Math.min(target.length, term.length);
+        final int targetLimit2 = Math.min(target.length, term.length());
         while (targetUpto < targetLimit2) {
-          cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+          cmp = (term.byteAt(targetUpto)&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
           // if (DEBUG) {
           //    System.out.println("    cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
           // }
@@ -337,7 +336,7 @@ public final class IDVersionSegmentTerms
         }
 
         if (cmp == 0) {
-          cmp = term.length - target.length;
+          cmp = term.length() - target.length;
         }
         targetUpto = targetUptoMid;
       }
@@ -365,7 +364,7 @@ public final class IDVersionSegmentTerms
         currentFrame.rewind();
       } else {
         // Target is exactly the same as current term
-        assert term.length == target.length;
+        assert term.length() == target.length;
         if (termExists) {
 
           if (currentFrame.maxIDVersion < minIDVersion) {
@@ -448,8 +447,8 @@ public final class IDVersionSegmentTerms
 
         if (!currentFrame.hasTerms) {
           termExists = false;
-          term.bytes[targetUpto] = (byte) targetLabel;
-          term.length = 1+targetUpto;
+          term.setByteAt(targetUpto, (byte) targetLabel);
+          term.setLength(1+targetUpto);
           // if (DEBUG) {
           //    System.out.println("  FAST NOT_FOUND term=" + brToString(term));
           //  }
@@ -466,12 +465,12 @@ public final class IDVersionSegmentTerms
           if (currentFrame.fp != startFrameFP || changed) {
           //if (targetUpto+1 > term.length) {
             termExists = false;
-            term.bytes[targetUpto] = (byte) targetLabel;
-            term.length = 1+targetUpto;
+            term.setByteAt(targetUpto, (byte) targetLabel);
+            term.setLength(1+targetUpto);
             // if (DEBUG) {
             //   System.out.println("    reset current term");
             // }
-            validIndexPrefix = Math.min(validIndexPrefix, term.length);
+            validIndexPrefix = Math.min(validIndexPrefix, term.length());
           }
             //if (currentFrame.ord != startFrameOrd) {
             //termExists = false;
@@ -512,12 +511,12 @@ public final class IDVersionSegmentTerms
       } else {
         // Follow this arc
         arc = nextArc;
-        if (term.bytes[targetUpto] != (byte) targetLabel) {
+        if (term.byteAt(targetUpto) != (byte) targetLabel) {
           // if (DEBUG) {
           //   System.out.println("  now set termExists=false targetUpto=" + targetUpto + " term=" + term.bytes[targetUpto] + " targetLabel=" + targetLabel);
           // }
           changed = true;
-          term.bytes[targetUpto] = (byte) targetLabel;
+          term.setByteAt(targetUpto, (byte) targetLabel);
           termExists = false;
         }
         // Aggregate output as we go:
@@ -547,7 +546,7 @@ public final class IDVersionSegmentTerms
     // Target term is entirely contained in the index:
     if (!currentFrame.hasTerms) {
       termExists = false;
-      term.length = targetUpto;
+      term.setLength(targetUpto);
       // if (DEBUG) {
       //    System.out.println("  FAST NOT_FOUND term=" + brToString(term));
       //  }
@@ -561,7 +560,7 @@ public final class IDVersionSegmentTerms
     if (currentFrame.maxIDVersion < minIDVersion) {
       // The max version for all terms in this block is lower than the minVersion
       termExists = false;
-      term.length = targetUpto;
+      term.setLength(targetUpto);
       return false;
     }
 
@@ -592,10 +591,8 @@ public final class IDVersionSegmentTerms
     if (fr.index == null) {
       throw new IllegalStateException("terms index was not loaded");
     }
-   
-    if (term.bytes.length <= target.length) {
-      term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
-    }
+
+    term.grow(1 + target.length);
 
     assert clearEOF();
 
@@ -629,7 +626,7 @@ public final class IDVersionSegmentTerms
       targetUpto = 0;
           
       IDVersionSegmentTermsEnumFrame lastFrame = stack[0];
-      assert validIndexPrefix <= term.length;
+      assert validIndexPrefix <= term.length();
 
       final int targetLimit = Math.min(target.length, validIndexPrefix);
 
@@ -640,7 +637,7 @@ public final class IDVersionSegmentTerms
 
       // First compare up to valid seek frames:
       while (targetUpto < targetLimit) {
-        cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+        cmp = (term.byteAt(targetUpto)&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
         //if (DEBUG) {
         //System.out.println("    cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")"   + " arc.output=" + arc.output + " output=" + output);
         //}
@@ -668,9 +665,9 @@ public final class IDVersionSegmentTerms
         final int targetUptoMid = targetUpto;
         // Second compare the rest of the term, but
         // don't save arc/output/frame:
-        final int targetLimit2 = Math.min(target.length, term.length);
+        final int targetLimit2 = Math.min(target.length, term.length());
         while (targetUpto < targetLimit2) {
-          cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+          cmp = (term.byteAt(targetUpto)&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
           //if (DEBUG) {
           //System.out.println("    cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
           //}
@@ -681,7 +678,7 @@ public final class IDVersionSegmentTerms
         }
 
         if (cmp == 0) {
-          cmp = term.length - target.length;
+          cmp = term.length() - target.length;
         }
         targetUpto = targetUptoMid;
       }
@@ -708,7 +705,7 @@ public final class IDVersionSegmentTerms
         currentFrame.rewind();
       } else {
         // Target is exactly the same as current term
-        assert term.length == target.length;
+        assert term.length() == target.length;
         if (termExists) {
           //if (DEBUG) {
           //System.out.println("  target is same as current; return FOUND");
@@ -792,7 +789,7 @@ public final class IDVersionSegmentTerms
         }
       } else {
         // Follow this arc
-        term.bytes[targetUpto] = (byte) targetLabel;
+        term.setByteAt(targetUpto, (byte) targetLabel);
         arc = nextArc;
         // Aggregate output as we go:
         assert arc.output != null;
@@ -852,7 +849,7 @@ public final class IDVersionSegmentTerms
       while(true) {
         IDVersionSegmentTermsEnumFrame f = getFrame(ord);
         assert f != null;
-        final BytesRef prefix = new BytesRef(term.bytes, 0, f.prefix);
+        final BytesRef prefix = new BytesRef(term.bytes(), 0, f.prefix);
         if (f.nextEnt == -1) {
           out.println("    frame " + (isSeekFrame ? "(seek)" : "(next)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + brToString(prefix) + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<VersionBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? VersionBlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? VersionBlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
         } else {
@@ -860,8 +857,8 @@ public final class IDVersionSegmentTerms
         }
         if (fr.index != null) {
           assert !isSeekFrame || f.arc != null: "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
-          if (f.prefix > 0 && isSeekFrame && f.arc.label != (term.bytes[f.prefix-1]&0xFF)) {
-            out.println("      broken seek state: arc.label=" + (char) f.arc.label + " vs term byte=" + (char) (term.bytes[f.prefix-1]&0xFF));
+          if (f.prefix > 0 && isSeekFrame && f.arc.label != (term.byteAt(f.prefix-1)&0xFF)) {
+            out.println("      broken seek state: arc.label=" + (char) f.arc.label + " vs term byte=" + (char) (term.byteAt(f.prefix-1)&0xFF));
             throw new RuntimeException("seek state is broken");
           }
           Pair<BytesRef,Long> output = Util.get(fr.index, prefix);
@@ -925,7 +922,7 @@ public final class IDVersionSegmentTerms
       // this method catches up all internal state so next()
       // works properly:
       //if (DEBUG) System.out.println("  re-seek to pending term=" + term.utf8ToString() + " " + term);
-      final boolean result = seekExact(term);
+      final boolean result = seekExact(term.get());
       assert result;
     }
 
@@ -938,7 +935,7 @@ public final class IDVersionSegmentTerms
         if (currentFrame.ord == 0) {
           //if (DEBUG) System.out.println("  return null");
           assert setEOF();
-          term.length = 0;
+          term.clear();
           validIndexPrefix = 0;
           currentFrame.rewind();
           termExists = false;
@@ -950,7 +947,7 @@ public final class IDVersionSegmentTerms
         if (currentFrame.nextEnt == -1 || currentFrame.lastSubFP != lastFP) {
           // We popped into a frame that's not loaded
           // yet or not scan'd to the right entry
-          currentFrame.scanToFloorFrame(term);
+          currentFrame.scanToFloorFrame(term.get());
           currentFrame.loadBlock();
           currentFrame.scanToSubBlock(lastFP);
         }
@@ -968,7 +965,7 @@ public final class IDVersionSegmentTerms
       if (currentFrame.next()) {
         // Push to new block:
         //if (DEBUG) System.out.println("  push frame");
-        currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+        currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length());
         // This is a "next" frame -- even if it's
         // floor'd we must pretend it isn't so we don't
         // try to scan to the right floor frame:
@@ -977,7 +974,7 @@ public final class IDVersionSegmentTerms
         currentFrame.loadBlock();
       } else {
         //if (DEBUG) System.out.println("  return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord);
-        return term;
+        return term.get();
       }
     }
   }
@@ -985,7 +982,7 @@ public final class IDVersionSegmentTerms
   @Override
   public BytesRef term() {
     assert !eof;
-    return term;
+    return term.get();
   }
 
   @Override
@@ -1031,7 +1028,7 @@ public final class IDVersionSegmentTerms
     //   System.out.println("BTTR.seekExact termState seg=" + segment + " target=" + target.utf8ToString() + " " + target + " state=" + otherState);
     // }
     assert clearEOF();
-    if (target.compareTo(term) != 0 || !termExists) {
+    if (target.compareTo(term.get()) != 0 || !termExists) {
       assert otherState != null && otherState instanceof BlockTermState;
       currentFrame = staticFrame;
       currentFrame.state.copyFrom(otherState);

Modified: lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnumFrame.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnumFrame.java (original)
+++ lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnumFrame.java Wed Aug 13 09:36:54 2014
@@ -266,11 +266,9 @@ final class IDVersionSegmentTermsEnumFra
     nextEnt++;
     suffix = suffixesReader.readVInt();
     startBytePos = suffixesReader.getPosition();
-    ste.term.length = prefix + suffix;
-    if (ste.term.bytes.length < ste.term.length) {
-      ste.term.grow(ste.term.length);
-    }
-    suffixesReader.readBytes(ste.term.bytes, prefix, suffix);
+    ste.term.setLength(prefix + suffix);
+    ste.term.grow(ste.term.length());
+    suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
     // A normal term
     ste.termExists = true;
     return false;
@@ -283,11 +281,9 @@ final class IDVersionSegmentTermsEnumFra
     final int code = suffixesReader.readVInt();
     suffix = code >>> 1;
     startBytePos = suffixesReader.getPosition();
-    ste.term.length = prefix + suffix;
-    if (ste.term.bytes.length < ste.term.length) {
-      ste.term.grow(ste.term.length);
-    }
-    suffixesReader.readBytes(ste.term.bytes, prefix, suffix);
+    ste.term.setLength(prefix + suffix);
+    ste.term.grow(ste.term.length());
+    suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
     if ((code & 1) == 0) {
       // A normal term
       ste.termExists = true;
@@ -417,7 +413,7 @@ final class IDVersionSegmentTermsEnumFra
   // Used only by assert
   private boolean prefixMatches(BytesRef target) {
     for(int bytePos=0;bytePos<prefix;bytePos++) {
-      if (target.bytes[target.offset + bytePos] != ste.term.bytes[bytePos]) {
+      if (target.bytes[target.offset + bytePos] != ste.term.byteAt(bytePos)) {
         return false;
       }
     }
@@ -552,7 +548,7 @@ final class IDVersionSegmentTermsEnumFra
             ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, termLen);
             ste.currentFrame.loadBlock();
             while (ste.currentFrame.next()) {
-              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length);
+              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length());
               ste.currentFrame.loadBlock();
             }
           }
@@ -685,7 +681,7 @@ final class IDVersionSegmentTermsEnumFra
             ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, termLen);
             ste.currentFrame.loadBlock();
             while (ste.currentFrame.next()) {
-              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length);
+              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length());
               ste.currentFrame.loadBlock();
             }
           }
@@ -729,10 +725,8 @@ final class IDVersionSegmentTermsEnumFra
 
   private void fillTerm() {
     final int termLength = prefix + suffix;
-    ste.term.length = prefix + suffix;
-    if (ste.term.bytes.length < termLength) {
-      ste.term.grow(termLength);
-    }
-    System.arraycopy(suffixBytes, startBytePos, ste.term.bytes, prefix, suffix);
+    ste.term.setLength(prefix + suffix);
+    ste.term.grow(termLength);
+    System.arraycopy(suffixBytes, startBytePos, ste.term.bytes(), prefix, suffix);
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java Wed Aug 13 09:36:54 2014
@@ -42,9 +42,10 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
@@ -323,7 +324,7 @@ public final class VersionBlockTreeTerms
       return "BLOCK: " + brToString(prefix);
     }
 
-    public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRef scratchIntsRef) throws IOException {
+    public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
 
       assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1): "isFloor=" + isFloor + " blocks=" + blocks;
       assert this == blocks.get(0);
@@ -389,7 +390,7 @@ public final class VersionBlockTreeTerms
     // TODO: maybe we could add bulk-add method to
     // Builder?  Takes FST and unions it w/ current
     // FST.
-    private void append(Builder<Pair<BytesRef,Long>> builder, FST<Pair<BytesRef,Long>> subIndex, IntsRef scratchIntsRef) throws IOException {
+    private void append(Builder<Pair<BytesRef,Long>> builder, FST<Pair<BytesRef,Long>> subIndex, IntsRefBuilder scratchIntsRef) throws IOException {
       final BytesRefFSTEnum<Pair<BytesRef,Long>> subIndexEnum = new BytesRefFSTEnum<>(subIndex);
       BytesRefFSTEnum.InputOutput<Pair<BytesRef,Long>> indexEnt;
       while((indexEnt = subIndexEnum.next()) != null) {
@@ -402,7 +403,7 @@ public final class VersionBlockTreeTerms
   }
 
   private final RAMOutputStream scratchBytes = new RAMOutputStream();
-  private final IntsRef scratchIntsRef = new IntsRef();
+  private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
 
   class TermsWriter extends TermsConsumer {
     private final FieldInfo fieldInfo;
@@ -416,7 +417,7 @@ public final class VersionBlockTreeTerms
     // startsByPrefix[0] is the index into pending for the first
     // term/sub-block starting with 't'.  We use this to figure out when
     // to write a new block:
-    private final BytesRef lastTerm = new BytesRef();
+    private final BytesRefBuilder lastTerm = new BytesRefBuilder();
     private int[] prefixStarts = new int[8];
 
     private final long[] longs;
@@ -554,7 +555,7 @@ public final class VersionBlockTreeTerms
       boolean hasFloorLeadLabel = isFloor && floorLeadLabel != -1;
 
       final BytesRef prefix = new BytesRef(prefixLength + (hasFloorLeadLabel ? 1 : 0));
-      System.arraycopy(lastTerm.bytes, 0, prefix.bytes, 0, prefixLength);
+      System.arraycopy(lastTerm.bytes(), 0, prefix.bytes, 0, prefixLength);
       prefix.length = prefixLength;
 
       // Write block header:
@@ -770,18 +771,18 @@ public final class VersionBlockTreeTerms
 
     /** Pushes the new term to the top of the stack, and writes new blocks. */
     private void pushTerm(BytesRef text) throws IOException {
-      int limit = Math.min(lastTerm.length, text.length);
+      int limit = Math.min(lastTerm.length(), text.length);
 
       // Find common prefix between last term and current term:
       int pos = 0;
-      while (pos < limit && lastTerm.bytes[pos] == text.bytes[text.offset+pos]) {
+      while (pos < limit && lastTerm.byteAt(pos) == text.bytes[text.offset+pos]) {
         pos++;
       }
 
       // if (DEBUG) System.out.println("  shared=" + pos + "  lastTerm.length=" + lastTerm.length);
 
       // Close the "abandoned" suffix now:
-      for(int i=lastTerm.length-1;i>=pos;i--) {
+      for(int i=lastTerm.length()-1;i>=pos;i--) {
 
         // How many items on top of the stack share the current suffix
         // we are closing:

Modified: lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java (original)
+++ lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java Wed Aug 13 09:36:54 2014
@@ -28,6 +28,7 @@ import org.apache.lucene.search.FuzzyTer
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -103,7 +104,7 @@ public final class SlowFuzzyTermsEnum ex
     
     private final BytesRef prefixBytesRef;
     // used for unicode conversion from BytesRef byte[] to int[]
-    private final IntsRef utf32 = new IntsRef(20);
+    private final IntsRefBuilder utf32 = new IntsRefBuilder();
     
     /**
      * <p>The termCompare method in FuzzyTermEnum uses Levenshtein distance to 
@@ -121,8 +122,8 @@ public final class SlowFuzzyTermsEnum ex
     @Override
     protected final AcceptStatus accept(BytesRef term) {
       if (StringHelper.startsWith(term, prefixBytesRef)) {
-        UnicodeUtil.UTF8toUTF32(term, utf32);
-        final int distance = calcDistance(utf32.ints, realPrefixLength, utf32.length - realPrefixLength);
+        utf32.copyUTF8Bytes(term);
+        final int distance = calcDistance(utf32.ints(), realPrefixLength, utf32.length() - realPrefixLength);
        
         //Integer.MIN_VALUE is the sentinel that Levenshtein stopped early
         if (distance == Integer.MIN_VALUE){
@@ -132,7 +133,7 @@ public final class SlowFuzzyTermsEnum ex
         if (raw == true && distance > maxEdits){
               return AcceptStatus.NO;
         } 
-        final float similarity = calcSimilarity(distance, (utf32.length - realPrefixLength), text.length);
+        final float similarity = calcSimilarity(distance, (utf32.length() - realPrefixLength), text.length);
         
         //if raw is true, then distance must also be <= maxEdits by now
         //given the previous if statement

Modified: lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/regex/JakartaRegexpCapabilities.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/regex/JakartaRegexpCapabilities.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/regex/JakartaRegexpCapabilities.java (original)
+++ lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/regex/JakartaRegexpCapabilities.java Wed Aug 13 09:36:54 2014
@@ -19,10 +19,12 @@ package org.apache.lucene.sandbox.querie
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.regexp.CharacterIterator;
 import org.apache.regexp.RE;
 import org.apache.regexp.REProgram;
+
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 
@@ -113,27 +115,27 @@ public class JakartaRegexpCapabilities i
   class JakartaRegexMatcher implements RegexCapabilities.RegexMatcher {
     
     private RE regexp;
-    private final CharsRef utf16 = new CharsRef(10);
+    private final CharsRefBuilder utf16 = new CharsRefBuilder();
     private final CharacterIterator utf16wrapper = new CharacterIterator() {
 
       @Override
       public char charAt(int pos) {
-        return utf16.chars[pos];
+        return utf16.charAt(pos);
       }
 
       @Override
       public boolean isEnd(int pos) {
-        return pos >= utf16.length;
+        return pos >= utf16.length();
       }
 
       @Override
       public String substring(int beginIndex) {
-        return substring(beginIndex, utf16.length);
+        return substring(beginIndex, utf16.length());
       }
 
       @Override
       public String substring(int beginIndex, int endIndex) {
-        return new String(utf16.chars, beginIndex, endIndex - beginIndex);
+        return new String(utf16.chars(), beginIndex, endIndex - beginIndex);
       }
       
     };
@@ -144,7 +146,7 @@ public class JakartaRegexpCapabilities i
     
     @Override
     public boolean match(BytesRef term) {
-      UnicodeUtil.UTF8toUTF16(term.bytes, term.offset, term.length, utf16);
+      utf16.copyUTF8Bytes(term);
       return regexp.match(utf16wrapper, 0);
     }
 

Modified: lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/regex/JavaUtilRegexCapabilities.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/regex/JavaUtilRegexCapabilities.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/regex/JavaUtilRegexCapabilities.java (original)
+++ lucene/dev/branches/branch_4x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/regex/JavaUtilRegexCapabilities.java Wed Aug 13 09:36:54 2014
@@ -22,6 +22,7 @@ import java.util.regex.Pattern;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.UnicodeUtil;
 
 /**
@@ -104,16 +105,17 @@ public class JavaUtilRegexCapabilities i
   class JavaUtilRegexMatcher implements RegexCapabilities.RegexMatcher {
     private final Pattern pattern;
     private final Matcher matcher;
-    private final CharsRef utf16 = new CharsRef(10);
+    private final CharsRefBuilder utf16 = new CharsRefBuilder();
     
     public JavaUtilRegexMatcher(String regex, int flags) {
       this.pattern = Pattern.compile(regex, flags);
-      this.matcher = this.pattern.matcher(utf16);
+      this.matcher = this.pattern.matcher(utf16.get());
     }
     
     @Override
     public boolean match(BytesRef term) {
-      UnicodeUtil.UTF8toUTF16(term.bytes, term.offset, term.length, utf16);
+      utf16.copyUTF8Bytes(term);
+      utf16.get();
       return matcher.reset().matches();
     }
 

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java Wed Aug 13 09:36:54 2014
@@ -21,6 +21,7 @@ import com.spatial4j.core.context.Spatia
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Rectangle;
 import com.spatial4j.core.shape.Shape;
+
 import org.apache.lucene.document.DoubleField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
@@ -43,6 +44,7 @@ import org.apache.lucene.spatial.query.S
 import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
 import org.apache.lucene.spatial.util.DistanceToShapeValueSource;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.NumericUtils;
 
 
@@ -586,9 +588,9 @@ public class BBoxStrategy extends Spatia
   }
 
   private Query makeNumberTermQuery(String field, double number) {
-    BytesRef bytes = new BytesRef();
+    BytesRefBuilder bytes = new BytesRefBuilder();
     NumericUtils.longToPrefixCodedBytes(NumericUtils.doubleToSortableLong(number), 0, bytes);
-    return new TermQuery(new Term(field, bytes));
+    return new TermQuery(new Term(field, bytes.get()));
   }
 
 }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java Wed Aug 13 09:36:54 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.spatial.prefix
  */
 
 import com.spatial4j.core.shape.Shape;
+
 import org.apache.lucene.queries.TermsFilter;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.spatial.prefix.tree.Cell;

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java Wed Aug 13 09:36:54 2014
@@ -38,6 +38,7 @@ import org.apache.lucene.spatial.util.Di
 import org.apache.lucene.spatial.util.ShapePredicateValueSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -211,14 +212,14 @@ public class SerializedDVStrategy extend
 
       return new FunctionValues() {
         int bytesRefDoc = -1;
-        BytesRef bytesRef = new BytesRef();//scratch
+        BytesRefBuilder bytesRef = new BytesRefBuilder();
 
         boolean fillBytes(int doc) {
           if (bytesRefDoc != doc) {
             bytesRef.copyBytes(docValues.get(doc));
             bytesRefDoc = doc;
           }
-          return bytesRef.length != 0;
+          return bytesRef.length() != 0;
         }
 
         @Override
@@ -227,14 +228,12 @@ public class SerializedDVStrategy extend
         }
 
         @Override
-        public boolean bytesVal(int doc, BytesRef target) {
+        public boolean bytesVal(int doc, BytesRefBuilder target) {
+          target.clear();
           if (fillBytes(doc)) {
-            target.bytes = bytesRef.bytes;
-            target.offset = bytesRef.offset;
-            target.length = bytesRef.length;
+            target.copyBytes(bytesRef);
             return true;
           } else {
-            target.length = 0;
             return false;
           }
         }
@@ -244,7 +243,7 @@ public class SerializedDVStrategy extend
           if (!fillBytes(docId))
             return null;
           DataInputStream dataInput = new DataInputStream(
-              new ByteArrayInputStream(bytesRef.bytes, bytesRef.offset, bytesRef.length));
+              new ByteArrayInputStream(bytesRef.bytes(), 0, bytesRef.length()));
           try {
             return binaryCodec.readShape(dataInput);
           } catch (IOException e) {

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java Wed Aug 13 09:36:54 2014
@@ -28,6 +28,7 @@ import org.apache.lucene.util.ArrayUtil;
 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.UnicodeUtil;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
@@ -317,7 +318,7 @@ public class DirectSpellChecker {
    */
   public SuggestWord[] suggestSimilar(Term term, int numSug, IndexReader ir, 
       SuggestMode suggestMode, float accuracy) throws IOException {
-    final CharsRef spare = new CharsRef();
+    final CharsRefBuilder spare = new CharsRefBuilder();
     String text = term.text();
     if (minQueryLength > 0 && text.codePointCount(0, text.length()) < minQueryLength)
       return new SuggestWord[0];
@@ -367,7 +368,7 @@ public class DirectSpellChecker {
     for (ScoreTerm s : terms) {
       SuggestWord suggestion = new SuggestWord();
       if (s.termAsString == null) {
-        UnicodeUtil.UTF8toUTF16(s.term, spare);
+        spare.copyUTF8Bytes(s.term);
         s.termAsString = spare.toString();
       }
       suggestion.string = s.termAsString;
@@ -399,7 +400,7 @@ public class DirectSpellChecker {
    * @throws IOException If I/O related errors occur
    */
   protected Collection<ScoreTerm> suggestSimilar(Term term, int numSug, IndexReader ir, int docfreq, int editDistance,
-                                                 float accuracy, final CharsRef spare) throws IOException {
+                                                 float accuracy, final CharsRefBuilder spare) throws IOException {
     
     AttributeSource atts = new AttributeSource();
     MaxNonCompetitiveBoostAttribute maxBoostAtt =
@@ -440,7 +441,7 @@ public class DirectSpellChecker {
         // undo FuzzyTermsEnum's scale factor for a real scaled lev score
         score = boost / e.getScaleFactor() + e.getMinSimilarity();
       } else {
-        UnicodeUtil.UTF8toUTF16(candidateTerm, spare);
+        spare.copyUTF8Bytes(candidateTerm);
         termAsString = spare.toString();
         score = distance.getDistance(term.text(), termAsString);
       }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java Wed Aug 13 09:36:54 2014
@@ -27,6 +27,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.suggest.InputIterator;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * HighFrequencyDictionary: terms taken from the given field
@@ -62,7 +63,7 @@ public class HighFrequencyDictionary imp
   }
 
   final class HighFrequencyIterator implements InputIterator {
-    private final BytesRef spare = new BytesRef();
+    private final BytesRefBuilder spare = new BytesRefBuilder();
     private final TermsEnum termsEnum;
     private int minNumDocs;
     private long freq;
@@ -94,7 +95,7 @@ public class HighFrequencyDictionary imp
           if (isFrequent(termsEnum.docFreq())) {
             freq = termsEnum.docFreq();
             spare.copyBytes(next);
-            return spare;
+            return spare.get();
           }
         }
       }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java Wed Aug 13 09:36:54 2014
@@ -24,6 +24,7 @@ import java.io.*;
 
 import org.apache.lucene.search.suggest.InputIterator;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IOUtils;
 
@@ -72,7 +73,7 @@ public class PlainTextDictionary impleme
 
   final class FileIterator implements BytesRefIterator {
     private boolean done = false;
-    private final BytesRef spare = new BytesRef();
+    private final BytesRefBuilder spare = new BytesRefBuilder();
     @Override
     public BytesRef next() throws IOException {
       if (done) {
@@ -84,7 +85,7 @@ public class PlainTextDictionary impleme
         String line;
         if ((line = in.readLine()) != null) {
           spare.copyChars(line);
-          result = spare;
+          result = spare.get();
         } else {
           done = true;
           IOUtils.close(in);

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferedInputIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferedInputIterator.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferedInputIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferedInputIterator.java Wed Aug 13 09:36:54 2014
@@ -26,6 +26,7 @@ import java.util.Set;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefArray;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.Counter;
 
 /**
@@ -44,8 +45,8 @@ public class BufferedInputIterator imple
   protected int curPos = -1;
   /** buffered weights, parallel with {@link #entries} */
   protected long[] freqs = new long[1];
-  private final BytesRef spare = new BytesRef();
-  private final BytesRef payloadSpare = new BytesRef();
+  private final BytesRefBuilder spare = new BytesRefBuilder();
+  private final BytesRefBuilder payloadSpare = new BytesRefBuilder();
   private final boolean hasPayloads;
   private final Comparator<BytesRef> comp;
 
@@ -82,7 +83,7 @@ public class BufferedInputIterator imple
   public BytesRef next() throws IOException {
     if (++curPos < entries.size()) {
       entries.get(spare, curPos);
-      return spare;
+      return spare.get();
     }
     return null;
   }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java Wed Aug 13 09:36:54 2014
@@ -19,10 +19,12 @@ package org.apache.lucene.search.suggest
 
 import java.io.IOException;
 import java.util.Comparator;
+
 import org.apache.lucene.search.spell.TermFreqIterator;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefArray;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.Counter;
 
 /**
@@ -37,7 +39,7 @@ public class BufferingTermFreqIteratorWr
   protected int curPos = -1;
   /** buffered weights, parallel with {@link #entries} */
   protected long[] freqs = new long[1];
-  private final BytesRef spare = new BytesRef();
+  private final BytesRefBuilder spare = new BytesRefBuilder();
   private final Comparator<BytesRef> comp;
   
   /** Creates a new iterator, buffering entries from the specified iterator */
@@ -64,7 +66,7 @@ public class BufferingTermFreqIteratorWr
   public BytesRef next() throws IOException {
     if (++curPos < entries.size()) {
       entries.get(spare, curPos);
-      return spare;
+      return spare.get();
     }
     return null;
   }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java Wed Aug 13 09:36:54 2014
@@ -25,6 +25,7 @@ import java.util.Set;
 
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 
 
@@ -119,8 +120,8 @@ public class FileDictionary implements D
 
   final class FileIterator implements InputIterator {
     private long curWeight;
-    private final BytesRef spare = new BytesRef();
-    private BytesRef curPayload = new BytesRef();
+    private final BytesRefBuilder spare = new BytesRefBuilder();
+    private BytesRefBuilder curPayload = new BytesRefBuilder();
     private boolean isFirstLine = true;
     private boolean hasPayloads = false;
     
@@ -160,7 +161,7 @@ public class FileDictionary implements D
       }
       if (isFirstLine) {
         isFirstLine = false;
-        return spare;
+        return spare.get();
       }
       line = in.readLine();
       if (line != null) {
@@ -177,16 +178,16 @@ public class FileDictionary implements D
           spare.copyChars(fields[0]);
           readWeight(fields[1]);
           if (hasPayloads) { // have an empty payload
-            curPayload = new BytesRef();
+            curPayload = new BytesRefBuilder();
           }
         } else { // only term
           spare.copyChars(fields[0]);
           curWeight = 1;
           if (hasPayloads) {
-            curPayload = new BytesRef();
+            curPayload = new BytesRefBuilder();
           }
         }
-        return spare;
+        return spare.get();
       } else {
         done = true;
         IOUtils.close(in);
@@ -201,7 +202,7 @@ public class FileDictionary implements D
 
     @Override
     public BytesRef payload() {
-      return (hasPayloads) ? curPayload : null;
+      return (hasPayloads) ? curPayload.get() : null;
     }
 
     @Override

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java Wed Aug 13 09:36:54 2014
@@ -27,6 +27,7 @@ import org.apache.lucene.store.ByteArray
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.OfflineSorter.ByteSequencesReader;
@@ -48,7 +49,7 @@ public class SortedInputIterator impleme
   private boolean done = false;
   
   private long weight;
-  private final BytesRef scratch = new BytesRef();
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
   private BytesRef payload = new BytesRef();
   private Set<BytesRef> contexts = null;
   
@@ -81,15 +82,16 @@ public class SortedInputIterator impleme
     try {
       ByteArrayDataInput input = new ByteArrayDataInput();
       if (reader.read(scratch)) {
-        weight = decode(scratch, input);
+      final BytesRef bytes = scratch.get();
+        weight = decode(bytes, input);
         if (hasPayloads) {
-          payload = decodePayload(scratch, input);
+          payload = decodePayload(bytes, input);
         }
         if (hasContexts) {
-          contexts = decodeContexts(scratch, input);
+          contexts = decodeContexts(bytes, input);
         }
         success = true;
-        return scratch;
+        return bytes;
       }
       close();
       success = done = true;

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java Wed Aug 13 09:36:54 2014
@@ -26,6 +26,7 @@ import org.apache.lucene.store.ByteArray
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.OfflineSorter.ByteSequencesReader;
@@ -45,7 +46,7 @@ public class SortedTermFreqIteratorWrapp
   private boolean done = false;
   
   private long weight;
-  private final BytesRef scratch = new BytesRef();
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
   
   /**
    * Creates a new sorted wrapper, using {@link
@@ -79,9 +80,9 @@ public class SortedTermFreqIteratorWrapp
     try {
       ByteArrayDataInput input = new ByteArrayDataInput();
       if (reader.read(scratch)) {
-        weight = decode(scratch, input);
+        weight = decode(scratch.get(), input);
         success = true;
-        return scratch;
+        return scratch.get();
       }
       close();
       success = done = true;

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/UnsortedInputIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/UnsortedInputIterator.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/UnsortedInputIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/UnsortedInputIterator.java Wed Aug 13 09:36:54 2014
@@ -22,6 +22,7 @@ import java.util.Random;
 import java.util.Set;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * This wrapper buffers the incoming elements and makes sure they are in
@@ -32,8 +33,8 @@ public class UnsortedInputIterator exten
   // TODO keep this for now
   private final int[] ords;
   private int currentOrd = -1;
-  private final BytesRef spare = new BytesRef();
-  private final BytesRef payloadSpare = new BytesRef();
+  private final BytesRefBuilder spare = new BytesRefBuilder();
+  private final BytesRefBuilder payloadSpare = new BytesRefBuilder();
   /** 
    * Creates a new iterator, wrapping the specified iterator and
    * returning elements in a random order.

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Wed Aug 13 09:36:54 2014
@@ -38,9 +38,12 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.Operations;
@@ -420,7 +423,7 @@ public class AnalyzingSuggester extends 
 
     OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(tempInput);
     OfflineSorter.ByteSequencesReader reader = null;
-    BytesRef scratch = new BytesRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
 
     TokenStreamToAutomaton ts2a = getTokenStreamToAutomaton();
 
@@ -441,10 +444,10 @@ public class AnalyzingSuggester extends 
           Util.toBytesRef(path, scratch);
           
           // length of the analyzed text (FST input)
-          if (scratch.length > Short.MAX_VALUE-2) {
-            throw new IllegalArgumentException("cannot handle analyzed forms > " + (Short.MAX_VALUE-2) + " in length (got " + scratch.length + ")");
+          if (scratch.length() > Short.MAX_VALUE-2) {
+            throw new IllegalArgumentException("cannot handle analyzed forms > " + (Short.MAX_VALUE-2) + " in length (got " + scratch.length() + ")");
           }
-          short analyzedLength = (short) scratch.length;
+          short analyzedLength = (short) scratch.length();
 
           // compute the required length:
           // analyzed sequence + weight (4) + surface + analyzedLength (short)
@@ -469,7 +472,7 @@ public class AnalyzingSuggester extends 
 
           output.writeShort(analyzedLength);
 
-          output.writeBytes(scratch.bytes, scratch.offset, scratch.length);
+          output.writeBytes(scratch.bytes(), 0, scratch.length());
 
           output.writeInt(encodeWeight(iterator.weight()));
 
@@ -505,10 +508,10 @@ public class AnalyzingSuggester extends 
       Builder<Pair<Long,BytesRef>> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
 
       // Build FST:
-      BytesRef previousAnalyzed = null;
-      BytesRef analyzed = new BytesRef();
+      BytesRefBuilder previousAnalyzed = null;
+      BytesRefBuilder analyzed = new BytesRefBuilder();
       BytesRef surface = new BytesRef();
-      IntsRef scratchInts = new IntsRef();
+      IntsRefBuilder scratchInts = new IntsRefBuilder();
       ByteArrayDataInput input = new ByteArrayDataInput();
 
       // Used to remove duplicate surface forms (but we
@@ -519,28 +522,28 @@ public class AnalyzingSuggester extends 
 
       int dedup = 0;
       while (reader.read(scratch)) {
-        input.reset(scratch.bytes, scratch.offset, scratch.length);
+        input.reset(scratch.bytes(), 0, scratch.length());
         short analyzedLength = input.readShort();
         analyzed.grow(analyzedLength+2);
-        input.readBytes(analyzed.bytes, 0, analyzedLength);
-        analyzed.length = analyzedLength;
+        input.readBytes(analyzed.bytes(), 0, analyzedLength);
+        analyzed.setLength(analyzedLength);
 
         long cost = input.readInt();
 
-        surface.bytes = scratch.bytes;
+        surface.bytes = scratch.bytes();
         if (hasPayloads) {
           surface.length = input.readShort();
           surface.offset = input.getPosition();
         } else {
           surface.offset = input.getPosition();
-          surface.length = scratch.length - surface.offset;
+          surface.length = scratch.length() - surface.offset;
         }
         
         if (previousAnalyzed == null) {
-          previousAnalyzed = new BytesRef();
-          previousAnalyzed.copyBytes(analyzed);
+          previousAnalyzed = new BytesRefBuilder();
+          previousAnalyzed.copyBytes(analyzed.get());
           seenSurfaceForms.add(BytesRef.deepCopyOf(surface));
-        } else if (analyzed.equals(previousAnalyzed)) {
+        } else if (analyzed.get().equals(previousAnalyzed.get())) {
           dedup++;
           if (dedup >= maxSurfaceFormsPerAnalyzedForm) {
             // More than maxSurfaceFormsPerAnalyzedForm
@@ -566,23 +569,22 @@ public class AnalyzingSuggester extends 
 
         // NOTE: must be byte 0 so we sort before whatever
         // is next
-        analyzed.bytes[analyzed.offset+analyzed.length] = 0;
-        analyzed.bytes[analyzed.offset+analyzed.length+1] = (byte) dedup;
-        analyzed.length += 2;
+        analyzed.append((byte) 0);
+        analyzed.append((byte) dedup);
 
-        Util.toIntsRef(analyzed, scratchInts);
+        Util.toIntsRef(analyzed.get(), scratchInts);
         //System.out.println("ADD: " + scratchInts + " -> " + cost + ": " + surface.utf8ToString());
         if (!hasPayloads) {
-          builder.add(scratchInts, outputs.newPair(cost, BytesRef.deepCopyOf(surface)));
+          builder.add(scratchInts.get(), outputs.newPair(cost, BytesRef.deepCopyOf(surface)));
         } else {
           int payloadOffset = input.getPosition() + surface.length;
-          int payloadLength = scratch.length - payloadOffset;
+          int payloadLength = scratch.length() - payloadOffset;
           BytesRef br = new BytesRef(surface.length + 1 + payloadLength);
           System.arraycopy(surface.bytes, surface.offset, br.bytes, 0, surface.length);
           br.bytes[surface.length] = PAYLOAD_SEP;
-          System.arraycopy(scratch.bytes, payloadOffset, br.bytes, surface.length+1, payloadLength);
+          System.arraycopy(scratch.bytes(), payloadOffset, br.bytes, surface.length+1, payloadLength);
           br.length = br.bytes.length;
-          builder.add(scratchInts, outputs.newPair(cost, br));
+          builder.add(scratchInts.get(), outputs.newPair(cost, br));
         }
       }
       fst = builder.finish();
@@ -624,7 +626,7 @@ public class AnalyzingSuggester extends 
     return true;
   }
 
-  private LookupResult getLookupResult(Long output1, BytesRef output2, CharsRef spare) {
+  private LookupResult getLookupResult(Long output1, BytesRef output2, CharsRefBuilder spare) {
     LookupResult result;
     if (hasPayloads) {
       int sepIndex = -1;
@@ -637,14 +639,14 @@ public class AnalyzingSuggester extends 
       assert sepIndex != -1;
       spare.grow(sepIndex);
       final int payloadLen = output2.length - sepIndex - 1;
-      UnicodeUtil.UTF8toUTF16(output2.bytes, output2.offset, sepIndex, spare);
+      spare.copyUTF8Bytes(output2.bytes, output2.offset, sepIndex);
       BytesRef payload = new BytesRef(payloadLen);
       System.arraycopy(output2.bytes, sepIndex+1, payload.bytes, 0, payloadLen);
       payload.length = payloadLen;
       result = new LookupResult(spare.toString(), decodeWeight(output1), payload);
     } else {
       spare.grow(output2.length);
-      UnicodeUtil.UTF8toUTF16(output2, spare);
+      spare.copyUTF8Bytes(output2);
       result = new LookupResult(spare.toString(), decodeWeight(output1));
     }
 
@@ -695,7 +697,7 @@ public class AnalyzingSuggester extends 
     try {
       Automaton lookupAutomaton = toLookupAutomaton(key);
 
-      final CharsRef spare = new CharsRef();
+      final CharsRefBuilder spare = new CharsRefBuilder();
 
       //System.out.println("  now intersect exactFirst=" + exactFirst);
     

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java Wed Aug 13 09:36:54 2014
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.Transition;
 import org.apache.lucene.util.fst.FST;
@@ -51,10 +52,10 @@ public class FSTUtil {
     T output;
 
     /** Input of the path so far: */
-    public final IntsRef input;
+    public final IntsRefBuilder input;
 
     /** Sole constructor. */
-    public Path(int state, FST.Arc<T> fstNode, T output, IntsRef input) {
+    public Path(int state, FST.Arc<T> fstNode, T output, IntsRefBuilder input) {
       this.state = state;
       this.fstNode = fstNode;
       this.output = output;
@@ -77,7 +78,7 @@ public class FSTUtil {
 
     queue.add(new Path<>(0, fst
         .getFirstArc(new FST.Arc<T>()), fst.outputs.getNoOutput(),
-        new IntsRef()));
+        new IntsRefBuilder()));
     
     final FST.Arc<T> scratchArc = new FST.Arc<>();
     final FST.BytesReader fstReader = fst.getBytesReader();
@@ -93,7 +94,7 @@ public class FSTUtil {
         continue;
       }
       
-      IntsRef currentInput = path.input;
+      IntsRefBuilder currentInput = path.input;
       int count = a.initTransition(path.state, t);
       for (int i=0;i<count;i++) {
         a.getNextTransition(t);
@@ -103,10 +104,9 @@ public class FSTUtil {
           final FST.Arc<T> nextArc = fst.findTargetArc(t.min,
               path.fstNode, scratchArc, fstReader);
           if (nextArc != null) {
-            final IntsRef newInput = new IntsRef(currentInput.length + 1);
-            newInput.copyInts(currentInput);
-            newInput.ints[currentInput.length] = t.min;
-            newInput.length = currentInput.length + 1;
+            final IntsRefBuilder newInput = new IntsRefBuilder();
+            newInput.copyInts(currentInput.get());
+            newInput.append(t.min);
             queue.add(new Path<>(t.dest, new FST.Arc<T>()
                 .copyFrom(nextArc), fst.outputs
                 .add(path.output, nextArc.output), newInput));
@@ -125,10 +125,9 @@ public class FSTUtil {
             assert nextArc.label <=  max;
             assert nextArc.label >= min : nextArc.label + " "
                 + min;
-            final IntsRef newInput = new IntsRef(currentInput.length + 1);
-            newInput.copyInts(currentInput);
-            newInput.ints[currentInput.length] = nextArc.label;
-            newInput.length = currentInput.length + 1;
+            final IntsRefBuilder newInput = new IntsRefBuilder();
+            newInput.copyInts(currentInput.get());
+            newInput.append(nextArc.label);
             queue.add(new Path<>(t.dest, new FST.Arc<T>()
                 .copyFrom(nextArc), fst.outputs
                 .add(path.output, nextArc.output), newInput));

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java Wed Aug 13 09:36:54 2014
@@ -50,9 +50,12 @@ import org.apache.lucene.store.DataOutpu
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.Version;
@@ -350,7 +353,7 @@ public class FreeTextSuggester extends L
       Outputs<Long> outputs = PositiveIntOutputs.getSingleton();
       Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
 
-      IntsRef scratchInts = new IntsRef();
+      IntsRefBuilder scratchInts = new IntsRefBuilder();
       while (true) {
         BytesRef term = termsEnum.next();
         if (term == null) {
@@ -484,7 +487,7 @@ public class FreeTextSuggester extends L
       PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
       ts.reset();
       
-      BytesRef[] lastTokens = new BytesRef[grams];
+      BytesRefBuilder[] lastTokens = new BytesRefBuilder[grams];
       //System.out.println("lookup: key='" + key + "'");
       
       // Run full analysis, but save only the
@@ -507,7 +510,9 @@ public class FreeTextSuggester extends L
           throw new IllegalArgumentException("tokens must not contain separator byte; got token=" + tokenBytes + " but gramCount=" + gramCount + " does not match recalculated count=" + countGrams(tokenBytes));
         }
         maxEndOffset = Math.max(maxEndOffset, offsetAtt.endOffset());
-        lastTokens[gramCount-1] = BytesRef.deepCopyOf(tokenBytes);
+        BytesRefBuilder b = new BytesRefBuilder();
+        b.append(tokenBytes);
+        lastTokens[gramCount-1] = b;
       }
       ts.end();
       
@@ -535,16 +540,14 @@ public class FreeTextSuggester extends L
         // all bigrams starting w/ foo, and not any unigrams
         // starting with "foo":
         for(int i=grams-1;i>0;i--) {
-          BytesRef token = lastTokens[i-1];
+          BytesRefBuilder token = lastTokens[i-1];
           if (token == null) {
             continue;
           }
-          token.grow(token.length+1);
-          token.bytes[token.length] = separator;
-          token.length++;
+          token.append(separator);
           lastTokens[i] = token;
         }
-        lastTokens[0] = new BytesRef();
+        lastTokens[0] = new BytesRefBuilder();
       }
       
       Arc<Long> arc = new Arc<>();
@@ -563,9 +566,9 @@ public class FreeTextSuggester extends L
       final Set<BytesRef> seen = new HashSet<>();
       
       for(int gram=grams-1;gram>=0;gram--) {
-        BytesRef token = lastTokens[gram];
+        BytesRefBuilder token = lastTokens[gram];
         // Don't make unigram predictions from empty string:
-        if (token == null || (token.length == 0 && key.length() > 0)) {
+        if (token == null || (token.length() == 0 && key.length() > 0)) {
           // Input didn't have enough tokens:
           //System.out.println("  gram=" + gram + ": skip: not enough input");
           continue;
@@ -586,7 +589,7 @@ public class FreeTextSuggester extends L
         //Pair<Long,BytesRef> prefixOutput = null;
         Long prefixOutput = null;
         try {
-          prefixOutput = lookupPrefix(fst, bytesReader, token, arc);
+          prefixOutput = lookupPrefix(fst, bytesReader, token.get(), arc);
         } catch (IOException bogus) {
           throw new RuntimeException(bogus);
         }
@@ -608,27 +611,25 @@ public class FreeTextSuggester extends L
         
         BytesRef lastTokenFragment = null;
         
-        for(int i=token.length-1;i>=0;i--) {
-          if (token.bytes[token.offset+i] == separator) {
-            BytesRef context = new BytesRef(token.bytes, token.offset, i);
-            Long output = Util.get(fst, Util.toIntsRef(context, new IntsRef()));
+        for(int i=token.length()-1;i>=0;i--) {
+          if (token.byteAt(i) == separator) {
+            BytesRef context = new BytesRef(token.bytes(), 0, i);
+            Long output = Util.get(fst, Util.toIntsRef(context, new IntsRefBuilder()));
             assert output != null;
             contextCount = decodeWeight(output);
-            lastTokenFragment = new BytesRef(token.bytes, token.offset + i + 1, token.length - i - 1);
+            lastTokenFragment = new BytesRef(token.bytes(), i + 1, token.length() - i - 1);
             break;
           }
         }
         
-        final BytesRef finalLastToken;
-        
+        final BytesRefBuilder finalLastToken = new BytesRefBuilder();
         if (lastTokenFragment == null) {
-          finalLastToken = BytesRef.deepCopyOf(token);
+          finalLastToken.copyBytes(token.get());
         } else {
-          finalLastToken = BytesRef.deepCopyOf(lastTokenFragment);
+          finalLastToken.copyBytes(lastTokenFragment);
         }
-        assert finalLastToken.offset == 0;
         
-        CharsRef spare = new CharsRef();
+        CharsRefBuilder spare = new CharsRefBuilder();
         
         // complete top-N
         TopResults<Long> completions = null;
@@ -647,7 +648,7 @@ public class FreeTextSuggester extends L
           // reject up to seen.size() paths in acceptResult():
           Util.TopNSearcher<Long> searcher = new Util.TopNSearcher<Long>(fst, num, num+seen.size(), weightComparator) {
             
-            BytesRef scratchBytes = new BytesRef();
+            BytesRefBuilder scratchBytes = new BytesRefBuilder();
             
             @Override
             protected void addIfCompetitive(Util.FSTPath<Long> path) {
@@ -662,20 +663,20 @@ public class FreeTextSuggester extends L
             @Override
             protected boolean acceptResult(IntsRef input, Long output) {
               Util.toBytesRef(input, scratchBytes);
-              finalLastToken.grow(finalLastToken.length + scratchBytes.length);
-              int lenSav = finalLastToken.length;
+              finalLastToken.grow(finalLastToken.length() + scratchBytes.length());
+              int lenSav = finalLastToken.length();
               finalLastToken.append(scratchBytes);
               //System.out.println("    accept? input='" + scratchBytes.utf8ToString() + "'; lastToken='" + finalLastToken.utf8ToString() + "'; return " + (seen.contains(finalLastToken) == false));
-              boolean ret = seen.contains(finalLastToken) == false;
+              boolean ret = seen.contains(finalLastToken.get()) == false;
               
-              finalLastToken.length = lenSav;
+              finalLastToken.setLength(lenSav);
               return ret;
             }
           };
           
           // since this search is initialized with a single start node 
           // it is okay to start with an empty input path here
-          searcher.addStartPaths(arc, prefixOutput, true, new IntsRef());
+          searcher.addStartPaths(arc, prefixOutput, true, new IntsRefBuilder());
           
           completions = searcher.search();
           assert completions.isComplete;
@@ -683,14 +684,14 @@ public class FreeTextSuggester extends L
           throw new RuntimeException(bogus);
         }
         
-        int prefixLength = token.length;
+        int prefixLength = token.length();
         
-        BytesRef suffix = new BytesRef(8);
+        BytesRefBuilder suffix = new BytesRefBuilder();
         //System.out.println("    " + completions.length + " completions");
         
         nextCompletion:
           for (Result<Long> completion : completions) {
-            token.length = prefixLength;
+            token.setLength(prefixLength);
             // append suffix
             Util.toBytesRef(completion.input, suffix);
             token.append(suffix);
@@ -699,11 +700,11 @@ public class FreeTextSuggester extends L
             
             // Skip this path if a higher-order model already
             // saw/predicted its last token:
-            BytesRef lastToken = token;
-            for(int i=token.length-1;i>=0;i--) {
-              if (token.bytes[token.offset+i] == separator) {
-                assert token.length-i-1 > 0;
-                lastToken = new BytesRef(token.bytes, token.offset+i+1, token.length-i-1);
+            BytesRef lastToken = token.get();
+            for(int i=token.length()-1;i>=0;i--) {
+              if (token.byteAt(i) == separator) {
+                assert token.length()-i-1 > 0;
+                lastToken = new BytesRef(token.bytes(), i+1, token.length()-i-1);
                 break;
               }
             }
@@ -712,8 +713,7 @@ public class FreeTextSuggester extends L
               continue nextCompletion;
             }
             seen.add(BytesRef.deepCopyOf(lastToken));
-            spare.grow(token.length);
-            UnicodeUtil.UTF8toUTF16(token, spare);
+            spare.copyUTF8Bytes(token.get());
             LookupResult result = new LookupResult(spare.toString(), (long) (Long.MAX_VALUE * backoff * ((double) decodeWeight(completion.output)) / contextCount));
             results.add(result);
             assert results.size() == seen.size();

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java Wed Aug 13 09:36:54 2014
@@ -22,8 +22,10 @@ import java.io.IOException;
 
 import org.apache.lucene.search.suggest.InMemorySorter;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.*;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -137,7 +139,7 @@ public class FSTCompletionBuilder {
   /**
    * Scratch buffer for {@link #add(BytesRef, int)}.
    */
-  private final BytesRef scratch = new BytesRef();
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
 
   /**
    * Max tail sharing length.
@@ -206,14 +208,11 @@ public class FSTCompletionBuilder {
           "Bucket outside of the allowed range [0, " + buckets + "): " + bucket);
     }
     
-    if (scratch.bytes.length < utf8.length + 1) {
-      scratch.grow(utf8.length + 10);
-    }
-    
-    scratch.length = 1;
-    scratch.bytes[0] = (byte) bucket;
+    scratch.grow(utf8.length + 10);
+    scratch.clear();
+    scratch.append((byte) bucket);
     scratch.append(utf8);
-    sorter.add(scratch);
+    sorter.add(scratch.get());
   }
 
   /**
@@ -242,14 +241,14 @@ public class FSTCompletionBuilder {
         shareMaxTailLength, outputs, false, 
         PackedInts.DEFAULT, true, 15);
     
-    BytesRef scratch = new BytesRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
     BytesRef entry;
-    final IntsRef scratchIntsRef = new IntsRef();
+    final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
     int count = 0;
     BytesRefIterator iter = sorter.iterator();
     while((entry = iter.next()) != null) {
       count++;
-      if (scratch.compareTo(entry) != 0) {
+      if (scratch.get().compareTo(entry) != 0) {
         builder.add(Util.toIntsRef(entry, scratchIntsRef), empty);
         scratch.copyBytes(entry);
       }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java Wed Aug 13 09:36:54 2014
@@ -34,7 +34,9 @@ import org.apache.lucene.store.DataOutpu
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.OfflineSorter.SortInfo;
@@ -197,10 +199,10 @@ public class FSTCompletionLookup extends
       int previousBucket = 0;
       int previousScore = 0;
       ByteArrayDataInput input = new ByteArrayDataInput();
-      BytesRef tmp1 = new BytesRef();
+      BytesRefBuilder tmp1 = new BytesRefBuilder();
       BytesRef tmp2 = new BytesRef();
       while (reader.read(tmp1)) {
-        input.reset(tmp1.bytes);
+        input.reset(tmp1.bytes());
         int currentScore = input.readInt();
 
         int bucket;
@@ -213,9 +215,9 @@ public class FSTCompletionLookup extends
         previousBucket = bucket;
 
         // Only append the input, discard the weight.
-        tmp2.bytes = tmp1.bytes;
+        tmp2.bytes = tmp1.bytes();
         tmp2.offset = input.getPosition();
-        tmp2.length = tmp1.length - input.getPosition();
+        tmp2.length = tmp1.length() - input.getPosition();
         builder.add(tmp2, bucket);
 
         line++;
@@ -260,10 +262,9 @@ public class FSTCompletionLookup extends
     }
     
     final ArrayList<LookupResult> results = new ArrayList<>(completions.size());
-    CharsRef spare = new CharsRef();
+    CharsRefBuilder spare = new CharsRefBuilder();
     for (Completion c : completions) {
-      spare.grow(c.utf8.length);
-      UnicodeUtil.UTF8toUTF16(c.utf8, spare);
+      spare.copyUTF8Bytes(c.utf8);
       results.add(new LookupResult(spare.toString(), c.bucket));
     }
     return results;

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Wed Aug 13 09:36:54 2014
@@ -33,10 +33,10 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.CharsRef;
-import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
-import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.FST.Arc;
 import org.apache.lucene.util.fst.FST.BytesReader;
@@ -104,21 +104,21 @@ public class WFSTCompletionLookup extend
     count = 0;
     BytesRef scratch = new BytesRef();
     InputIterator iter = new WFSTInputIterator(iterator);
-    IntsRef scratchInts = new IntsRef();
-    BytesRef previous = null;
+    IntsRefBuilder scratchInts = new IntsRefBuilder();
+    BytesRefBuilder previous = null;
     PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
     while ((scratch = iter.next()) != null) {
       long cost = iter.weight();
       
       if (previous == null) {
-        previous = new BytesRef();
-      } else if (scratch.equals(previous)) {
+        previous = new BytesRefBuilder();
+      } else if (scratch.equals(previous.get())) {
         continue; // for duplicate suggestions, the best weight is actually
                   // added
       }
       Util.toIntsRef(scratch, scratchInts);
-      builder.add(scratchInts, cost);
+      builder.add(scratchInts.get(), cost);
       previous.copyBytes(scratch);
       count++;
     }
@@ -158,14 +158,15 @@ public class WFSTCompletionLookup extend
       return Collections.emptyList();
     }
 
-    BytesRef scratch = new BytesRef(key);
-    int prefixLength = scratch.length;
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    scratch.copyChars(key);
+    int prefixLength = scratch.length();
     Arc<Long> arc = new Arc<>();
     
     // match the prefix portion exactly
     Long prefixOutput = null;
     try {
-      prefixOutput = lookupPrefix(scratch, arc);
+      prefixOutput = lookupPrefix(scratch.get(), arc);
     } catch (IOException bogus) { throw new RuntimeException(bogus); }
     
     if (prefixOutput == null) {
@@ -173,10 +174,9 @@ public class WFSTCompletionLookup extend
     }
     
     List<LookupResult> results = new ArrayList<>(num);
-    CharsRef spare = new CharsRef();
+    CharsRefBuilder spare = new CharsRefBuilder();
     if (exactFirst && arc.isFinal()) {
-      spare.grow(scratch.length);
-      UnicodeUtil.UTF8toUTF16(scratch, spare);
+      spare.copyUTF8Bytes(scratch.get());
       results.add(new LookupResult(spare.toString(), decodeWeight(prefixOutput + arc.nextFinalOutput)));
       if (--num == 0) {
         return results; // that was quick
@@ -192,14 +192,13 @@ public class WFSTCompletionLookup extend
       throw new RuntimeException(bogus);
     }
     
-    BytesRef suffix = new BytesRef(8);
+    BytesRefBuilder suffix = new BytesRefBuilder();
     for (Result<Long> completion : completions) {
-      scratch.length = prefixLength;
+      scratch.setLength(prefixLength);
       // append suffix
       Util.toBytesRef(completion.input, suffix);
       scratch.append(suffix);
-      spare.grow(scratch.length);
-      UnicodeUtil.UTF8toUTF16(scratch, spare);
+      spare.copyUTF8Bytes(scratch.get());
       results.add(new LookupResult(spare.toString(), decodeWeight(completion.output)));
     }
     return results;

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java Wed Aug 13 09:36:54 2014
@@ -31,6 +31,7 @@ import org.apache.lucene.store.DataOutpu
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.UnicodeUtil;
 
 /**
@@ -72,15 +73,14 @@ public class JaspellLookup extends Looku
     trie = new JaspellTernarySearchTrie();
     trie.setMatchAlmostDiff(editDistance);
     BytesRef spare;
-    final CharsRef charsSpare = new CharsRef();
+    final CharsRefBuilder charsSpare = new CharsRefBuilder();
 
     while ((spare = tfit.next()) != null) {
       final long weight = tfit.weight();
       if (spare.length == 0) {
         continue;
       }
-      charsSpare.grow(spare.length);
-      UnicodeUtil.UTF8toUTF16(spare.bytes, spare.offset, spare.length, charsSpare);
+      charsSpare.copyUTF8Bytes(spare);
       trie.put(charsSpare.toString(), Long.valueOf(weight));
     }
   }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java Wed Aug 13 09:36:54 2014
@@ -29,6 +29,7 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -69,10 +70,9 @@ public class TSTLookup extends Lookup {
     ArrayList<String> tokens = new ArrayList<>();
     ArrayList<Number> vals = new ArrayList<>();
     BytesRef spare;
-    CharsRef charsSpare = new CharsRef();
+    CharsRefBuilder charsSpare = new CharsRefBuilder();
     while ((spare = tfit.next()) != null) {
-      charsSpare.grow(spare.length);
-      UnicodeUtil.UTF8toUTF16(spare.bytes, spare.offset, spare.length, charsSpare);
+      charsSpare.copyUTF8Bytes(spare);
       tokens.add(charsSpare.toString());
       vals.add(Long.valueOf(tfit.weight()));
     }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/InputArrayIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/InputArrayIterator.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/InputArrayIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/InputArrayIterator.java Wed Aug 13 09:36:54 2014
@@ -23,6 +23,7 @@ import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * A {@link InputIterator} over a sequence of {@link Input}s.
@@ -33,7 +34,7 @@ public final class InputArrayIterator im
   private final boolean hasContexts;
   private boolean first;
   private Input current;
-  private final BytesRef spare = new BytesRef();
+  private final BytesRefBuilder spare = new BytesRefBuilder();
 
   public InputArrayIterator(Iterator<Input> i) {
     this.i = i;
@@ -69,7 +70,7 @@ public final class InputArrayIterator im
         current = i.next();
       }
       spare.copyBytes(current.term);
-      return spare;
+      return spare.get();
     }
     return null;
   }