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/12 16:46:49 UTC

svn commit: r1617493 [4/8] - in /lucene/dev/trunk: lucene/ lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/ lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ lucene/analysis/common/src/java/org/apache/lucene/ana...

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java Tue Aug 12 14:46:45 2014
@@ -20,11 +20,12 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
@@ -58,7 +59,7 @@ class AutomatonTermsEnum extends Filtere
   private final long[] visited;
   private long curGen;
   // the reference used for seeking forwards through the term dictionary
-  private final BytesRef seekBytesRef = new BytesRef(10); 
+  private final BytesRefBuilder seekBytesRef = new BytesRefBuilder(); 
   // true if we are enumerating an infinite portion of the DFA.
   // in this case it is faster to drive the query based on the terms dictionary.
   // when this is true, linearUpperBound indicate the end of range
@@ -108,10 +109,10 @@ class AutomatonTermsEnum extends Filtere
   protected BytesRef nextSeekTerm(final BytesRef term) throws IOException {
     //System.out.println("ATE.nextSeekTerm term=" + term);
     if (term == null) {
-      assert seekBytesRef.length == 0;
+      assert seekBytesRef.length() == 0;
       // return the empty term, as its valid
       if (runAutomaton.isAccept(runAutomaton.getInitialState())) {   
-        return seekBytesRef;
+        return seekBytesRef.get();
       }
     } else {
       seekBytesRef.copyBytes(term);
@@ -119,7 +120,7 @@ class AutomatonTermsEnum extends Filtere
 
     // seek to the next possible string;
     if (nextString()) {
-      return seekBytesRef;  // reposition
+      return seekBytesRef.get();  // reposition
     } else {
       return null;          // no more possible strings can match
     }
@@ -140,15 +141,15 @@ class AutomatonTermsEnum extends Filtere
     int maxInterval = 0xff;
     //System.out.println("setLinear pos=" + position + " seekbytesRef=" + seekBytesRef);
     for (int i = 0; i < position; i++) {
-      state = runAutomaton.step(state, seekBytesRef.bytes[i] & 0xff);
+      state = runAutomaton.step(state, seekBytesRef.byteAt(i) & 0xff);
       assert state >= 0: "state=" + state;
     }
     final int numTransitions = automaton.getNumTransitions(state);
     automaton.initTransition(state, transition);
     for (int i = 0; i < numTransitions; i++) {
       automaton.getNextTransition(transition);
-      if (transition.min <= (seekBytesRef.bytes[position] & 0xff) && 
-          (seekBytesRef.bytes[position] & 0xff) <= transition.max) {
+      if (transition.min <= (seekBytesRef.byteAt(position) & 0xff) && 
+          (seekBytesRef.byteAt(position) & 0xff) <= transition.max) {
         maxInterval = transition.max;
         break;
       }
@@ -159,14 +160,14 @@ class AutomatonTermsEnum extends Filtere
     int length = position + 1; /* position + maxTransition */
     if (linearUpperBound.bytes.length < length)
       linearUpperBound.bytes = new byte[length];
-    System.arraycopy(seekBytesRef.bytes, 0, linearUpperBound.bytes, 0, position);
+    System.arraycopy(seekBytesRef.bytes(), 0, linearUpperBound.bytes, 0, position);
     linearUpperBound.bytes[position] = (byte) maxInterval;
     linearUpperBound.length = length;
     
     linear = true;
   }
 
-  private final IntsRef savedStates = new IntsRef(10);
+  private final IntsRefBuilder savedStates = new IntsRefBuilder();
   
   /**
    * Increments the byte buffer to the next String in binary order after s that will not put
@@ -181,20 +182,19 @@ class AutomatonTermsEnum extends Filtere
   private boolean nextString() {
     int state;
     int pos = 0;
-    savedStates.grow(seekBytesRef.length+1);
-    final int[] states = savedStates.ints;
-    states[0] = runAutomaton.getInitialState();
+    savedStates.grow(seekBytesRef.length()+1);
+    savedStates.setIntAt(0, runAutomaton.getInitialState());
     
     while (true) {
       curGen++;
       linear = false;
       // walk the automaton until a character is rejected.
-      for (state = states[pos]; pos < seekBytesRef.length; pos++) {
+      for (state = savedStates.intAt(pos); pos < seekBytesRef.length(); pos++) {
         visited[state] = curGen;
-        int nextState = runAutomaton.step(state, seekBytesRef.bytes[pos] & 0xff);
+        int nextState = runAutomaton.step(state, seekBytesRef.byteAt(pos) & 0xff);
         if (nextState == -1)
           break;
-        states[pos+1] = nextState;
+        savedStates.setIntAt(pos+1, nextState);
         // we found a loop, record it for faster enumeration
         if (!finite && !linear && visited[nextState] == curGen) {
           setLinear(pos);
@@ -209,7 +209,7 @@ class AutomatonTermsEnum extends Filtere
       } else { /* no more solutions exist from this useful portion, backtrack */
         if ((pos = backtrack(pos)) < 0) /* no more solutions at all */
           return false;
-        final int newState = runAutomaton.step(states[pos], seekBytesRef.bytes[pos] & 0xff);
+        final int newState = runAutomaton.step(savedStates.intAt(pos), seekBytesRef.byteAt(pos) & 0xff);
         if (newState >= 0 && runAutomaton.isAccept(newState))
           /* String is good to go as-is */
           return true;
@@ -245,8 +245,8 @@ class AutomatonTermsEnum extends Filtere
      * character, if it exists.
      */
     int c = 0;
-    if (position < seekBytesRef.length) {
-      c = seekBytesRef.bytes[position] & 0xff;
+    if (position < seekBytesRef.length()) {
+      c = seekBytesRef.byteAt(position) & 0xff;
       // if the next byte is 0xff and is not part of the useful portion,
       // then by definition it puts us in a reject state, and therefore this
       // path is dead. there cannot be any higher transitions. backtrack.
@@ -254,7 +254,7 @@ class AutomatonTermsEnum extends Filtere
         return false;
     }
 
-    seekBytesRef.length = position;
+    seekBytesRef.setLength(position);
     visited[state] = curGen;
 
     final int numTransitions = automaton.getNumTransitions(state);
@@ -266,9 +266,8 @@ class AutomatonTermsEnum extends Filtere
       if (transition.max >= c) {
         int nextChar = Math.max(c, transition.min);
         // append either the next sequential char, or the minimum transition
-        seekBytesRef.grow(seekBytesRef.length + 1);
-        seekBytesRef.length++;
-        seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) nextChar;
+        seekBytesRef.grow(seekBytesRef.length() + 1);
+        seekBytesRef.append((byte) nextChar);
         state = transition.dest;
         /* 
          * as long as is possible, continue down the minimal path in
@@ -286,13 +285,12 @@ class AutomatonTermsEnum extends Filtere
           state = transition.dest;
           
           // append the minimum transition
-          seekBytesRef.grow(seekBytesRef.length + 1);
-          seekBytesRef.length++;
-          seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.min;
+          seekBytesRef.grow(seekBytesRef.length() + 1);
+          seekBytesRef.append((byte) transition.min);
           
           // we found a loop, record it for faster enumeration
           if (!finite && !linear && visited[state] == curGen) {
-            setLinear(seekBytesRef.length-1);
+            setLinear(seekBytesRef.length()-1);
           }
         }
         return true;
@@ -311,12 +309,12 @@ class AutomatonTermsEnum extends Filtere
    */
   private int backtrack(int position) {
     while (position-- > 0) {
-      int nextChar = seekBytesRef.bytes[position] & 0xff;
+      int nextChar = seekBytesRef.byteAt(position) & 0xff;
       // if a character is 0xff its a dead-end too,
       // because there is no higher character in binary sort order.
       if (nextChar++ != 0xff) {
-        seekBytesRef.bytes[position] = (byte) nextChar;
-        seekBytesRef.length = position+1;
+        seekBytesRef.setByteAt(position, (byte) nextChar);
+        seekBytesRef.setLength(position+1);
         return position;
       }
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java Tue Aug 12 14:46:45 2014
@@ -4,6 +4,7 @@ import org.apache.lucene.document.Binary
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PagedGrowableWriter;
@@ -96,7 +97,7 @@ class BinaryDocValuesFieldUpdates extend
 
   private PagedMutable docs;
   private PagedGrowableWriter offsets, lengths;
-  private BytesRef values;
+  private BytesRefBuilder values;
   private int size;
   private final int bitsPerValue;
   
@@ -106,7 +107,7 @@ class BinaryDocValuesFieldUpdates extend
     docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
     offsets = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
     lengths = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
-    values = new BytesRef(16); // start small
+    values = new BytesRefBuilder();
     size = 0;
   }
   
@@ -127,7 +128,7 @@ class BinaryDocValuesFieldUpdates extend
     }
     
     docs.set(size, doc);
-    offsets.set(size, values.length);
+    offsets.set(size, values.length());
     lengths.set(size, val.length);
     values.append(val);
     ++size;
@@ -138,7 +139,7 @@ class BinaryDocValuesFieldUpdates extend
     final PagedMutable docs = this.docs;
     final PagedGrowableWriter offsets = this.offsets;
     final PagedGrowableWriter lengths = this.lengths;
-    final BytesRef values = this.values;
+    final BytesRef values = this.values.get();
     new InPlaceMergeSorter() {
       @Override
       protected void swap(int i, int j) {
@@ -181,16 +182,12 @@ class BinaryDocValuesFieldUpdates extend
     for (int i = 0; i < otherUpdates.size; i++) {
       int doc = (int) otherUpdates.docs.get(i);
       docs.set(size, doc);
-      offsets.set(size, values.length + otherUpdates.offsets.get(i)); // correct relative offset
+      offsets.set(size, values.length() + otherUpdates.offsets.get(i)); // correct relative offset
       lengths.set(size, otherUpdates.lengths.get(i));
       ++size;
     }
-    int newLen = values.length + otherUpdates.values.length;
-    if (values.bytes.length < newLen) {
-      values.bytes = ArrayUtil.grow(values.bytes, newLen);
-    }
-    System.arraycopy(otherUpdates.values.bytes, otherUpdates.values.offset, values.bytes, values.length, otherUpdates.values.length);
-    values.length = newLen;
+
+    values.append(otherUpdates.values);
   }
 
   @Override
@@ -204,7 +201,7 @@ class BinaryDocValuesFieldUpdates extend
     final int capacity = estimateCapacity(size);
     bytesPerDoc += (long) Math.ceil((double) offsets.ramBytesUsed() / capacity); // offsets
     bytesPerDoc += (long) Math.ceil((double) lengths.ramBytesUsed() / capacity); // lengths
-    bytesPerDoc += (long) Math.ceil((double) values.length / size); // values
+    bytesPerDoc += (long) Math.ceil((double) values.length() / size); // values
     return bytesPerDoc;
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Tue Aug 12 14:46:45 2014
@@ -26,6 +26,7 @@ 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.Counter;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.PagedBytes;
@@ -124,7 +125,7 @@ class BinaryDocValuesWriter extends DocV
 
   // iterates over the values we have in ram
   private class BytesIterator implements Iterator<BytesRef> {
-    final BytesRef value = new BytesRef();
+    final BytesRefBuilder value = new BytesRefBuilder();
     final PackedLongValues.Iterator lengthsIterator;
     final DataInput bytesIterator = bytes.getDataInput();
     final int size = (int) lengths.size();
@@ -150,15 +151,15 @@ class BinaryDocValuesWriter extends DocV
       if (upto < size) {
         int length = (int) lengthsIterator.next();
         value.grow(length);
-        value.length = length;
+        value.setLength(length);
         try {
-          bytesIterator.readBytes(value.bytes, value.offset, value.length);
+          bytesIterator.readBytes(value.bytes(), 0, value.length());
         } catch (IOException ioe) {
           // Should never happen!
           throw new RuntimeException(ioe);
         }
         if (docsWithField.get(upto)) {
-          v = value;
+          v = value.get();
         } else {
           v = null;
         }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Tue Aug 12 14:46:45 2014
@@ -41,6 +41,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.CommandLineUtil;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
@@ -852,7 +853,7 @@ public class CheckIndex {
       boolean hasOrd = true;
       final long termCountStart = status.delTermCount + status.termCount;
       
-      BytesRef lastTerm = null;
+      BytesRefBuilder lastTerm = null;
       
       long sumTotalTermFreq = 0;
       long sumDocFreq = 0;
@@ -870,9 +871,10 @@ public class CheckIndex {
         // make sure terms arrive in order according to
         // the comp
         if (lastTerm == null) {
-          lastTerm = BytesRef.deepCopyOf(term);
+          lastTerm = new BytesRefBuilder();
+          lastTerm.copyBytes(term);
         } else {
-          if (lastTerm.compareTo(term) >= 0) {
+          if (lastTerm.get().compareTo(term) >= 0) {
             throw new RuntimeException("terms out of order: lastTerm=" + lastTerm + " term=" + term);
           }
           lastTerm.copyBytes(term);
@@ -1184,7 +1186,7 @@ public class CheckIndex {
         
         // Test seek to last term:
         if (lastTerm != null) {
-          if (termsEnum.seekCeil(lastTerm) != TermsEnum.SeekStatus.FOUND) { 
+          if (termsEnum.seekCeil(lastTerm.get()) != TermsEnum.SeekStatus.FOUND) { 
             throw new RuntimeException("seek to last term " + lastTerm + " failed");
           }
           

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java Tue Aug 12 14:46:45 2014
@@ -28,6 +28,7 @@ import org.apache.lucene.index.FreqProxT
 import org.apache.lucene.util.AttributeSource; // javadocs
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /** Implements limited (iterators only, no stats) {@link
  *  Fields} interface over the in-RAM buffered
@@ -405,7 +406,7 @@ class FreqProxFields extends Fields {
     int termID;
     boolean ended;
     boolean hasPayload;
-    BytesRef payload = new BytesRef();
+    BytesRefBuilder payload = new BytesRefBuilder();
 
     public FreqProxDocsAndPositionsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray) {
       this.terms = terms;
@@ -485,11 +486,9 @@ class FreqProxFields extends Fields {
       if ((code & 1) != 0) {
         hasPayload = true;
         // has a payload
-        payload.length = posReader.readVInt();
-        if (payload.bytes.length < payload.length) {
-          payload.grow(payload.length);
-        }
-        posReader.readBytes(payload.bytes, 0, payload.length);
+        payload.setLength(posReader.readVInt());
+        payload.grow(payload.length());
+        posReader.readBytes(payload.bytes(), 0, payload.length());
       } else {
         hasPayload = false;
       }
@@ -521,7 +520,7 @@ class FreqProxFields extends Fields {
     @Override
     public BytesRef getPayload() {
       if (hasPayload) {
-        return payload;
+        return payload.get();
       } else {
         return null;
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Tue Aug 12 14:46:45 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Bits;
@@ -41,7 +42,7 @@ public final class MultiTermsEnum extend
 
   private BytesRef lastSeek;
   private boolean lastSeekExact;
-  private final BytesRef lastSeekScratch = new BytesRef();
+  private final BytesRefBuilder lastSeekScratch = new BytesRefBuilder();
 
   private int numTop;
   private int numSubs;
@@ -186,7 +187,7 @@ public final class MultiTermsEnum extend
     }
 
     lastSeekScratch.copyBytes(term);
-    lastSeek = lastSeekScratch;
+    lastSeek = lastSeekScratch.get();
 
     for(int i=0;i<numSubs;i++) {
       final SeekStatus status;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java Tue Aug 12 14:46:45 2014
@@ -26,6 +26,7 @@ import org.apache.lucene.store.RAMInputS
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * Prefix codes term instances (prefixes are shared)
@@ -52,8 +53,8 @@ class PrefixCodedTerms implements Iterab
   class PrefixCodedTermsIterator implements Iterator<Term> {
     final IndexInput input;
     String field = "";
-    BytesRef bytes = new BytesRef();
-    Term term = new Term(field, bytes);
+    BytesRefBuilder bytes = new BytesRefBuilder();
+    Term term = new Term(field, bytes.get());
 
     PrefixCodedTermsIterator() {
       try {
@@ -80,9 +81,9 @@ class PrefixCodedTerms implements Iterab
         int prefix = code >>> 1;
         int suffix = input.readVInt();
         bytes.grow(prefix + suffix);
-        input.readBytes(bytes.bytes, prefix, suffix);
-        bytes.length = prefix + suffix;
-        term.set(field, bytes);
+        input.readBytes(bytes.bytes(), prefix, suffix);
+        bytes.setLength(prefix + suffix);
+        term.set(field, bytes.get());
         return term;
       } catch (IOException e) {
         throw new RuntimeException(e);
@@ -100,6 +101,7 @@ class PrefixCodedTerms implements Iterab
     private RAMFile buffer = new RAMFile();
     private RAMOutputStream output = new RAMOutputStream(buffer, false);
     private Term lastTerm = new Term("");
+    private BytesRefBuilder lastTermBytes = new BytesRefBuilder();
 
     /** add a term */
     public void add(Term term) {
@@ -116,7 +118,8 @@ class PrefixCodedTerms implements Iterab
         }
         output.writeVInt(suffix);
         output.writeBytes(term.bytes.bytes, term.bytes.offset + prefix, suffix);
-        lastTerm.bytes.copyBytes(term.bytes);
+        lastTermBytes.copyBytes(term.bytes);
+        lastTerm.bytes = lastTermBytes.get();
         lastTerm.field = term.field;
       } catch (IOException e) {
         throw new RuntimeException(e);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java Tue Aug 12 14:46:45 2014
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /** Implements a {@link TermsEnum} wrapping a provided
  * {@link SortedDocValues}. */
@@ -29,12 +30,12 @@ class SortedDocValuesTermsEnum extends T
   private final SortedDocValues values;
   private int currentOrd = -1;
   private BytesRef term;
-  private final BytesRef scratch;
+  private final BytesRefBuilder scratch;
 
   /** Creates a new TermsEnum over the provided values */
   public SortedDocValuesTermsEnum(SortedDocValues values) {
     this.values = values;
-    scratch = new BytesRef();
+    scratch = new BytesRefBuilder();
   }
 
   @Override
@@ -43,7 +44,7 @@ class SortedDocValuesTermsEnum extends T
     if (ord >= 0) {
       currentOrd = ord;
       scratch.copyBytes(text);
-      term = scratch;
+      term = scratch.get();
       return SeekStatus.FOUND;
     } else {
       currentOrd = -ord-1;
@@ -63,7 +64,7 @@ class SortedDocValuesTermsEnum extends T
     if (ord >= 0) {
       currentOrd = ord;
       scratch.copyBytes(text);
-      term = scratch;
+      term = scratch.get();
       return true;
     } else {
       return false;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java Tue Aug 12 14:46:45 2014
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /** Implements a {@link TermsEnum} wrapping a provided
  * {@link SortedSetDocValues}. */
@@ -29,12 +30,12 @@ class SortedSetDocValuesTermsEnum extend
   private final SortedSetDocValues values;
   private long currentOrd = -1;
   private BytesRef term;
-  private final BytesRef scratch;
+  private final BytesRefBuilder scratch;
 
   /** Creates a new TermsEnum over the provided values */
   public SortedSetDocValuesTermsEnum(SortedSetDocValues values) {
     this.values = values;
-    scratch = new BytesRef();
+    scratch = new BytesRefBuilder();
   }
 
   @Override
@@ -43,7 +44,7 @@ class SortedSetDocValuesTermsEnum extend
     if (ord >= 0) {
       currentOrd = ord;
       scratch.copyBytes(text);
-      term = scratch;
+      term = scratch.get();
       return SeekStatus.FOUND;
     } else {
       currentOrd = -ord-1;
@@ -63,7 +64,7 @@ class SortedSetDocValuesTermsEnum extend
     if (ord >= 0) {
       currentOrd = ord;
       scratch.copyBytes(text);
-      term = scratch;
+      term = scratch.get();
       return true;
     } else {
       return false;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java Tue Aug 12 14:46:45 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
@@ -156,9 +157,8 @@ public abstract class Terms {
       return v;
     }
 
-    BytesRef scratch = new BytesRef(1);
-
-    scratch.length = 1;
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    scratch.append((byte) 0);
 
     // Iterates over digits:
     while (true) {
@@ -170,12 +170,12 @@ public abstract class Terms {
       // digit before END:
       while (low != high) {
         int mid = (low+high) >>> 1;
-        scratch.bytes[scratch.length-1] = (byte) mid;
-        if (iterator.seekCeil(scratch) == TermsEnum.SeekStatus.END) {
+        scratch.setByteAt(scratch.length()-1, (byte) mid);
+        if (iterator.seekCeil(scratch.get()) == TermsEnum.SeekStatus.END) {
           // Scratch was too high
           if (mid == 0) {
-            scratch.length--;
-            return scratch;
+            scratch.setLength(scratch.length() - 1);
+            return scratch.get();
           }
           high = mid;
         } else {
@@ -189,8 +189,8 @@ public abstract class Terms {
       }
 
       // Recurse to next digit:
-      scratch.length++;
-      scratch.grow(scratch.length);
+      scratch.setLength(scratch.length() + 1);
+      scratch.grow(scratch.length());
     }
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Tue Aug 12 14:46:45 2014
@@ -27,6 +27,7 @@ import org.apache.lucene.index.NumericDo
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * Expert: a FieldComparator compares hits so as to determine their
@@ -687,7 +688,7 @@ public abstract class FieldComparator<T>
     /* Values for each slot.
        @lucene.internal */
     final BytesRef[] values;
-    private final BytesRef[] tempBRs;
+    private final BytesRefBuilder[] tempBRs;
 
     /* Which reader last copied a value into the slot. When
        we compare two slots, we just compare-by-ord if the
@@ -748,7 +749,7 @@ public abstract class FieldComparator<T>
     public TermOrdValComparator(int numHits, String field, boolean sortMissingLast) {
       ords = new int[numHits];
       values = new BytesRef[numHits];
-      tempBRs = new BytesRef[numHits];
+      tempBRs = new BytesRefBuilder[numHits];
       readerGen = new int[numHits];
       this.field = field;
       if (sortMissingLast) {
@@ -808,10 +809,10 @@ public abstract class FieldComparator<T>
       } else {
         assert ord >= 0;
         if (tempBRs[slot] == null) {
-          tempBRs[slot] = new BytesRef();
+          tempBRs[slot] = new BytesRefBuilder();
         }
-        values[slot] = tempBRs[slot];
-        values[slot].copyBytes(termsIndex.lookupOrd(ord));
+        tempBRs[slot].copyBytes(termsIndex.lookupOrd(ord));
+        values[slot] = tempBRs[slot].get();
       }
       ords[slot] = ord;
       readerGen[slot] = currentReaderGen;
@@ -938,7 +939,7 @@ public abstract class FieldComparator<T>
   public static class TermValComparator extends FieldComparator<BytesRef> {
     
     private final BytesRef[] values;
-    private final BytesRef[] tempBRs;
+    private final BytesRefBuilder[] tempBRs;
     private BinaryDocValues docTerms;
     private Bits docsWithField;
     private final String field;
@@ -949,7 +950,7 @@ public abstract class FieldComparator<T>
     /** Sole constructor. */
     public TermValComparator(int numHits, String field, boolean sortMissingLast) {
       values = new BytesRef[numHits];
-      tempBRs = new BytesRef[numHits];
+      tempBRs = new BytesRefBuilder[numHits];
       this.field = field;
       missingSortCmp = sortMissingLast ? 1 : -1;
     }
@@ -974,10 +975,10 @@ public abstract class FieldComparator<T>
         values[slot] = null;
       } else {
         if (tempBRs[slot] == null) {
-          tempBRs[slot] = new BytesRef();
+          tempBRs[slot] = new BytesRefBuilder();
         }
-        values[slot] = tempBRs[slot];
-        values[slot].copyBytes(comparableBytes);
+        tempBRs[slot].copyBytes(comparableBytes);
+        values[slot] = tempBRs[slot].get();
       }
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Tue Aug 12 14:46:45 2014
@@ -34,6 +34,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.UnicodeUtil;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
@@ -152,7 +153,7 @@ public class FuzzyTermsEnum extends Term
     if (editDistance < runAutomata.size()) {
       //System.out.println("FuzzyTE.getAEnum: ed=" + editDistance + " lastTerm=" + (lastTerm==null ? "null" : lastTerm.utf8ToString()));
       final CompiledAutomaton compiled = runAutomata.get(editDistance);
-      return new AutomatonFuzzyTermsEnum(terms.intersect(compiled, lastTerm == null ? null : compiled.floor(lastTerm, new BytesRef())),
+      return new AutomatonFuzzyTermsEnum(terms.intersect(compiled, lastTerm == null ? null : compiled.floor(lastTerm, new BytesRefBuilder())),
                                          runAutomata.subList(0, editDistance + 1).toArray(new CompiledAutomaton[editDistance + 1]));
     } else {
       return null;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java Tue Aug 12 14:46:45 2014
@@ -30,6 +30,7 @@ import org.apache.lucene.index.TermState
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * Base rewrite method for collecting only the top terms
@@ -86,14 +87,15 @@ public abstract class TopTermsRewrite<Q 
       }
     
       // for assert:
-      private BytesRef lastTerm;
+      private BytesRefBuilder lastTerm;
       private boolean compareToLastTerm(BytesRef t) {
         if (lastTerm == null && t != null) {
-          lastTerm = BytesRef.deepCopyOf(t);
+          lastTerm = new BytesRefBuilder();
+          lastTerm.append(t);
         } else if (t == null) {
           lastTerm = null;
         } else {
-          assert lastTerm.compareTo(t) < 0: "lastTerm=" + lastTerm + " t=" + t;
+          assert lastTerm.get().compareTo(t) < 0: "lastTerm=" + lastTerm + " t=" + t;
           lastTerm.copyBytes(t);
         }
         return true;
@@ -113,7 +115,7 @@ public abstract class TopTermsRewrite<Q 
           final ScoreTerm t = stQueue.peek();
           if (boost < t.boost)
             return true;
-          if (boost == t.boost && bytes.compareTo(t.bytes) > 0)
+          if (boost == t.boost && bytes.compareTo(t.bytes.get()) > 0)
             return true;
         }
         ScoreTerm t = visitedTerms.get(bytes);
@@ -127,14 +129,14 @@ public abstract class TopTermsRewrite<Q 
           // add new entry in PQ, we must clone the term, else it may get overwritten!
           st.bytes.copyBytes(bytes);
           st.boost = boost;
-          visitedTerms.put(st.bytes, st);
+          visitedTerms.put(st.bytes.get(), st);
           assert st.termState.docFreq() == 0;
           st.termState.register(state, readerContext.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
           stQueue.offer(st);
           // possibly drop entries from queue
           if (stQueue.size() > maxSize) {
             st = stQueue.poll();
-            visitedTerms.remove(st.bytes);
+            visitedTerms.remove(st.bytes.get());
             st.termState.clear(); // reset the termstate! 
           } else {
             st = new ScoreTerm(new TermContext(topReaderContext));
@@ -144,7 +146,7 @@ public abstract class TopTermsRewrite<Q 
           if (stQueue.size() == maxSize) {
             t = stQueue.peek();
             maxBoostAtt.setMaxNonCompetitiveBoost(t.boost);
-            maxBoostAtt.setCompetitiveTerm(t.bytes);
+            maxBoostAtt.setCompetitiveTerm(t.bytes.get());
           }
         }
        
@@ -157,7 +159,7 @@ public abstract class TopTermsRewrite<Q 
     ArrayUtil.timSort(scoreTerms, scoreTermSortByTermComp);
     
     for (final ScoreTerm st : scoreTerms) {
-      final Term term = new Term(query.field, st.bytes);
+      final Term term = new Term(query.field, st.bytes.toBytesRef());
       assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq() + " term=" + term;
       addClause(q, term, st.termState.docFreq(), query.getBoost() * st.boost, st.termState); // add to query
     }
@@ -183,12 +185,12 @@ public abstract class TopTermsRewrite<Q 
     new Comparator<ScoreTerm>() {
       @Override
       public int compare(ScoreTerm st1, ScoreTerm st2) {
-        return st1.bytes.compareTo(st2.bytes);
+        return st1.bytes.get().compareTo(st2.bytes.get());
       }
     };
 
   static final class ScoreTerm implements Comparable<ScoreTerm> {
-    public final BytesRef bytes = new BytesRef();
+    public final BytesRefBuilder bytes = new BytesRefBuilder();
     public float boost;
     public final TermContext termState;
     public ScoreTerm(TermContext termState) {
@@ -198,7 +200,7 @@ public abstract class TopTermsRewrite<Q 
     @Override
     public int compareTo(ScoreTerm other) {
       if (this.boost == other.boost)
-        return other.bytes.compareTo(this.bytes);
+        return other.bytes.get().compareTo(this.bytes.get());
       else
         return Float.compare(this.boost, other.boost);
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java Tue Aug 12 14:46:45 2014
@@ -257,10 +257,9 @@ public abstract class DataOutput {
    * @see DataInput#readString()
    */
   public void writeString(String s) throws IOException {
-    final BytesRef utf8Result = new BytesRef(10);
-    UnicodeUtil.UTF16toUTF8(s, 0, s.length(), utf8Result);
+    final BytesRef utf8Result = new BytesRef(s);
     writeVInt(utf8Result.length);
-    writeBytes(utf8Result.bytes, 0, utf8Result.length);
+    writeBytes(utf8Result.bytes, utf8Result.offset, utf8Result.length);
   }
 
   private static int COPY_BUFFER_SIZE = 16384;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java?rev=1617493&r1=1617492&r2=1617493&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java Tue Aug 12 14:46:45 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.util.Arrays;
 import java.util.Comparator;
 
 /** Represents byte[], as a slice (offset + length) into an
@@ -80,19 +81,8 @@ public final class BytesRef implements C
    * unicode text, with no unpaired surrogates.
    */
   public BytesRef(CharSequence text) {
-    this();
-    copyChars(text);
-  }
-
-  /**
-   * Copies the UTF8 bytes for this string.
-   * 
-   * @param text Must be well-formed unicode text, with no
-   * unpaired surrogates or invalid UTF16 code units.
-   */
-  public void copyChars(CharSequence text) {
-    assert offset == 0;   // TODO broken if offset != 0
-    UnicodeUtil.UTF16toUTF8(text, 0, text.length(), this);
+    this(new byte[UnicodeUtil.MAX_UTF8_BYTES_PER_CHAR * text.length()]);
+    length = UnicodeUtil.UTF16toUTF8(text, 0, text.length(), bytes);
   }
   
   /**
@@ -155,9 +145,9 @@ public final class BytesRef implements C
   /** Interprets stored bytes as UTF8 bytes, returning the
    *  resulting string */
   public String utf8ToString() {
-    final CharsRef ref = new CharsRef(length);
-    UnicodeUtil.UTF8toUTF16(bytes, offset, length, ref);
-    return ref.toString(); 
+    final char[] ref = new char[length];
+    final int len = UnicodeUtil.UTF8toUTF16(bytes, offset, length, ref);
+    return new String(ref, 0, len);
   }
 
   /** Returns hex encoded bytes, eg [0x6c 0x75 0x63 0x65 0x6e 0x65] */
@@ -176,49 +166,6 @@ public final class BytesRef implements C
     return sb.toString();
   }
 
-  /**
-   * Copies the bytes from the given {@link BytesRef}
-   * <p>
-   * NOTE: if this would exceed the array size, this method creates a 
-   * new reference array.
-   */
-  public void copyBytes(BytesRef other) {
-    if (bytes.length - offset < other.length) {
-      bytes = new byte[other.length];
-      offset = 0;
-    }
-    System.arraycopy(other.bytes, other.offset, bytes, offset, other.length);
-    length = other.length;
-  }
-
-  /**
-   * Appends the bytes from the given {@link BytesRef}
-   * <p>
-   * NOTE: if this would exceed the array size, this method creates a 
-   * new reference array.
-   */
-  public void append(BytesRef other) {
-    int newLen = length + other.length;
-    if (bytes.length - offset < newLen) {
-      byte[] newBytes = new byte[newLen];
-      System.arraycopy(bytes, offset, newBytes, 0, length);
-      offset = 0;
-      bytes = newBytes;
-    }
-    System.arraycopy(other.bytes, other.offset, bytes, length+offset, other.length);
-    length = newLen;
-  }
-
-  /** 
-   * Used to grow the reference array. 
-   * 
-   * In general this should not be used as it does not take the offset into account.
-   * @lucene.internal */
-  public void grow(int newLength) {
-    assert offset == 0; // NOTE: senseless if offset != 0
-    bytes = ArrayUtil.grow(bytes, newLength);
-  }
-
   /** Unsigned byte order comparison */
   @Override
   public int compareTo(BytesRef other) {
@@ -331,7 +278,9 @@ public final class BytesRef implements C
    */
   public static BytesRef deepCopyOf(BytesRef other) {
     BytesRef copy = new BytesRef();
-    copy.copyBytes(other);
+    copy.bytes = Arrays.copyOfRange(other.bytes, other.offset, other.offset + other.length);
+    copy.offset = 0;
+    copy.length = other.length;
     return copy;
   }
   

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

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefBuilder.java?rev=1617493&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefBuilder.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefBuilder.java Tue Aug 12 14:46:45 2014
@@ -0,0 +1,185 @@
+package org.apache.lucene.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Arrays;
+
+/**
+ * A builder for {@link BytesRef} instances.
+ * @lucene.internal
+ */
+public class BytesRefBuilder {
+
+  private final BytesRef ref;
+
+  /** Sole constructor. */
+  public BytesRefBuilder() {
+    ref = new BytesRef();
+  }
+
+  /** Return a reference to the bytes of this builder. */
+  public byte[] bytes() {
+    return ref.bytes;
+  }
+
+  /** Return the number of bytes in this buffer. */
+  public int length() {
+    return ref.length;
+  }
+
+  /** Set the length. */
+  public void setLength(int length) {
+    this.ref.length = length;
+  }
+  
+  /** Return the byte at the given offset. */
+  public byte byteAt(int offset) {
+    return ref.bytes[offset];
+  }
+
+  /** Set a byte. */
+  public void setByteAt(int offset, byte b) {
+    ref.bytes[offset] = b;
+  }
+
+  /**
+   * Ensure that this builder can hold at least <code>capacity</code> bytes
+   * without resizing.
+   */
+  public void grow(int capacity) {
+    ref.bytes = ArrayUtil.grow(ref.bytes, capacity);
+  }
+
+  /**
+   * Append a single byte to this builder.
+   */
+  public void append(byte b) {
+    grow(ref.length + 1);
+    ref.bytes[ref.length++] = b;
+  }
+
+  /**
+   * Append the provided bytes to this builder.
+   */
+  public void append(byte[] b, int off, int len) {
+    grow(ref.length + len);
+    System.arraycopy(b, off, ref.bytes, ref.length, len);
+    ref.length += len;
+  }
+
+  /**
+   * Append the provided bytes to this builder.
+   */
+  public void append(BytesRef ref) {
+    append(ref.bytes, ref.offset, ref.length);
+  }
+
+  /**
+   * Append the provided bytes to this builder.
+   */
+  public void append(BytesRefBuilder builder) {
+    append(builder.get());
+  }
+
+  /**
+   * Reset this builder to the empty state.
+   */
+  public void clear() {
+    setLength(0);
+  }
+
+  /**
+   * Replace the content of this builder with the provided bytes. Equivalent to
+   * calling {@link #clear()} and then {@link #append(byte[], int, int)}.
+   */
+  public void copyBytes(byte[] b, int off, int len) {
+    clear();
+    append(b, off, len);
+  }
+
+  /**
+   * Replace the content of this builder with the provided bytes. Equivalent to
+   * calling {@link #clear()} and then {@link #append(BytesRef)}.
+   */
+  public void copyBytes(BytesRef ref) {
+    clear();
+    append(ref);
+  }
+
+  /**
+   * Replace the content of this builder with the provided bytes. Equivalent to
+   * calling {@link #clear()} and then {@link #append(BytesRefBuilder)}.
+   */
+  public void copyBytes(BytesRefBuilder builder) {
+    clear();
+    append(builder);
+  }
+
+  /**
+   * Replace the content of this buffer with UTF-8 encoded bytes that would
+   * represent the provided text.
+   */
+  public void copyChars(CharSequence text) {
+    copyChars(text, 0, text.length());
+  }
+
+  /**
+   * Replace the content of this buffer with UTF-8 encoded bytes that would
+   * represent the provided text.
+   */
+  public void copyChars(CharSequence text, int off, int len) {
+    grow(len * UnicodeUtil.MAX_UTF8_BYTES_PER_CHAR);
+    ref.length = UnicodeUtil.UTF16toUTF8(text, off, len, ref.bytes);
+  }
+
+  /**
+   * Replace the content of this buffer with UTF-8 encoded bytes that would
+   * represent the provided text.
+   */
+  public void copyChars(char[] text, int off, int len) {
+    grow(len * UnicodeUtil.MAX_UTF8_BYTES_PER_CHAR);
+    ref.length = UnicodeUtil.UTF16toUTF8(text, off, len, ref.bytes);
+  }
+
+  /**
+   * Return a {@link BytesRef} that points to the internal content of this
+   * builder. Any update to the content of this builder might invalidate
+   * the provided <code>ref</code> and vice-versa.
+   */
+  public BytesRef get() {
+    assert ref.offset == 0 : "Modifying the offset of the returned ref is illegal";
+    return ref;
+  }
+
+  /**
+   * Build a new {@link BytesRef} that has the same content as this buffer.
+   */
+  public BytesRef toBytesRef() {
+    return new BytesRef(Arrays.copyOf(ref.bytes, ref.length));
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int hashCode() {
+    throw new UnsupportedOperationException();
+  }
+}

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

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CharsRefBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CharsRefBuilder.java?rev=1617493&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CharsRefBuilder.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CharsRefBuilder.java Tue Aug 12 14:46:45 2014
@@ -0,0 +1,144 @@
+package org.apache.lucene.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Arrays;
+
+/**
+ * A builder for {@link CharsRef} instances.
+ * @lucene.internal
+ */
+public class CharsRefBuilder {
+
+  private final CharsRef ref;
+
+  /** Sole constructor. */
+  public CharsRefBuilder() {
+    ref = new CharsRef();
+  }
+
+  /** Return a reference to the chars of this builder. */
+  public char[] chars() {
+    return ref.chars;
+  }
+
+  /** Return the number of chars in this buffer. */
+  public int length() {
+    return ref.length;
+  }
+
+  /** Set the length. */
+  public void setLength(int length) {
+    this.ref.length = length;
+  }
+
+  /** Return the char at the given offset. */
+  public char charAt(int offset) {
+    return ref.chars[offset];
+  }
+
+  /** Set a char. */
+  public void setCharAt(int offset, char b) {
+    ref.chars[offset] = b;
+  }
+
+  /**
+   * Reset this builder to the empty state.
+   */
+  public void clear() {
+    ref.length = 0;
+  }
+
+  /**
+   * Copies the given {@link CharsRef} referenced content into this instance.
+   */
+  public void copyChars(CharsRef other) {
+    copyChars(other.chars, other.offset, other.length);
+  }
+
+  /**
+   * Used to grow the reference array.
+   */
+  public void grow(int newLength) {
+    ref.chars = ArrayUtil.grow(ref.chars, newLength);
+  }
+
+  /**
+   * Copy the provided bytes, interpreted as UTF-8 bytes.
+   */
+  public void copyUTF8Bytes(byte[] bytes, int offset, int length) {
+    grow(length);
+    ref.length = UnicodeUtil.UTF8toUTF16(bytes, offset, length, ref.chars);
+  }
+
+  /**
+   * Copy the provided bytes, interpreted as UTF-8 bytes.
+   */
+  public void copyUTF8Bytes(BytesRef bytes) {
+    copyUTF8Bytes(bytes.bytes, bytes.offset, bytes.length);
+  }
+
+  /**
+   * Copies the given array into this instance.
+   */
+  public void copyChars(char[] otherChars, int otherOffset, int otherLength) {
+    grow(otherLength);
+    System.arraycopy(otherChars, otherOffset, ref.chars, 0, otherLength);
+    ref.length = otherLength;
+  }
+
+  /**
+   * Appends the given array to this CharsRef
+   */
+  public void append(char[] otherChars, int otherOffset, int otherLength) {
+    int newLen = ref.length + otherLength;
+    grow(newLen);
+    System.arraycopy(otherChars, otherOffset, ref.chars, ref.length, otherLength);
+    ref.length = newLen;
+  }
+
+  /**
+   * Return a {@link CharsRef} that points to the internal content of this
+   * builder. Any update to the content of this builder might invalidate
+   * the provided <code>ref</code> and vice-versa.
+   */
+  public CharsRef get() {
+    assert ref.offset == 0 : "Modifying the offset of the returned ref is illegal";
+    return ref;
+  }
+
+  /** Build a new {@link CharsRef} that has the same content as this builder. */
+  public CharsRef toCharsRef() {
+    return new CharsRef(Arrays.copyOf(ref.chars, ref.length), 0, ref.length);
+  }
+
+  @Override
+  public String toString() {
+    return get().toString();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int hashCode() {
+    throw new UnsupportedOperationException();
+  }
+}

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

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntsRefBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntsRefBuilder.java?rev=1617493&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntsRefBuilder.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntsRefBuilder.java Tue Aug 12 14:46:45 2014
@@ -0,0 +1,127 @@
+package org.apache.lucene.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * A builder for {@link IntsRef} instances.
+ * @lucene.internal
+ */
+public class IntsRefBuilder {
+
+  private final IntsRef ref;
+
+  /** Sole constructor. */
+  public IntsRefBuilder() {
+    ref = new IntsRef();
+  }
+
+  /** Return a reference to the ints of this builder. */
+  public int[] ints() {
+    return ref.ints;
+  }
+
+  /** Return the number of ints in this buffer. */
+  public int length() {
+    return ref.length;
+  }
+
+  /** Set the length. */
+  public void setLength(int length) {
+    this.ref.length = length;
+  }
+
+  /** Empty this builder. */
+  public void clear() {
+    setLength(0);
+  }
+
+  /** Return the int at the given offset. */
+  public int intAt(int offset) {
+    return ref.ints[offset];
+  }
+
+  /** Set an int. */
+  public void setIntAt(int offset, int b) {
+    ref.ints[offset] = b;
+  }
+
+  /** Append the provided int to this buffer. */
+  public void append(int i) {
+    grow(ref.length + 1);
+    ref.ints[ref.length++] = i;
+  }
+
+  /**
+   * Used to grow the reference array.
+   *
+   * In general this should not be used as it does not take the offset into account.
+   * @lucene.internal */
+  public void grow(int newLength) {
+    ref.ints = ArrayUtil.grow(ref.ints, newLength);
+  }
+
+  /**
+   * Copies the given array into this instance.
+   */
+  public void copyInts(int[] otherInts, int otherOffset, int otherLength) {
+    grow(otherLength);
+    System.arraycopy(otherInts, otherOffset, ref.ints, 0, otherLength);
+    ref.length = otherLength;
+  }
+
+  /**
+   * Copies the given array into this instance.
+   */
+  public void copyInts(IntsRef ints) {
+    copyInts(ints.ints, ints.offset, ints.length);
+  }
+
+  /**
+   * Copy the given UTF-8 bytes into this builder. Works as if the bytes were
+   * first converted from UTF-8 to UTF-32 and then copied into this builder.
+   */
+  public void copyUTF8Bytes(BytesRef bytes) {
+    grow(bytes.length);
+    ref.length = UnicodeUtil.UTF8toUTF32(bytes, ref.ints);
+  }
+
+  /**
+   * Return a {@link IntsRef} that points to the internal content of this
+   * builder. Any update to the content of this builder might invalidate
+   * the provided <code>ref</code> and vice-versa.
+   */
+  public IntsRef get() {
+    assert ref.offset == 0 : "Modifying the offset of the returned ref is illegal";
+    return ref;
+  }
+
+  /** Build a new {@link CharsRef} that has the same content as this builder. */
+  public IntsRef toIntsRef() {
+    return IntsRef.deepCopyOf(get());
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int hashCode() {
+    throw new UnsupportedOperationException();
+  }
+}

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

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

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

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

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

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