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 [2/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/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java Wed Aug 13 09:36:54 2014
@@ -26,6 +26,7 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -123,7 +124,7 @@ public class FixedGapTermsIndexWriter ex
     private long lastTermsPointer;
     private long totTermLength;
 
-    private final BytesRef lastTerm = new BytesRef();
+    private final BytesRefBuilder lastTerm = new BytesRefBuilder();
 
     SimpleFieldWriter(FieldInfo fieldInfo, long termsFilePointer) {
       this.fieldInfo = fieldInfo;
@@ -151,7 +152,7 @@ public class FixedGapTermsIndexWriter ex
 
     @Override
     public void add(BytesRef text, TermStats stats, long termsFilePointer) throws IOException {
-      final int indexedTermLength = indexedTermPrefixLength(lastTerm, text);
+      final int indexedTermLength = indexedTermPrefixLength(lastTerm.get(), text);
       //System.out.println("FGW: add text=" + text.utf8ToString() + " " + text + " fp=" + termsFilePointer);
 
       // write only the min prefix that shows the diff

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java Wed Aug 13 09:36:54 2014
@@ -31,6 +31,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
@@ -200,7 +201,7 @@ public class VariableGapTermsIndexReader
 
         if (indexDivisor > 1) {
           // subsample
-          final IntsRef scratchIntsRef = new IntsRef();
+          final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
           final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
           final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
           final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(fst);

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java Wed Aug 13 09:36:54 2014
@@ -29,8 +29,10 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
@@ -231,7 +233,7 @@ public class VariableGapTermsIndexWriter
     FST<Long> fst;
     final long indexStart;
 
-    private final BytesRef lastTerm = new BytesRef();
+    private final BytesRefBuilder lastTerm = new BytesRefBuilder();
     private boolean first = true;
 
     public FSTFieldWriter(FieldInfo fieldInfo, long termsFilePointer) throws IOException {
@@ -261,7 +263,7 @@ public class VariableGapTermsIndexWriter
       }
     }
 
-    private final IntsRef scratchIntsRef = new IntsRef();
+    private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
 
     @Override
     public void add(BytesRef text, TermStats stats, long termsFilePointer) throws IOException {
@@ -271,7 +273,7 @@ public class VariableGapTermsIndexWriter
         return;
       }
       final int lengthSave = text.length;
-      text.length = indexedTermPrefixLength(lastTerm, text);
+      text.length = indexedTermPrefixLength(lastTerm.get(), text);
       try {
         fstBuilder.add(Util.toIntsRef(text, scratchIntsRef), termsFilePointer);
       } finally {

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java Wed Aug 13 09:36:54 2014
@@ -43,9 +43,11 @@ 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.BytesRefFSTEnum;
@@ -325,7 +327,7 @@ public final class OrdsBlockTreeTermsWri
       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);
@@ -403,7 +405,7 @@ public final class OrdsBlockTreeTermsWri
     // TODO: maybe we could add bulk-add method to
     // Builder?  Takes FST and unions it w/ current
     // FST.
-    private void append(Builder<Output> builder, FST<Output> subIndex, long termOrdOffset, IntsRef scratchIntsRef) throws IOException {
+    private void append(Builder<Output> builder, FST<Output> subIndex, long termOrdOffset, IntsRefBuilder scratchIntsRef) throws IOException {
       final BytesRefFSTEnum<Output> subIndexEnum = new BytesRefFSTEnum<>(subIndex);
       BytesRefFSTEnum.InputOutput<Output> indexEnt;
       while ((indexEnt = subIndexEnum.next()) != null) {
@@ -420,7 +422,7 @@ public final class OrdsBlockTreeTermsWri
   }
 
   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;
@@ -436,7 +438,7 @@ public final class OrdsBlockTreeTermsWri
     // 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;
@@ -574,7 +576,7 @@ public final class OrdsBlockTreeTermsWri
       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:
@@ -816,18 +818,18 @@ public final class OrdsBlockTreeTermsWri
 
     /** 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/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnum.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnum.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnum.java Wed Aug 13 09:36:54 2014
@@ -36,7 +36,9 @@ 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.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.Util;
@@ -65,7 +67,7 @@ public final class OrdsSegmentTermsEnum 
   // 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<Output>[] arcs =
@@ -245,9 +247,7 @@ public final class OrdsSegmentTermsEnum 
       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();
 
@@ -283,7 +283,7 @@ public final class OrdsSegmentTermsEnum 
       targetUpto = 0;
           
       OrdsSegmentTermsEnumFrame lastFrame = stack[0];
-      assert validIndexPrefix <= term.length;
+      assert validIndexPrefix <= term.length();
 
       final int targetLimit = Math.min(target.length, validIndexPrefix);
 
@@ -294,7 +294,7 @@ public final class OrdsSegmentTermsEnum 
 
       // 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);
         // }
@@ -319,9 +319,9 @@ public final class OrdsSegmentTermsEnum 
         // 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]) + ")");
           // }
@@ -332,7 +332,7 @@ public final class OrdsSegmentTermsEnum 
         }
 
         if (cmp == 0) {
-          cmp = term.length - target.length;
+          cmp = term.length() - target.length;
         }
         targetUpto = targetUptoMid;
       }
@@ -359,7 +359,7 @@ public final class OrdsSegmentTermsEnum 
         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 true");
@@ -424,8 +424,8 @@ public final class OrdsSegmentTermsEnum 
 
         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));
           // }
@@ -449,7 +449,7 @@ public final class OrdsSegmentTermsEnum 
       } else {
         // Follow this arc
         arc = nextArc;
-        term.bytes[targetUpto] = (byte) targetLabel;
+        term.setByteAt(targetUpto, (byte) targetLabel);
         // Aggregate output as we go:
         assert arc.output != null;
         if (arc.output != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
@@ -477,7 +477,7 @@ public final class OrdsSegmentTermsEnum 
     // 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));
       // }
@@ -507,9 +507,7 @@ public final class OrdsSegmentTermsEnum 
       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();
 
@@ -543,7 +541,7 @@ public final class OrdsSegmentTermsEnum 
       targetUpto = 0;
           
       OrdsSegmentTermsEnumFrame lastFrame = stack[0];
-      assert validIndexPrefix <= term.length;
+      assert validIndexPrefix <= term.length();
 
       final int targetLimit = Math.min(target.length, validIndexPrefix);
 
@@ -554,7 +552,7 @@ public final class OrdsSegmentTermsEnum 
 
       // 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);
         //}
@@ -582,9 +580,9 @@ public final class OrdsSegmentTermsEnum 
         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]) + ")");
           //}
@@ -595,7 +593,7 @@ public final class OrdsSegmentTermsEnum 
         }
 
         if (cmp == 0) {
-          cmp = term.length - target.length;
+          cmp = term.length() - target.length;
         }
         targetUpto = targetUptoMid;
       }
@@ -622,7 +620,7 @@ public final class OrdsSegmentTermsEnum 
         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");
@@ -708,7 +706,7 @@ public final class OrdsSegmentTermsEnum 
         }
       } 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;
@@ -768,7 +766,7 @@ public final class OrdsSegmentTermsEnum 
       while(true) {
         OrdsSegmentTermsEnumFrame 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<<OrdsBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? OrdsBlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? OrdsBlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd() + " termOrd=" + f.termOrd);
         } else {
@@ -776,8 +774,8 @@ public final class OrdsSegmentTermsEnum 
         }
         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");
           }
           Output output = Util.get(fr.index, prefix);
@@ -842,7 +840,7 @@ public final class OrdsSegmentTermsEnum 
       // 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;
     }
 
@@ -855,7 +853,7 @@ public final class OrdsSegmentTermsEnum 
         if (currentFrame.ord == 0) {
           //if (DEBUG) System.out.println("  return null");
           assert setEOF();
-          term.length = 0;
+          term.setLength(0);
           validIndexPrefix = 0;
           currentFrame.rewind();
           termExists = false;
@@ -868,7 +866,7 @@ public final class OrdsSegmentTermsEnum 
         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);
         }
@@ -887,7 +885,7 @@ public final class OrdsSegmentTermsEnum 
       if (currentFrame.next()) {
         // Push to new block:
         //if (DEBUG) System.out.println("  push frame");
-        currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length, prevTermOrd);
+        currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length(), prevTermOrd);
         // 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:
@@ -897,7 +895,7 @@ public final class OrdsSegmentTermsEnum 
       } else {
         //if (DEBUG) System.out.println("  return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord);
         positioned = true;
-        return term;
+        return term.get();
       }
     }
   }
@@ -905,7 +903,7 @@ public final class OrdsSegmentTermsEnum 
   @Override
   public BytesRef term() {
     assert !eof;
-    return term;
+    return term.get();
   }
 
   @Override
@@ -962,7 +960,7 @@ public final class OrdsSegmentTermsEnum 
     //   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;
       BlockTermState blockState = (BlockTermState) otherState;
       currentFrame = staticFrame;
@@ -1002,9 +1000,7 @@ public final class OrdsSegmentTermsEnum 
 
     // First do reverse lookup in the index to find the block that holds this term:
     InputOutput io = getByOutput(targetOrd);
-    if (term.bytes.length < io.input.length) {
-      term.bytes = ArrayUtil.grow(term.bytes, io.input.length);
-    }
+    term.grow(io.input.length);
 
     Util.toBytesRef(io.input, term);
     if (io.input.length == 0) {
@@ -1060,7 +1056,7 @@ public final class OrdsSegmentTermsEnum 
   /** Specialized getByOutput that can understand the ranges (startOrd to endOrd) we use here, not just startOrd. */
   private InputOutput getByOutput(long targetOrd) throws IOException {
 
-    final IntsRef result = new IntsRef();
+    final IntsRefBuilder result = new IntsRefBuilder();
 
     fr.index.getFirstArc(arc);
     Output output = arc.output;
@@ -1092,9 +1088,7 @@ public final class OrdsSegmentTermsEnum 
 
       if (FST.targetHasArcs(arc)) {
         // System.out.println("  targetHasArcs");
-        if (result.ints.length == upto) {
-          result.grow(1+upto);
-        }
+        result.grow(1+upto);
         
         fr.index.readFirstRealTargetArc(arc.target, arc, fstReader);
 
@@ -1134,9 +1128,9 @@ public final class OrdsSegmentTermsEnum 
             // Keep recursing
             arc.arcIdx = mid-1;
           } else {
-            result.length = bestUpto;
+            result.setLength(bestUpto);
             InputOutput io = new InputOutput();
-            io.input = result;
+            io.input = result.get();
             io.output = bestOutput;
             // System.out.println("  ret0=" + io);
             return io;
@@ -1145,7 +1139,7 @@ public final class OrdsSegmentTermsEnum 
           fr.index.readNextRealArc(arc, fstReader);
 
           // Recurse on this arc:
-          result.ints[upto++] = arc.label;
+          result.setIntAt(upto++, arc.label);
           output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
 
         } else {
@@ -1163,12 +1157,12 @@ public final class OrdsSegmentTermsEnum 
             if (targetOrd >= minArcOutput.startOrd && targetOrd <= endOrd) {
               // Recurse on this arc:
               output = minArcOutput;
-              result.ints[upto++] = arc.label;
+              result.setIntAt(upto++, arc.label);
               break;
             } else if (targetOrd < endOrd || arc.isLast()) {
-              result.length = bestUpto;
+              result.setLength(bestUpto);
               InputOutput io = new InputOutput();
-              io.input = result;
+              io.input = result.get();
               assert bestOutput != null;
               io.output = bestOutput;
               // System.out.println("  ret2=" + io);
@@ -1181,9 +1175,9 @@ public final class OrdsSegmentTermsEnum 
           }
         }
       } else {
-        result.length = bestUpto;
+        result.setLength(bestUpto);
         InputOutput io = new InputOutput();
-        io.input = result;
+        io.input = result.get();
         io.output = bestOutput;
         // System.out.println("  ret3=" + io);
         return io;

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnumFrame.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnumFrame.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnumFrame.java Wed Aug 13 09:36:54 2014
@@ -296,11 +296,9 @@ final class OrdsSegmentTermsEnumFrame {
     termOrd++;
     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;
@@ -313,11 +311,9 @@ final class OrdsSegmentTermsEnumFrame {
     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;
@@ -524,7 +520,7 @@ final class OrdsSegmentTermsEnumFrame {
   // 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;
       }
     }
@@ -787,7 +783,7 @@ final class OrdsSegmentTermsEnumFrame {
             ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, termLen, prevTermOrd);
             ste.currentFrame.loadBlock();
             while (ste.currentFrame.next()) {
-              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length, prevTermOrd);
+              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length(), prevTermOrd);
               ste.currentFrame.loadBlock();
             }
           }
@@ -831,10 +827,8 @@ final class OrdsSegmentTermsEnumFrame {
 
   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/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java Wed Aug 13 09:36:54 2014
@@ -49,6 +49,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.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
@@ -285,8 +286,6 @@ public class FSTOrdTermsReader extends F
 
     // Only wraps common operations for PBF interact
     abstract class BaseTermsEnum extends TermsEnum {
-      /* Current term, null when enum ends or unpositioned */
-      BytesRef term;
 
       /* Current term's ord, starts from 0 */
       long ord;
@@ -314,7 +313,6 @@ public class FSTOrdTermsReader extends F
 
       BaseTermsEnum() throws IOException {
         this.state = postingsReader.newTermState();
-        this.term = null;
         this.statsReader.reset(statsBlock);
         this.metaLongsReader.reset(metaLongsBlock);
         this.metaBytesReader.reset(metaBytesBlock);
@@ -407,11 +405,6 @@ public class FSTOrdTermsReader extends F
       }
 
       @Override
-      public BytesRef term() {
-        return term;
-      }
-
-      @Override
       public int docFreq() throws IOException {
         return state.docFreq;
       }
@@ -452,6 +445,8 @@ public class FSTOrdTermsReader extends F
     // Iterates through all terms in this field
     private final class SegmentTermsEnum extends BaseTermsEnum {
       final BytesRefFSTEnum<Long> fstEnum;
+      /* Current term, null when enum ends or unpositioned */
+      BytesRef term;
 
       /* True when current term's metadata is decoded */
       boolean decoded;
@@ -466,6 +461,11 @@ public class FSTOrdTermsReader extends F
       }
 
       @Override
+      public BytesRef term() throws IOException {
+        return term;
+      }
+
+      @Override
       void decodeMetaData() throws IOException {
         if (!decoded && !seekPending) {
           super.decodeMetaData();
@@ -525,6 +525,9 @@ public class FSTOrdTermsReader extends F
 
     // Iterates intersect result with automaton (cannot seek!)
     private final class IntersectTermsEnum extends BaseTermsEnum {
+      /* Current term, null when enum ends or unpositioned */
+      BytesRefBuilder term;
+
       /* True when current term's metadata is decoded */
       boolean decoded;
 
@@ -588,11 +591,16 @@ public class FSTOrdTermsReader extends F
           pending = isAccept(topFrame());
         } else {
           doSeekCeil(startTerm);
-          pending = !startTerm.equals(term) && isValid(topFrame()) && isAccept(topFrame());
+          pending = (term == null || !startTerm.equals(term.get())) && isValid(topFrame()) && isAccept(topFrame());
         }
       }
 
       @Override
+      public BytesRef term() throws IOException {
+        return term == null ? null : term.get();
+      }
+
+      @Override
       void decodeMetaData() throws IOException {
         if (!decoded) {
           super.decodeMetaData();
@@ -619,7 +627,7 @@ public class FSTOrdTermsReader extends F
         if (pending) {
           pending = false;
           decodeStats();
-          return term;
+          return term();
         }
         decoded = false;
       DFS:
@@ -646,7 +654,7 @@ public class FSTOrdTermsReader extends F
           return null;
         }
         decodeStats();
-        return term;
+        return term();
       }
 
       BytesRef doSeekCeil(BytesRef target) throws IOException {
@@ -665,11 +673,11 @@ public class FSTOrdTermsReader extends F
           upto++;
         }
         if (upto == limit) {  // got target
-          return term;
+          return term();
         }
         if (frame != null) {  // got larger term('s prefix)
           pushFrame(frame);
-          return isAccept(frame) ? term : next();
+          return isAccept(frame) ? term() : next();
         }
         while (level > 0) {   // got target's prefix, advance to larger term
           frame = popFrame();
@@ -678,7 +686,7 @@ public class FSTOrdTermsReader extends F
           }
           if (loadNextFrame(topFrame(), frame) != null) {
             pushFrame(frame);
-            return isAccept(frame) ? term : next();
+            return isAccept(frame) ? term() : next();
           }
         }
         return null;
@@ -790,23 +798,20 @@ public class FSTOrdTermsReader extends F
         return stack[level];
       }
 
-      BytesRef grow(int label) {
+      BytesRefBuilder grow(int label) {
         if (term == null) {
-          term = new BytesRef(new byte[16], 0, 0);
+          term = new BytesRefBuilder();
         } else {
-          if (term.length == term.bytes.length) {
-            term.grow(term.length+1);
-          }
-          term.bytes[term.length++] = (byte)label;
+          term.append((byte) label);
         }
         return term;
       }
 
-      BytesRef shrink() {
-        if (term.length == 0) {
+      BytesRefBuilder shrink() {
+        if (term.length() == 0) {
           term = null;
         } else {
-          term.length--;
+          term.setLength(term.length() - 1);
         }
         return term;
       }

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java Wed Aug 13 09:36:54 2014
@@ -34,6 +34,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
@@ -266,7 +267,7 @@ public class FSTOrdTermsWriter extends F
     private final int longsSize;
     private long numTerms;
 
-    private final IntsRef scratchTerm = new IntsRef();
+    private final IntsRefBuilder scratchTerm = new IntsRefBuilder();
     private final RAMOutputStream statsOut = new RAMOutputStream();
     private final RAMOutputStream metaLongsOut = new RAMOutputStream();
     private final RAMOutputStream metaBytesOut = new RAMOutputStream();

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java Wed Aug 13 09:36:54 2014
@@ -47,6 +47,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
@@ -250,8 +251,6 @@ public class FSTTermsReader extends Fiel
 
     // Only wraps common operations for PBF interact
     abstract class BaseTermsEnum extends TermsEnum {
-      /* Current term, null when enum ends or unpositioned */
-      BytesRef term;
 
       /* Current term stats + decoded metadata (customized by PBF) */
       final BlockTermState state;
@@ -266,7 +265,6 @@ public class FSTTermsReader extends Fiel
       BaseTermsEnum() throws IOException {
         this.state = postingsReader.newTermState();
         this.bytesReader = new ByteArrayDataInput();
-        this.term = null;
         // NOTE: metadata will only be initialized in child class
       }
 
@@ -277,11 +275,6 @@ public class FSTTermsReader extends Fiel
       }
 
       @Override
-      public BytesRef term() {
-        return term;
-      }
-
-      @Override
       public int docFreq() throws IOException {
         return state.docFreq;
       }
@@ -320,6 +313,8 @@ public class FSTTermsReader extends Fiel
 
     // Iterates through all terms in this field
     private final class SegmentTermsEnum extends BaseTermsEnum {
+      /* Current term, null when enum ends or unpositioned */
+      BytesRef term;
       final BytesRefFSTEnum<FSTTermOutputs.TermData> fstEnum;
 
       /* True when current term's metadata is decoded */
@@ -341,6 +336,11 @@ public class FSTTermsReader extends Fiel
         return BytesRef.getUTF8SortedAsUnicodeComparator();
       }
 
+      @Override
+      public BytesRef term() throws IOException {
+        return term;
+      }
+
       // Let PBF decode metadata from long[] and byte[]
       @Override
       void decodeMetaData() throws IOException {
@@ -406,6 +406,8 @@ public class FSTTermsReader extends Fiel
 
     // Iterates intersect result with automaton (cannot seek!)
     private final class IntersectTermsEnum extends BaseTermsEnum {
+      /* Current term, null when enum ends or unpositioned */
+      BytesRefBuilder term;
       /* True when current term's metadata is decoded */
       boolean decoded;
 
@@ -476,7 +478,7 @@ public class FSTTermsReader extends Fiel
           pending = isAccept(topFrame());
         } else {
           doSeekCeil(startTerm);
-          pending = !startTerm.equals(term) && isValid(topFrame()) && isAccept(topFrame());
+          pending = (term == null || !startTerm.equals(term.get())) && isValid(topFrame()) && isAccept(topFrame());
         }
       }
 
@@ -486,6 +488,11 @@ public class FSTTermsReader extends Fiel
       }
 
       @Override
+      public BytesRef term() throws IOException {
+        return term == null ? null : term.get();
+      }
+
+      @Override
       void decodeMetaData() throws IOException {
         assert term != null;
         if (!decoded) {
@@ -519,7 +526,7 @@ public class FSTTermsReader extends Fiel
       @Override
       public SeekStatus seekCeil(BytesRef target) throws IOException {
         decoded = false;
-        term = doSeekCeil(target);
+        doSeekCeil(target);
         loadMetaData();
         if (term == null) {
           return SeekStatus.END;
@@ -534,7 +541,7 @@ public class FSTTermsReader extends Fiel
         if (pending) {
           pending = false;
           loadMetaData();
-          return term;
+          return term();
         }
         decoded = false;
       DFS:
@@ -561,7 +568,7 @@ public class FSTTermsReader extends Fiel
           return null;
         }
         loadMetaData();
-        return term;
+        return term();
       }
 
       private BytesRef doSeekCeil(BytesRef target) throws IOException {
@@ -580,11 +587,11 @@ public class FSTTermsReader extends Fiel
           upto++;
         }
         if (upto == limit) {  // got target
-          return term;
+          return term();
         }
         if (frame != null) {  // got larger term('s prefix)
           pushFrame(frame);
-          return isAccept(frame) ? term : next();
+          return isAccept(frame) ? term() : next();
         }
         while (level > 0) {  // got target's prefix, advance to larger term
           frame = popFrame();
@@ -593,7 +600,7 @@ public class FSTTermsReader extends Fiel
           }
           if (loadNextFrame(topFrame(), frame) != null) {
             pushFrame(frame);
-            return isAccept(frame) ? term : next();
+            return isAccept(frame) ? term() : next();
           }
         }
         return null;
@@ -706,23 +713,20 @@ public class FSTTermsReader extends Fiel
         return stack[level];
       }
 
-      BytesRef grow(int label) {
+      BytesRefBuilder grow(int label) {
         if (term == null) {
-          term = new BytesRef(new byte[16], 0, 0);
+          term = new BytesRefBuilder();
         } else {
-          if (term.length == term.bytes.length) {
-            term.grow(term.length+1);
-          }
-          term.bytes[term.length++] = (byte)label;
+          term.append((byte)label);
         }
         return term;
       }
 
-      BytesRef shrink() {
-        if (term.length == 0) {
+      BytesRefBuilder shrink() {
+        if (term.length() == 0) {
           term = null;
         } else {
-          term.length--;
+          term.setLength(term.length() - 1);
         }
         return term;
       }

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java Wed Aug 13 09:36:54 2014
@@ -34,6 +34,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.Util;
@@ -224,7 +225,7 @@ public class FSTTermsWriter extends Fiel
     private final int longsSize;
     private long numTerms;
 
-    private final IntsRef scratchTerm = new IntsRef();
+    private final IntsRefBuilder scratchTerm = new IntsRefBuilder();
     private final RAMOutputStream statsWriter = new RAMOutputStream();
     private final RAMOutputStream metaWriter = new RAMOutputStream();
 

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java Wed Aug 13 09:36:54 2014
@@ -34,6 +34,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.MathUtil;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.FST.INPUT_TYPE;
@@ -359,7 +360,7 @@ class MemoryDocValuesConsumer extends Do
     meta.writeLong(data.getFilePointer());
     PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     Builder<Long> builder = new Builder<>(INPUT_TYPE.BYTE1, outputs);
-    IntsRef scratch = new IntsRef();
+    IntsRefBuilder scratch = new IntsRefBuilder();
     long ord = 0;
     for (BytesRef v : values) {
       builder.add(Util.toIntsRef(v, scratch), ord);

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java Wed Aug 13 09:36:54 2014
@@ -44,9 +44,11 @@ import org.apache.lucene.store.ChecksumI
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 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.LongValues;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -398,11 +400,11 @@ class MemoryDocValuesProducer extends Do
     final BytesReader in = fst.getBytesReader();
     final Arc<Long> firstArc = new Arc<>();
     final Arc<Long> scratchArc = new Arc<>();
-    final IntsRef scratchInts = new IntsRef();
+    final IntsRefBuilder scratchInts = new IntsRefBuilder();
     final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(fst);
     
     return new SortedDocValues() {
-      final BytesRef term = new BytesRef();
+      final BytesRefBuilder term = new BytesRefBuilder();
 
       @Override
       public int getOrd(int docID) {
@@ -415,8 +417,7 @@ class MemoryDocValuesProducer extends Do
           in.setPosition(0);
           fst.getFirstArc(firstArc);
           IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
-          Util.toBytesRef(output, term);
-          return term;
+          return Util.toBytesRef(output, term);
         } catch (IOException bogus) {
           throw new RuntimeException(bogus);
         }
@@ -546,11 +547,11 @@ class MemoryDocValuesProducer extends Do
     final BytesReader in = fst.getBytesReader();
     final Arc<Long> firstArc = new Arc<>();
     final Arc<Long> scratchArc = new Arc<>();
-    final IntsRef scratchInts = new IntsRef();
+    final IntsRefBuilder scratchInts = new IntsRefBuilder();
     final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(fst);
     final ByteArrayDataInput input = new ByteArrayDataInput();
     return new SortedSetDocValues() {
-      final BytesRef term = new BytesRef();
+      final BytesRefBuilder term = new BytesRefBuilder();
       BytesRef ref;
       long currentOrd;
 
@@ -577,8 +578,7 @@ class MemoryDocValuesProducer extends Do
           in.setPosition(0);
           fst.getFirstArc(firstArc);
           IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
-          Util.toBytesRef(output, term);
-          return term;
+          return Util.toBytesRef(output, term);
         } catch (IOException bogus) {
           throw new RuntimeException(bogus);
         }
@@ -712,8 +712,8 @@ class MemoryDocValuesProducer extends Do
     final FST.BytesReader bytesReader;
     final Arc<Long> firstArc = new Arc<>();
     final Arc<Long> scratchArc = new Arc<>();
-    final IntsRef scratchInts = new IntsRef();
-    final BytesRef scratchBytes = new BytesRef();
+    final IntsRefBuilder scratchInts = new IntsRefBuilder();
+    final BytesRefBuilder scratchBytes = new BytesRefBuilder();
     
     FSTTermsEnum(FST<Long> fst) {
       this.fst = fst;
@@ -765,12 +765,8 @@ class MemoryDocValuesProducer extends Do
       bytesReader.setPosition(0);
       fst.getFirstArc(firstArc);
       IntsRef output = Util.getByOutput(fst, ord, bytesReader, firstArc, scratchArc, scratchInts);
-      scratchBytes.bytes = new byte[output.length];
-      scratchBytes.offset = 0;
-      scratchBytes.length = 0;
-      Util.toBytesRef(output, scratchBytes);
       // TODO: we could do this lazily, better to try to push into FSTEnum though?
-      in.seekExact(scratchBytes);
+      in.seekExact(Util.toBytesRef(output, new BytesRefBuilder()));
     }
 
     @Override

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Wed Aug 13 09:36:54 2014
@@ -54,6 +54,7 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
@@ -231,7 +232,7 @@ public final class MemoryPostingsFormat 
     private final BytesRef spare = new BytesRef();
     private byte[] finalBuffer = new byte[128];
 
-    private final IntsRef scratchIntsRef = new IntsRef();
+    private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
 
     @Override
     public void finishTerm(BytesRef text, TermStats stats) throws IOException {

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java Wed Aug 13 09:36:54 2014
@@ -41,6 +41,7 @@ import org.apache.lucene.util.AttributeI
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 
 /** Concrete class that reads the current doc/freq/skip
@@ -405,7 +406,7 @@ public class PulsingPostingsReader exten
     private int posPending;
     private int position;
     private int payloadLength;
-    private BytesRef payload;
+    private BytesRefBuilder payload;
     private int startOffset;
     private int offsetLength;
 
@@ -552,17 +553,16 @@ public class PulsingPostingsReader exten
     public BytesRef getPayload() throws IOException {
       //System.out.println("PR  getPayload payloadLength=" + payloadLength + " this=" + this);
       if (payloadRetrieved) {
-        return payload;
+        return payload.get();
       } else if (storePayloads && payloadLength > 0) {
         payloadRetrieved = true;
         if (payload == null) {
-          payload = new BytesRef(payloadLength);
-        } else {
-          payload.grow(payloadLength);
+          payload = new BytesRefBuilder();
         }
-        postings.readBytes(payload.bytes, 0, payloadLength);
-        payload.length = payloadLength;
-        return payload;
+        payload.grow(payloadLength);
+        postings.readBytes(payload.bytes(), 0, payloadLength);
+        payload.setLength(payloadLength);
+        return payload.get();
       } else {
         return null;
       }

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java Wed Aug 13 09:36:54 2014
@@ -33,6 +33,7 @@ import org.apache.lucene.store.DataOutpu
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 
 // TODO: we now inline based on total TF of the term,
@@ -97,7 +98,7 @@ public final class PulsingPostingsWriter
   private Position currentDoc;                    // first Position entry of current doc
 
   private static final class Position {
-    BytesRef payload;
+    BytesRefBuilder payload;
     int termFreq;                                 // only incremented on first position for a given doc
     int pos;
     int docID;
@@ -240,12 +241,11 @@ public final class PulsingPostingsWriter
       pos.docID = currentDoc.docID;
       if (payload != null && payload.length > 0) {
         if (pos.payload == null) {
-          pos.payload = BytesRef.deepCopyOf(payload);
-        } else {
-          pos.payload.copyBytes(payload);
+          pos.payload = new BytesRefBuilder();
         }
+        pos.payload.copyBytes(payload);
       } else if (pos.payload != null) {
-        pos.payload.length = 0;
+        pos.payload.clear();
       }
     }
   }
@@ -315,7 +315,7 @@ public final class PulsingPostingsWriter
             final int posDelta = pos.pos - lastPos;
             lastPos = pos.pos;
             // if (DEBUG) System.out.println("    write pos=" + pos.pos);
-            final int payloadLength = pos.payload == null ? 0 : pos.payload.length;
+            final int payloadLength = pos.payload == null ? 0 : pos.payload.length();
             if (storePayloads) {
               if (payloadLength != lastPayloadLength) {
                 buffer.writeVInt((posDelta << 1)|1);
@@ -344,7 +344,7 @@ public final class PulsingPostingsWriter
             
             if (payloadLength > 0) {
               assert storePayloads;
-              buffer.writeBytes(pos.payload.bytes, 0, pos.payload.length);
+              buffer.writeBytes(pos.payload.bytes(), 0, pos.payload.length());
             }
           }
         }
@@ -445,7 +445,8 @@ public final class PulsingPostingsWriter
           wrappedPostingsWriter.startDoc(doc.docID, doc.termFreq);
         }
         // if (DEBUG) System.out.println("PW:   wrapped.addPos pos=" + pos.pos);
-        wrappedPostingsWriter.addPosition(pos.pos, pos.payload, pos.startOffset, pos.endOffset);
+        final BytesRef payload = pos.payload == null ? null : pos.payload.get();
+        wrappedPostingsWriter.addPosition(pos.pos, payload, pos.startOffset, pos.endOffset);
       }
       //wrappedPostingsWriter.finishDoc();
     } else {

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java Wed Aug 13 09:36:54 2014
@@ -36,6 +36,7 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -666,7 +667,7 @@ public class SepPostingsReader extends P
       return -1;
     }
 
-    private BytesRef payload;
+    private BytesRefBuilder payload;
 
     @Override
     public BytesRef getPayload() throws IOException {
@@ -675,7 +676,7 @@ public class SepPostingsReader extends P
       }
       
       if (pendingPayloadBytes == 0) {
-        return payload;
+        return payload.get();
       }
 
       assert pendingPayloadBytes >= payloadLength;
@@ -685,16 +686,14 @@ public class SepPostingsReader extends P
       }
 
       if (payload == null) {
-        payload = new BytesRef();
-        payload.bytes = new byte[payloadLength];
-      } else if (payload.bytes.length < payloadLength) {
-        payload.grow(payloadLength);
+        payload = new BytesRefBuilder();
       }
+      payload.grow(payloadLength);
 
-      payloadIn.readBytes(payload.bytes, 0, payloadLength);
-      payload.length = payloadLength;
+      payloadIn.readBytes(payload.bytes(), 0, payloadLength);
+      payload.setLength(payloadLength);
       pendingPayloadBytes = 0;
-      return payload;
+      return payload.get();
     }
     
     @Override

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Wed Aug 13 09:36:54 2014
@@ -55,6 +55,7 @@ import org.apache.lucene.store.ChecksumI
 import org.apache.lucene.store.IndexInput;
 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.StringHelper;
 
@@ -77,7 +78,7 @@ class SimpleTextDocValuesReader extends 
 
   final int maxDoc;
   final IndexInput data;
-  final BytesRef scratch = new BytesRef();
+  final BytesRefBuilder scratch = new BytesRefBuilder();
   final Map<String,OneField> fields = new HashMap<>();
   
   public SimpleTextDocValuesReader(SegmentReadState state, String ext) throws IOException {
@@ -87,10 +88,10 @@ class SimpleTextDocValuesReader extends 
     while(true) {
       readLine();
       //System.out.println("READ field=" + scratch.utf8ToString());
-      if (scratch.equals(END)) {
+      if (scratch.get().equals(END)) {
         break;
       }
-      assert startsWith(FIELD) : scratch.utf8ToString();
+      assert startsWith(FIELD) : scratch.get().utf8ToString();
       String fieldName = stripPrefix(FIELD);
       //System.out.println("  field=" + fieldName);
 
@@ -98,13 +99,13 @@ class SimpleTextDocValuesReader extends 
       fields.put(fieldName, field);
 
       readLine();
-      assert startsWith(TYPE) : scratch.utf8ToString();
+      assert startsWith(TYPE) : scratch.get().utf8ToString();
 
       DocValuesType dvType = DocValuesType.valueOf(stripPrefix(TYPE));
       assert dvType != null;
       if (dvType == DocValuesType.NUMERIC) {
         readLine();
-        assert startsWith(MINVALUE): "got " + scratch.utf8ToString() + " field=" + fieldName + " ext=" + ext;
+        assert startsWith(MINVALUE): "got " + scratch.get().utf8ToString() + " field=" + fieldName + " ext=" + ext;
         field.minValue = Long.parseLong(stripPrefix(MINVALUE));
         readLine();
         assert startsWith(PATTERN);
@@ -155,7 +156,7 @@ class SimpleTextDocValuesReader extends 
     assert field != null: "field=" + fieldInfo.name + " fields=" + fields;
 
     final IndexInput in = data.clone();
-    final BytesRef scratch = new BytesRef();
+    final BytesRefBuilder scratch = new BytesRefBuilder();
     final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
 
     decoder.setParseBigDecimal(true);
@@ -173,7 +174,7 @@ class SimpleTextDocValuesReader extends 
           //System.out.println("parsing delta: " + scratch.utf8ToString());
           BigDecimal bd;
           try {
-            bd = (BigDecimal) decoder.parse(scratch.utf8ToString());
+            bd = (BigDecimal) decoder.parse(scratch.get().utf8ToString());
           } catch (ParseException pe) {
             throw new CorruptIndexException("failed to parse BigDecimal value (resource=" + in + ")", pe);
           }
@@ -189,7 +190,7 @@ class SimpleTextDocValuesReader extends 
   private Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
     final OneField field = fields.get(fieldInfo.name);
     final IndexInput in = data.clone();
-    final BytesRef scratch = new BytesRef();
+    final BytesRefBuilder scratch = new BytesRefBuilder();
     return new Bits() {
       @Override
       public boolean get(int index) {
@@ -197,7 +198,7 @@ class SimpleTextDocValuesReader extends 
           in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*index);
           SimpleTextUtil.readLine(in, scratch); // data
           SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
-          return scratch.bytes[scratch.offset] == (byte) 'T';
+          return scratch.byteAt(0) == (byte) 'T';
         } catch (IOException e) {
           throw new RuntimeException(e);
         }
@@ -219,11 +220,11 @@ class SimpleTextDocValuesReader extends 
     assert field != null;
 
     final IndexInput in = data.clone();
-    final BytesRef scratch = new BytesRef();
+    final BytesRefBuilder scratch = new BytesRefBuilder();
     final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
 
     return new BinaryDocValues() {
-      final BytesRef term = new BytesRef();
+      final BytesRefBuilder term = new BytesRefBuilder();
 
       @Override
       public BytesRef get(int docID) {
@@ -233,18 +234,17 @@ class SimpleTextDocValuesReader extends 
           }
           in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*docID);
           SimpleTextUtil.readLine(in, scratch);
-          assert StringHelper.startsWith(scratch, LENGTH);
+          assert StringHelper.startsWith(scratch.get(), LENGTH);
           int len;
           try {
-            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, StandardCharsets.UTF_8)).intValue();
+            len = decoder.parse(new String(scratch.bytes(), LENGTH.length, scratch.length() - LENGTH.length, StandardCharsets.UTF_8)).intValue();
           } catch (ParseException pe) {
             throw new CorruptIndexException("failed to parse int length (resource=" + in + ")", pe);
           }
           term.grow(len);
-          term.offset = 0;
-          term.length = len;
-          in.readBytes(term.bytes, 0, len);
-          return term;
+          term.setLength(len);
+          in.readBytes(term.bytes(), 0, len);
+          return term.get();
         } catch (IOException ioe) {
           throw new RuntimeException(ioe);
         }
@@ -255,7 +255,7 @@ class SimpleTextDocValuesReader extends 
   private Bits getBinaryDocsWithField(FieldInfo fieldInfo) throws IOException {
     final OneField field = fields.get(fieldInfo.name);
     final IndexInput in = data.clone();
-    final BytesRef scratch = new BytesRef();
+    final BytesRefBuilder scratch = new BytesRefBuilder();
     final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
 
     return new Bits() {
@@ -264,10 +264,10 @@ class SimpleTextDocValuesReader extends 
         try {
           in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*index);
           SimpleTextUtil.readLine(in, scratch);
-          assert StringHelper.startsWith(scratch, LENGTH);
+          assert StringHelper.startsWith(scratch.get(), LENGTH);
           int len;
           try {
-            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, StandardCharsets.UTF_8)).intValue();
+            len = decoder.parse(new String(scratch.bytes(), LENGTH.length, scratch.length() - LENGTH.length, StandardCharsets.UTF_8)).intValue();
           } catch (ParseException pe) {
             throw new CorruptIndexException("failed to parse int length (resource=" + in + ")", pe);
           }
@@ -276,7 +276,7 @@ class SimpleTextDocValuesReader extends 
           in.readBytes(bytes, 0, len);
           SimpleTextUtil.readLine(in, scratch); // newline
           SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
-          return scratch.bytes[scratch.offset] == (byte) 'T';
+          return scratch.byteAt(0) == (byte) 'T';
         } catch (IOException ioe) {
           throw new RuntimeException(ioe);
         }
@@ -298,12 +298,12 @@ class SimpleTextDocValuesReader extends 
     assert field != null;
 
     final IndexInput in = data.clone();
-    final BytesRef scratch = new BytesRef();
+    final BytesRefBuilder scratch = new BytesRefBuilder();
     final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
     final DecimalFormat ordDecoder = new DecimalFormat(field.ordPattern, new DecimalFormatSymbols(Locale.ROOT));
 
     return new SortedDocValues() {
-      final BytesRef term = new BytesRef();
+      final BytesRefBuilder term = new BytesRefBuilder();
 
       @Override
       public int getOrd(int docID) {
@@ -314,7 +314,7 @@ class SimpleTextDocValuesReader extends 
           in.seek(field.dataStartFilePointer + field.numValues * (9 + field.pattern.length() + field.maxLength) + docID * (1 + field.ordPattern.length()));
           SimpleTextUtil.readLine(in, scratch);
           try {
-            return (int) ordDecoder.parse(scratch.utf8ToString()).longValue()-1;
+            return (int) ordDecoder.parse(scratch.get().utf8ToString()).longValue()-1;
           } catch (ParseException pe) {
             throw new CorruptIndexException("failed to parse ord (resource=" + in + ")", pe);
           }
@@ -331,18 +331,17 @@ class SimpleTextDocValuesReader extends 
           }
           in.seek(field.dataStartFilePointer + ord * (9 + field.pattern.length() + field.maxLength));
           SimpleTextUtil.readLine(in, scratch);
-          assert StringHelper.startsWith(scratch, LENGTH): "got " + scratch.utf8ToString() + " in=" + in;
+          assert StringHelper.startsWith(scratch.get(), LENGTH): "got " + scratch.get().utf8ToString() + " in=" + in;
           int len;
           try {
-            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, StandardCharsets.UTF_8)).intValue();
+            len = decoder.parse(new String(scratch.bytes(), LENGTH.length, scratch.length() - LENGTH.length, StandardCharsets.UTF_8)).intValue();
           } catch (ParseException pe) {
             throw new CorruptIndexException("failed to parse int length (resource=" + in + ")", pe);
           }
           term.grow(len);
-          term.offset = 0;
-          term.length = len;
-          in.readBytes(term.bytes, 0, len);
-          return term;
+          term.setLength(len);
+          in.readBytes(term.bytes(), 0, len);
+          return term.get();
         } catch (IOException ioe) {
           throw new RuntimeException(ioe);
         }
@@ -396,13 +395,13 @@ class SimpleTextDocValuesReader extends 
     assert field != null;
 
     final IndexInput in = data.clone();
-    final BytesRef scratch = new BytesRef();
+    final BytesRefBuilder scratch = new BytesRefBuilder();
     final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
     
     return new SortedSetDocValues() {
       String[] currentOrds = new String[0];
       int currentIndex = 0;
-      final BytesRef term = new BytesRef();
+      final BytesRefBuilder term = new BytesRefBuilder();
       
       @Override
       public long nextOrd() {
@@ -421,7 +420,7 @@ class SimpleTextDocValuesReader extends 
         try {
           in.seek(field.dataStartFilePointer + field.numValues * (9 + field.pattern.length() + field.maxLength) + docID * (1 + field.ordPattern.length()));
           SimpleTextUtil.readLine(in, scratch);
-          String ordList = scratch.utf8ToString().trim();
+          String ordList = scratch.get().utf8ToString().trim();
           if (ordList.isEmpty()) {
             currentOrds = new String[0];
           } else {
@@ -441,18 +440,17 @@ class SimpleTextDocValuesReader extends 
           }
           in.seek(field.dataStartFilePointer + ord * (9 + field.pattern.length() + field.maxLength));
           SimpleTextUtil.readLine(in, scratch);
-          assert StringHelper.startsWith(scratch, LENGTH): "got " + scratch.utf8ToString() + " in=" + in;
+          assert StringHelper.startsWith(scratch.get(), LENGTH): "got " + scratch.get().utf8ToString() + " in=" + in;
           int len;
           try {
-            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, StandardCharsets.UTF_8)).intValue();
+            len = decoder.parse(new String(scratch.bytes(), LENGTH.length, scratch.length() - LENGTH.length, StandardCharsets.UTF_8)).intValue();
           } catch (ParseException pe) {
             throw new CorruptIndexException("failed to parse int length (resource=" + in + ")", pe);
           }
           term.grow(len);
-          term.offset = 0;
-          term.length = len;
-          in.readBytes(term.bytes, 0, len);
-          return term;
+          term.setLength(len);
+          in.readBytes(term.bytes(), 0, len);
+          return term.get();
         } catch (IOException ioe) {
           throw new RuntimeException(ioe);
         }
@@ -496,29 +494,29 @@ class SimpleTextDocValuesReader extends 
 
   /** Used only in ctor: */
   private boolean startsWith(BytesRef prefix) {
-    return StringHelper.startsWith(scratch, prefix);
+    return StringHelper.startsWith(scratch.get(), prefix);
   }
 
   /** Used only in ctor: */
   private String stripPrefix(BytesRef prefix) throws IOException {
-    return new String(scratch.bytes, scratch.offset + prefix.length, scratch.length - prefix.length, StandardCharsets.UTF_8);
+    return new String(scratch.bytes(), prefix.length, scratch.length() - prefix.length, StandardCharsets.UTF_8);
   }
 
   @Override
   public long ramBytesUsed() {
-    return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(scratch.bytes)
+    return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(scratch.bytes())
         + fields.size() * (RamUsageEstimator.NUM_BYTES_OBJECT_REF * 2L + OneField.BASE_RAM_BYTES_USED);
   }
 
   @Override
   public void checkIntegrity() throws IOException {
-    BytesRef scratch = new BytesRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
     IndexInput clone = data.clone();
     clone.seek(0);
     ChecksumIndexInput input = new BufferedChecksumIndexInput(clone);
     while(true) {
       SimpleTextUtil.readLine(input, scratch);
-      if (scratch.equals(END)) {
+      if (scratch.get().equals(END)) {
         SimpleTextUtil.checkFooter(input);
         break;
       }

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Wed Aug 13 09:36:54 2014
@@ -33,6 +33,7 @@ import org.apache.lucene.index.SegmentWr
 import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 
 class SimpleTextDocValuesWriter extends DocValuesConsumer {
@@ -50,7 +51,7 @@ class SimpleTextDocValuesWriter extends 
   final static BytesRef ORDPATTERN = new BytesRef("  ordpattern ");
   
   IndexOutput data;
-  final BytesRef scratch = new BytesRef();
+  final BytesRefBuilder scratch = new BytesRefBuilder();
   final int numDocs;
   private final Set<String> fieldsSeen = new HashSet<>(); // for asserting
   
@@ -279,7 +280,7 @@ class SimpleTextDocValuesWriter extends 
       @Override
       public Iterator<BytesRef> iterator() {
         final StringBuilder builder = new StringBuilder();
-        final BytesRef scratch = new BytesRef();
+        final BytesRefBuilder scratch = new BytesRefBuilder();
         final Iterator<Number> counts = docToValueCount.iterator();
         final Iterator<Number> numbers = values.iterator();
         
@@ -301,7 +302,7 @@ class SimpleTextDocValuesWriter extends 
               builder.append(Long.toString(numbers.next().longValue()));
             }
             scratch.copyChars(builder);
-            return scratch;
+            return scratch.get();
           }
 
           @Override

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java Wed Aug 13 09:36:54 2014
@@ -32,7 +32,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 
@@ -50,76 +50,76 @@ public class SimpleTextFieldInfosReader 
   public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
     ChecksumIndexInput input = directory.openChecksumInput(fileName, iocontext);
-    BytesRef scratch = new BytesRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
     
     boolean success = false;
     try {
       
       SimpleTextUtil.readLine(input, scratch);
-      assert StringHelper.startsWith(scratch, NUMFIELDS);
+      assert StringHelper.startsWith(scratch.get(), NUMFIELDS);
       final int size = Integer.parseInt(readString(NUMFIELDS.length, scratch));
       FieldInfo infos[] = new FieldInfo[size];
 
       for (int i = 0; i < size; i++) {
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, NAME);
+        assert StringHelper.startsWith(scratch.get(), NAME);
         String name = readString(NAME.length, scratch);
         
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, NUMBER);
+        assert StringHelper.startsWith(scratch.get(), NUMBER);
         int fieldNumber = Integer.parseInt(readString(NUMBER.length, scratch));
 
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, ISINDEXED);
+        assert StringHelper.startsWith(scratch.get(), ISINDEXED);
         boolean isIndexed = Boolean.parseBoolean(readString(ISINDEXED.length, scratch));
         
         final IndexOptions indexOptions;
         if (isIndexed) {
           SimpleTextUtil.readLine(input, scratch);
-          assert StringHelper.startsWith(scratch, INDEXOPTIONS);
+          assert StringHelper.startsWith(scratch.get(), INDEXOPTIONS);
           indexOptions = IndexOptions.valueOf(readString(INDEXOPTIONS.length, scratch));          
         } else {
           indexOptions = null;
         }
         
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, STORETV);
+        assert StringHelper.startsWith(scratch.get(), STORETV);
         boolean storeTermVector = Boolean.parseBoolean(readString(STORETV.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, PAYLOADS);
+        assert StringHelper.startsWith(scratch.get(), PAYLOADS);
         boolean storePayloads = Boolean.parseBoolean(readString(PAYLOADS.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, NORMS);
+        assert StringHelper.startsWith(scratch.get(), NORMS);
         boolean omitNorms = !Boolean.parseBoolean(readString(NORMS.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, NORMS_TYPE);
+        assert StringHelper.startsWith(scratch.get(), NORMS_TYPE);
         String nrmType = readString(NORMS_TYPE.length, scratch);
         final DocValuesType normsType = docValuesType(nrmType);
         
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, DOCVALUES);
+        assert StringHelper.startsWith(scratch.get(), DOCVALUES);
         String dvType = readString(DOCVALUES.length, scratch);
         final DocValuesType docValuesType = docValuesType(dvType);
         
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, DOCVALUES_GEN);
+        assert StringHelper.startsWith(scratch.get(), DOCVALUES_GEN);
         final long dvGen = Long.parseLong(readString(DOCVALUES_GEN.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, NUM_ATTS);
+        assert StringHelper.startsWith(scratch.get(), NUM_ATTS);
         int numAtts = Integer.parseInt(readString(NUM_ATTS.length, scratch));
         Map<String,String> atts = new HashMap<>();
 
         for (int j = 0; j < numAtts; j++) {
           SimpleTextUtil.readLine(input, scratch);
-          assert StringHelper.startsWith(scratch, ATT_KEY);
+          assert StringHelper.startsWith(scratch.get(), ATT_KEY);
           String key = readString(ATT_KEY.length, scratch);
         
           SimpleTextUtil.readLine(input, scratch);
-          assert StringHelper.startsWith(scratch, ATT_VALUE);
+          assert StringHelper.startsWith(scratch.get(), ATT_VALUE);
           String value = readString(ATT_VALUE.length, scratch);
           atts.put(key, value);
         }
@@ -150,7 +150,7 @@ public class SimpleTextFieldInfosReader 
     }
   }
   
-  private String readString(int offset, BytesRef scratch) {
-    return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, StandardCharsets.UTF_8);
+  private String readString(int offset, BytesRefBuilder scratch) {
+    return new String(scratch.bytes(), offset, scratch.length()-offset, StandardCharsets.UTF_8);
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java Wed Aug 13 09:36:54 2014
@@ -29,6 +29,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -63,7 +64,7 @@ public class SimpleTextFieldInfosWriter 
   public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
     IndexOutput out = directory.createOutput(fileName, context);
-    BytesRef scratch = new BytesRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
     boolean success = false;
     try {
       SimpleTextUtil.write(out, NUMFIELDS);