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 [3/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/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Wed Aug 13 09:36:54 2014
@@ -52,10 +52,13 @@ import org.apache.lucene.util.Accountabl
 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.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 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.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
@@ -94,16 +97,16 @@ class SimpleTextFieldsReader extends Fie
   
   private TreeMap<String,Long> readFields(IndexInput in) throws IOException {
     ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
-    BytesRef scratch = new BytesRef(10);
+    BytesRefBuilder scratch = new BytesRefBuilder();
     TreeMap<String,Long> fields = new TreeMap<>();
     
     while (true) {
       SimpleTextUtil.readLine(input, scratch);
-      if (scratch.equals(END)) {
+      if (scratch.get().equals(END)) {
         SimpleTextUtil.checkFooter(input);
         return fields;
-      } else if (StringHelper.startsWith(scratch, FIELD)) {
-        String fieldName = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, StandardCharsets.UTF_8);
+      } else if (StringHelper.startsWith(scratch.get(), FIELD)) {
+        String fieldName = new String(scratch.bytes(), FIELD.length, scratch.length() - FIELD.length, StandardCharsets.UTF_8);
         fields.put(fieldName, input.getFilePointer());
       }
     }
@@ -246,8 +249,8 @@ class SimpleTextFieldsReader extends Fie
     private int docID = -1;
     private int tf;
     private Bits liveDocs;
-    private final BytesRef scratch = new BytesRef(10);
-    private final CharsRef scratchUTF16 = new CharsRef(10);
+    private final BytesRefBuilder scratch = new BytesRefBuilder();
+    private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
     private int cost;
     
     public SimpleTextDocsEnum() {
@@ -289,7 +292,7 @@ class SimpleTextFieldsReader extends Fie
       while(true) {
         final long lineStart = in.getFilePointer();
         SimpleTextUtil.readLine(in, scratch);
-        if (StringHelper.startsWith(scratch, DOC)) {
+        if (StringHelper.startsWith(scratch.get(), DOC)) {
           if (!first && (liveDocs == null || liveDocs.get(docID))) {
             in.seek(lineStart);
             if (!omitTF) {
@@ -297,23 +300,23 @@ class SimpleTextFieldsReader extends Fie
             }
             return docID;
           }
-          UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+DOC.length, scratch.length-DOC.length, scratchUTF16);
-          docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
+          scratchUTF16.copyUTF8Bytes(scratch.bytes(), DOC.length, scratch.length()-DOC.length);
+          docID = ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
           termFreq = 0;
           first = false;
-        } else if (StringHelper.startsWith(scratch, FREQ)) {
-          UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+FREQ.length, scratch.length-FREQ.length, scratchUTF16);
-          termFreq = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
-        } else if (StringHelper.startsWith(scratch, POS)) {
+        } else if (StringHelper.startsWith(scratch.get(), FREQ)) {
+          scratchUTF16.copyUTF8Bytes(scratch.bytes(), FREQ.length, scratch.length()-FREQ.length);
+          termFreq = ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
+        } else if (StringHelper.startsWith(scratch.get(), POS)) {
           // skip termFreq++;
-        } else if (StringHelper.startsWith(scratch, START_OFFSET)) {
+        } else if (StringHelper.startsWith(scratch.get(), START_OFFSET)) {
           // skip
-        } else if (StringHelper.startsWith(scratch, END_OFFSET)) {
+        } else if (StringHelper.startsWith(scratch.get(), END_OFFSET)) {
           // skip
-        } else if (StringHelper.startsWith(scratch, PAYLOAD)) {
+        } else if (StringHelper.startsWith(scratch.get(), PAYLOAD)) {
           // skip
         } else {
-          assert StringHelper.startsWith(scratch, TERM) || StringHelper.startsWith(scratch, FIELD) || StringHelper.startsWith(scratch, END): "scratch=" + scratch.utf8ToString();
+          assert StringHelper.startsWith(scratch.get(), TERM) || StringHelper.startsWith(scratch.get(), FIELD) || StringHelper.startsWith(scratch.get(), END): "scratch=" + scratch.get().utf8ToString();
           if (!first && (liveDocs == null || liveDocs.get(docID))) {
             in.seek(lineStart);
             if (!omitTF) {
@@ -344,10 +347,10 @@ class SimpleTextFieldsReader extends Fie
     private int docID = -1;
     private int tf;
     private Bits liveDocs;
-    private final BytesRef scratch = new BytesRef(10);
-    private final BytesRef scratch2 = new BytesRef(10);
-    private final CharsRef scratchUTF16 = new CharsRef(10);
-    private final CharsRef scratchUTF16_2 = new CharsRef(10);
+    private final BytesRefBuilder scratch = new BytesRefBuilder();
+    private final BytesRefBuilder scratch2 = new BytesRefBuilder();
+    private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
+    private final CharsRefBuilder scratchUTF16_2 = new CharsRefBuilder();
     private BytesRef payload;
     private long nextDocStart;
     private boolean readOffsets;
@@ -398,30 +401,30 @@ class SimpleTextFieldsReader extends Fie
         final long lineStart = in.getFilePointer();
         SimpleTextUtil.readLine(in, scratch);
         //System.out.println("NEXT DOC: " + scratch.utf8ToString());
-        if (StringHelper.startsWith(scratch, DOC)) {
+        if (StringHelper.startsWith(scratch.get(), DOC)) {
           if (!first && (liveDocs == null || liveDocs.get(docID))) {
             nextDocStart = lineStart;
             in.seek(posStart);
             return docID;
           }
-          UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+DOC.length, scratch.length-DOC.length, scratchUTF16);
-          docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
+          scratchUTF16.copyUTF8Bytes(scratch.bytes(), DOC.length, scratch.length()-DOC.length);
+          docID = ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
           tf = 0;
           first = false;
-        } else if (StringHelper.startsWith(scratch, FREQ)) {
-          UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+FREQ.length, scratch.length-FREQ.length, scratchUTF16);
-          tf = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
+        } else if (StringHelper.startsWith(scratch.get(), FREQ)) {
+          scratchUTF16.copyUTF8Bytes(scratch.bytes(), FREQ.length, scratch.length()-FREQ.length);
+          tf = ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
           posStart = in.getFilePointer();
-        } else if (StringHelper.startsWith(scratch, POS)) {
+        } else if (StringHelper.startsWith(scratch.get(), POS)) {
           // skip
-        } else if (StringHelper.startsWith(scratch, START_OFFSET)) {
+        } else if (StringHelper.startsWith(scratch.get(), START_OFFSET)) {
           // skip
-        } else if (StringHelper.startsWith(scratch, END_OFFSET)) {
+        } else if (StringHelper.startsWith(scratch.get(), END_OFFSET)) {
           // skip
-        } else if (StringHelper.startsWith(scratch, PAYLOAD)) {
+        } else if (StringHelper.startsWith(scratch.get(), PAYLOAD)) {
           // skip
         } else {
-          assert StringHelper.startsWith(scratch, TERM) || StringHelper.startsWith(scratch, FIELD) || StringHelper.startsWith(scratch, END);
+          assert StringHelper.startsWith(scratch.get(), TERM) || StringHelper.startsWith(scratch.get(), FIELD) || StringHelper.startsWith(scratch.get(), END);
           if (!first && (liveDocs == null || liveDocs.get(docID))) {
             nextDocStart = lineStart;
             in.seek(posStart);
@@ -443,34 +446,33 @@ class SimpleTextFieldsReader extends Fie
       final int pos;
       if (readPositions) {
         SimpleTextUtil.readLine(in, scratch);
-        assert StringHelper.startsWith(scratch, POS): "got line=" + scratch.utf8ToString();
-        UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+POS.length, scratch.length-POS.length, scratchUTF16_2);
-        pos = ArrayUtil.parseInt(scratchUTF16_2.chars, 0, scratchUTF16_2.length);
+        assert StringHelper.startsWith(scratch.get(), POS): "got line=" + scratch.get().utf8ToString();
+        scratchUTF16_2.copyUTF8Bytes(scratch.bytes(), POS.length, scratch.length()-POS.length);
+        pos = ArrayUtil.parseInt(scratchUTF16_2.chars(), 0, scratchUTF16_2.length());
       } else {
         pos = -1;
       }
 
       if (readOffsets) {
         SimpleTextUtil.readLine(in, scratch);
-        assert StringHelper.startsWith(scratch, START_OFFSET): "got line=" + scratch.utf8ToString();
-        UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+START_OFFSET.length, scratch.length-START_OFFSET.length, scratchUTF16_2);
-        startOffset = ArrayUtil.parseInt(scratchUTF16_2.chars, 0, scratchUTF16_2.length);
+        assert StringHelper.startsWith(scratch.get(), START_OFFSET): "got line=" + scratch.get().utf8ToString();
+        scratchUTF16_2.copyUTF8Bytes(scratch.bytes(), START_OFFSET.length, scratch.length()-START_OFFSET.length);
+        startOffset = ArrayUtil.parseInt(scratchUTF16_2.chars(), 0, scratchUTF16_2.length());
         SimpleTextUtil.readLine(in, scratch);
-        assert StringHelper.startsWith(scratch, END_OFFSET): "got line=" + scratch.utf8ToString();
-        UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+END_OFFSET.length, scratch.length-END_OFFSET.length, scratchUTF16_2);
-        endOffset = ArrayUtil.parseInt(scratchUTF16_2.chars, 0, scratchUTF16_2.length);
+        assert StringHelper.startsWith(scratch.get(), END_OFFSET): "got line=" + scratch.get().utf8ToString();
+        scratchUTF16_2.grow(scratch.length()-END_OFFSET.length);
+        scratchUTF16_2.copyUTF8Bytes(scratch.bytes(), END_OFFSET.length, scratch.length()-END_OFFSET.length);
+        endOffset = ArrayUtil.parseInt(scratchUTF16_2.chars(), 0, scratchUTF16_2.length());
       }
 
       final long fp = in.getFilePointer();
       SimpleTextUtil.readLine(in, scratch);
-      if (StringHelper.startsWith(scratch, PAYLOAD)) {
-        final int len = scratch.length - PAYLOAD.length;
-        if (scratch2.bytes.length < len) {
-          scratch2.grow(len);
-        }
-        System.arraycopy(scratch.bytes, PAYLOAD.length, scratch2.bytes, 0, len);
-        scratch2.length = len;
-        payload = scratch2;
+      if (StringHelper.startsWith(scratch.get(), PAYLOAD)) {
+        final int len = scratch.length() - PAYLOAD.length;
+        scratch2.grow(len);
+        System.arraycopy(scratch.bytes(), PAYLOAD.length, scratch2.bytes(), 0, len);
+        scratch2.setLength(len);
+        payload = scratch2.get();
       } else {
         payload = null;
         in.seek(fp);
@@ -522,8 +524,8 @@ class SimpleTextFieldsReader extends Fie
     private int docCount;
     private FST<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> fst;
     private int termCount;
-    private final BytesRef scratch = new BytesRef(10);
-    private final CharsRef scratchUTF16 = new CharsRef(10);
+    private final BytesRefBuilder scratch = new BytesRefBuilder();
+    private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
 
     public SimpleTextTerms(String field, long termsStart, int maxDoc) throws IOException {
       this.maxDoc = maxDoc;
@@ -541,43 +543,41 @@ class SimpleTextFieldsReader extends Fie
       b = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
       IndexInput in = SimpleTextFieldsReader.this.in.clone();
       in.seek(termsStart);
-      final BytesRef lastTerm = new BytesRef(10);
+      final BytesRefBuilder lastTerm = new BytesRefBuilder();
       long lastDocsStart = -1;
       int docFreq = 0;
       long totalTermFreq = 0;
       FixedBitSet visitedDocs = new FixedBitSet(maxDoc);
-      final IntsRef scratchIntsRef = new IntsRef();
+      final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
       while(true) {
         SimpleTextUtil.readLine(in, scratch);
-        if (scratch.equals(END) || StringHelper.startsWith(scratch, FIELD)) {
+        if (scratch.get().equals(END) || StringHelper.startsWith(scratch.get(), FIELD)) {
           if (lastDocsStart != -1) {
-            b.add(Util.toIntsRef(lastTerm, scratchIntsRef),
+            b.add(Util.toIntsRef(lastTerm.get(), scratchIntsRef),
                   outputs.newPair(lastDocsStart,
                                   outputsInner.newPair((long) docFreq, totalTermFreq)));
             sumTotalTermFreq += totalTermFreq;
           }
           break;
-        } else if (StringHelper.startsWith(scratch, DOC)) {
+        } else if (StringHelper.startsWith(scratch.get(), DOC)) {
           docFreq++;
           sumDocFreq++;
-          UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+DOC.length, scratch.length-DOC.length, scratchUTF16);
-          int docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
+          scratchUTF16.copyUTF8Bytes(scratch.bytes(), DOC.length, scratch.length()-DOC.length);
+          int docID = ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
           visitedDocs.set(docID);
-        } else if (StringHelper.startsWith(scratch, FREQ)) {
-          UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+FREQ.length, scratch.length-FREQ.length, scratchUTF16);
-          totalTermFreq += ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
-        } else if (StringHelper.startsWith(scratch, TERM)) {
+        } else if (StringHelper.startsWith(scratch.get(), FREQ)) {
+          scratchUTF16.copyUTF8Bytes(scratch.bytes(), FREQ.length, scratch.length()-FREQ.length);
+          totalTermFreq += ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
+        } else if (StringHelper.startsWith(scratch.get(), TERM)) {
           if (lastDocsStart != -1) {
-            b.add(Util.toIntsRef(lastTerm, scratchIntsRef), outputs.newPair(lastDocsStart,
+            b.add(Util.toIntsRef(lastTerm.get(), scratchIntsRef), outputs.newPair(lastDocsStart,
                                                                             outputsInner.newPair((long) docFreq, totalTermFreq)));
           }
           lastDocsStart = in.getFilePointer();
-          final int len = scratch.length - TERM.length;
-          if (len > lastTerm.length) {
-            lastTerm.grow(len);
-          }
-          System.arraycopy(scratch.bytes, TERM.length, lastTerm.bytes, 0, len);
-          lastTerm.length = len;
+          final int len = scratch.length() - TERM.length;
+          lastTerm.grow(len);
+          System.arraycopy(scratch.bytes(), TERM.length, lastTerm.bytes(), 0, len);
+          lastTerm.setLength(len);
           docFreq = 0;
           sumTotalTermFreq += totalTermFreq;
           totalTermFreq = 0;
@@ -598,7 +598,7 @@ class SimpleTextFieldsReader extends Fie
     @Override
     public long ramBytesUsed() {
       return TERMS_BASE_RAM_BYTES_USED + (fst!=null ? fst.ramBytesUsed() : 0)
-          + RamUsageEstimator.sizeOf(scratch.bytes) + RamUsageEstimator.sizeOf(scratchUTF16.chars);
+          + RamUsageEstimator.sizeOf(scratch.bytes()) + RamUsageEstimator.sizeOf(scratchUTF16.chars());
     }
 
     @Override

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java Wed Aug 13 09:36:54 2014
@@ -29,11 +29,12 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 class SimpleTextFieldsWriter extends FieldsConsumer {
   
   private IndexOutput out;
-  private final BytesRef scratch = new BytesRef(10);
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
 
   final static BytesRef END          = new BytesRef("END");
   final static BytesRef FIELD        = new BytesRef("field ");

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java Wed Aug 13 09:36:54 2014
@@ -31,7 +31,9 @@ import org.apache.lucene.store.IndexOutp
 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.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MutableBits;
 import org.apache.lucene.util.StringHelper;
@@ -65,8 +67,8 @@ public class SimpleTextLiveDocsFormat ex
   @Override
   public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
     assert info.hasDeletions();
-    BytesRef scratch = new BytesRef();
-    CharsRef scratchUTF16 = new CharsRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
     
     String fileName = IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getDelGen());
     ChecksumIndexInput in = null;
@@ -75,15 +77,15 @@ public class SimpleTextLiveDocsFormat ex
       in = dir.openChecksumInput(fileName, context);
       
       SimpleTextUtil.readLine(in, scratch);
-      assert StringHelper.startsWith(scratch, SIZE);
-      int size = parseIntAt(scratch, SIZE.length, scratchUTF16);
+      assert StringHelper.startsWith(scratch.get(), SIZE);
+      int size = parseIntAt(scratch.get(), SIZE.length, scratchUTF16);
       
       BitSet bits = new BitSet(size);
       
       SimpleTextUtil.readLine(in, scratch);
-      while (!scratch.equals(END)) {
-        assert StringHelper.startsWith(scratch, DOC);
-        int docid = parseIntAt(scratch, DOC.length, scratchUTF16);
+      while (!scratch.get().equals(END)) {
+        assert StringHelper.startsWith(scratch.get(), DOC);
+        int docid = parseIntAt(scratch.get(), DOC.length, scratchUTF16);
         bits.set(docid);
         SimpleTextUtil.readLine(in, scratch);
       }
@@ -101,16 +103,16 @@ public class SimpleTextLiveDocsFormat ex
     }
   }
   
-  private int parseIntAt(BytesRef bytes, int offset, CharsRef scratch) {
-    UnicodeUtil.UTF8toUTF16(bytes.bytes, bytes.offset+offset, bytes.length-offset, scratch);
-    return ArrayUtil.parseInt(scratch.chars, 0, scratch.length);
+  private int parseIntAt(BytesRef bytes, int offset, CharsRefBuilder scratch) {
+    scratch.copyUTF8Bytes(bytes.bytes, bytes.offset + offset, bytes.length-offset);
+    return ArrayUtil.parseInt(scratch.chars(), 0, scratch.length());
   }
 
   @Override
   public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
     BitSet set = ((SimpleTextBits) bits).bits;
     int size = bits.length();
-    BytesRef scratch = new BytesRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
     
     String fileName = IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getNextDelGen());
     IndexOutput out = null;

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java Wed Aug 13 09:36:54 2014
@@ -39,7 +39,7 @@ import org.apache.lucene.index.SegmentIn
 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;
 
@@ -53,47 +53,47 @@ public class SimpleTextSegmentInfoReader
 
   @Override
   public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
-    BytesRef scratch = new BytesRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
     String segFileName = IndexFileNames.segmentFileName(segmentName, "", SimpleTextSegmentInfoFormat.SI_EXTENSION);
     ChecksumIndexInput input = directory.openChecksumInput(segFileName, context);
     boolean success = false;
     try {
       SimpleTextUtil.readLine(input, scratch);
-      assert StringHelper.startsWith(scratch, SI_VERSION);
+      assert StringHelper.startsWith(scratch.get(), SI_VERSION);
       final String version = readString(SI_VERSION.length, scratch);
     
       SimpleTextUtil.readLine(input, scratch);
-      assert StringHelper.startsWith(scratch, SI_DOCCOUNT);
+      assert StringHelper.startsWith(scratch.get(), SI_DOCCOUNT);
       final int docCount = Integer.parseInt(readString(SI_DOCCOUNT.length, scratch));
     
       SimpleTextUtil.readLine(input, scratch);
-      assert StringHelper.startsWith(scratch, SI_USECOMPOUND);
+      assert StringHelper.startsWith(scratch.get(), SI_USECOMPOUND);
       final boolean isCompoundFile = Boolean.parseBoolean(readString(SI_USECOMPOUND.length, scratch));
     
       SimpleTextUtil.readLine(input, scratch);
-      assert StringHelper.startsWith(scratch, SI_NUM_DIAG);
+      assert StringHelper.startsWith(scratch.get(), SI_NUM_DIAG);
       int numDiag = Integer.parseInt(readString(SI_NUM_DIAG.length, scratch));
       Map<String,String> diagnostics = new HashMap<>();
 
       for (int i = 0; i < numDiag; i++) {
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, SI_DIAG_KEY);
+        assert StringHelper.startsWith(scratch.get(), SI_DIAG_KEY);
         String key = readString(SI_DIAG_KEY.length, scratch);
       
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, SI_DIAG_VALUE);
+        assert StringHelper.startsWith(scratch.get(), SI_DIAG_VALUE);
         String value = readString(SI_DIAG_VALUE.length, scratch);
         diagnostics.put(key, value);
       }
       
       SimpleTextUtil.readLine(input, scratch);
-      assert StringHelper.startsWith(scratch, SI_NUM_FILES);
+      assert StringHelper.startsWith(scratch.get(), SI_NUM_FILES);
       int numFiles = Integer.parseInt(readString(SI_NUM_FILES.length, scratch));
       Set<String> files = new HashSet<>();
 
       for (int i = 0; i < numFiles; i++) {
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, SI_FILE);
+        assert StringHelper.startsWith(scratch.get(), SI_FILE);
         String fileName = readString(SI_FILE.length, scratch);
         files.add(fileName);
       }
@@ -114,7 +114,7 @@ public class SimpleTextSegmentInfoReader
     }
   }
 
-  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/SimpleTextSegmentInfoWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.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;
 
 /**
@@ -58,7 +59,7 @@ public class SimpleTextSegmentInfoWriter
     IndexOutput output = dir.createOutput(segFileName, ioContext);
 
     try {
-      BytesRef scratch = new BytesRef();
+      BytesRefBuilder scratch = new BytesRefBuilder();
     
       SimpleTextUtil.write(output, SI_VERSION);
       SimpleTextUtil.write(output, si.getVersion(), scratch);

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java Wed Aug 13 09:36:54 2014
@@ -34,7 +34,9 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
@@ -57,8 +59,8 @@ public class SimpleTextStoredFieldsReade
 
   private long offsets[]; /* docid -> offset in .fld file */
   private IndexInput in;
-  private BytesRef scratch = new BytesRef();
-  private CharsRef scratchUTF16 = new CharsRef();
+  private BytesRefBuilder scratch = new BytesRefBuilder();
+  private CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
   private final FieldInfos fieldInfos;
 
   public SimpleTextStoredFieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
@@ -91,9 +93,9 @@ public class SimpleTextStoredFieldsReade
     ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
     offsets = new long[size];
     int upto = 0;
-    while (!scratch.equals(END)) {
+    while (!scratch.get().equals(END)) {
       SimpleTextUtil.readLine(input, scratch);
-      if (StringHelper.startsWith(scratch, DOC)) {
+      if (StringHelper.startsWith(scratch.get(), DOC)) {
         offsets[upto] = input.getFilePointer();
         upto++;
       }
@@ -108,28 +110,28 @@ public class SimpleTextStoredFieldsReade
     
     while (true) {
       readLine();
-      if (StringHelper.startsWith(scratch, FIELD) == false) {
+      if (StringHelper.startsWith(scratch.get(), FIELD) == false) {
         break;
       }
       int fieldNumber = parseIntAt(FIELD.length);
       FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
       readLine();
-      assert StringHelper.startsWith(scratch, NAME);
+      assert StringHelper.startsWith(scratch.get(), NAME);
       readLine();
-      assert StringHelper.startsWith(scratch, TYPE);
+      assert StringHelper.startsWith(scratch.get(), TYPE);
       
       final BytesRef type;
-      if (equalsAt(TYPE_STRING, scratch, TYPE.length)) {
+      if (equalsAt(TYPE_STRING, scratch.get(), TYPE.length)) {
         type = TYPE_STRING;
-      } else if (equalsAt(TYPE_BINARY, scratch, TYPE.length)) {
+      } else if (equalsAt(TYPE_BINARY, scratch.get(), TYPE.length)) {
         type = TYPE_BINARY;
-      } else if (equalsAt(TYPE_INT, scratch, TYPE.length)) {
+      } else if (equalsAt(TYPE_INT, scratch.get(), TYPE.length)) {
         type = TYPE_INT;
-      } else if (equalsAt(TYPE_LONG, scratch, TYPE.length)) {
+      } else if (equalsAt(TYPE_LONG, scratch.get(), TYPE.length)) {
         type = TYPE_LONG;
-      } else if (equalsAt(TYPE_FLOAT, scratch, TYPE.length)) {
+      } else if (equalsAt(TYPE_FLOAT, scratch.get(), TYPE.length)) {
         type = TYPE_FLOAT;
-      } else if (equalsAt(TYPE_DOUBLE, scratch, TYPE.length)) {
+      } else if (equalsAt(TYPE_DOUBLE, scratch.get(), TYPE.length)) {
         type = TYPE_DOUBLE;
       } else {
         throw new RuntimeException("unknown field type");
@@ -141,7 +143,7 @@ public class SimpleTextStoredFieldsReade
           break;
         case NO:   
           readLine();
-          assert StringHelper.startsWith(scratch, VALUE);
+          assert StringHelper.startsWith(scratch.get(), VALUE);
           break;
         case STOP: return;
       }
@@ -150,24 +152,24 @@ public class SimpleTextStoredFieldsReade
   
   private void readField(BytesRef type, FieldInfo fieldInfo, StoredFieldVisitor visitor) throws IOException {
     readLine();
-    assert StringHelper.startsWith(scratch, VALUE);
+    assert StringHelper.startsWith(scratch.get(), VALUE);
     if (type == TYPE_STRING) {
-      visitor.stringField(fieldInfo, new String(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, StandardCharsets.UTF_8));
+      visitor.stringField(fieldInfo, new String(scratch.bytes(), VALUE.length, scratch.length()-VALUE.length, StandardCharsets.UTF_8));
     } else if (type == TYPE_BINARY) {
-      byte[] copy = new byte[scratch.length-VALUE.length];
-      System.arraycopy(scratch.bytes, scratch.offset+VALUE.length, copy, 0, copy.length);
+      byte[] copy = new byte[scratch.length()-VALUE.length];
+      System.arraycopy(scratch.bytes(), VALUE.length, copy, 0, copy.length);
       visitor.binaryField(fieldInfo, copy);
     } else if (type == TYPE_INT) {
-      UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, scratchUTF16);
+      scratchUTF16.copyUTF8Bytes(scratch.bytes(), VALUE.length, scratch.length()-VALUE.length);
       visitor.intField(fieldInfo, Integer.parseInt(scratchUTF16.toString()));
     } else if (type == TYPE_LONG) {
-      UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, scratchUTF16);
+      scratchUTF16.copyUTF8Bytes(scratch.bytes(), VALUE.length, scratch.length()-VALUE.length);
       visitor.longField(fieldInfo, Long.parseLong(scratchUTF16.toString()));
     } else if (type == TYPE_FLOAT) {
-      UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, scratchUTF16);
+      scratchUTF16.copyUTF8Bytes(scratch.bytes(), VALUE.length, scratch.length()-VALUE.length);
       visitor.floatField(fieldInfo, Float.parseFloat(scratchUTF16.toString()));
     } else if (type == TYPE_DOUBLE) {
-      UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, scratchUTF16);
+      scratchUTF16.copyUTF8Bytes(scratch.bytes(), VALUE.length, scratch.length()-VALUE.length);
       visitor.doubleField(fieldInfo, Double.parseDouble(scratchUTF16.toString()));
     }
   }
@@ -195,8 +197,8 @@ public class SimpleTextStoredFieldsReade
   }
   
   private int parseIntAt(int offset) {
-    UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+offset, scratch.length-offset, scratchUTF16);
-    return ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
+    scratchUTF16.copyUTF8Bytes(scratch.bytes(), offset, scratch.length()-offset);
+    return ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
   }
   
   private boolean equalsAt(BytesRef a, BytesRef b, int bOffset) {
@@ -207,7 +209,7 @@ public class SimpleTextStoredFieldsReade
   @Override
   public long ramBytesUsed() {
     return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(offsets)
-        + RamUsageEstimator.sizeOf(scratch.bytes) + RamUsageEstimator.sizeOf(scratchUTF16.chars);
+        + RamUsageEstimator.sizeOf(scratch.bytes()) + RamUsageEstimator.sizeOf(scratchUTF16.chars());
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java Wed Aug 13 09:36:54 2014
@@ -28,6 +28,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;
 
 /**
@@ -58,7 +59,7 @@ public class SimpleTextStoredFieldsWrite
   final static BytesRef TYPE     = new BytesRef("    type ");
   final static BytesRef VALUE    = new BytesRef("    value ");
   
-  private final BytesRef scratch = new BytesRef();
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
   
   public SimpleTextStoredFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
     this.directory = directory;

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Wed Aug 13 09:36:54 2014
@@ -42,11 +42,12 @@ 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.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.UnicodeUtil;
 
 import static org.apache.lucene.codecs.simpletext.SimpleTextTermVectorsWriter.*;
 
@@ -65,8 +66,8 @@ public class SimpleTextTermVectorsReader
 
   private long offsets[]; /* docid -> offset in .vec file */
   private IndexInput in;
-  private BytesRef scratch = new BytesRef();
-  private CharsRef scratchUTF16 = new CharsRef();
+  private BytesRefBuilder scratch = new BytesRefBuilder();
+  private CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
   
   public SimpleTextTermVectorsReader(Directory directory, SegmentInfo si, IOContext context) throws IOException {
     boolean success = false;
@@ -96,9 +97,9 @@ public class SimpleTextTermVectorsReader
     ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
     offsets = new long[maxDoc];
     int upto = 0;
-    while (!scratch.equals(END)) {
+    while (!scratch.get().equals(END)) {
       SimpleTextUtil.readLine(input, scratch);
-      if (StringHelper.startsWith(scratch, DOC)) {
+      if (StringHelper.startsWith(scratch.get(), DOC)) {
         offsets[upto] = input.getFilePointer();
         upto++;
       }
@@ -112,54 +113,54 @@ public class SimpleTextTermVectorsReader
     SortedMap<String,SimpleTVTerms> fields = new TreeMap<>();
     in.seek(offsets[doc]);
     readLine();
-    assert StringHelper.startsWith(scratch, NUMFIELDS);
+    assert StringHelper.startsWith(scratch.get(), NUMFIELDS);
     int numFields = parseIntAt(NUMFIELDS.length);
     if (numFields == 0) {
       return null; // no vectors for this doc
     }
     for (int i = 0; i < numFields; i++) {
       readLine();
-      assert StringHelper.startsWith(scratch, FIELD);
+      assert StringHelper.startsWith(scratch.get(), FIELD);
       // skip fieldNumber:
       parseIntAt(FIELD.length);
       
       readLine();
-      assert StringHelper.startsWith(scratch, FIELDNAME);
+      assert StringHelper.startsWith(scratch.get(), FIELDNAME);
       String fieldName = readString(FIELDNAME.length, scratch);
       
       readLine();
-      assert StringHelper.startsWith(scratch, FIELDPOSITIONS);
+      assert StringHelper.startsWith(scratch.get(), FIELDPOSITIONS);
       boolean positions = Boolean.parseBoolean(readString(FIELDPOSITIONS.length, scratch));
       
       readLine();
-      assert StringHelper.startsWith(scratch, FIELDOFFSETS);
+      assert StringHelper.startsWith(scratch.get(), FIELDOFFSETS);
       boolean offsets = Boolean.parseBoolean(readString(FIELDOFFSETS.length, scratch));
       
       readLine();
-      assert StringHelper.startsWith(scratch, FIELDPAYLOADS);
+      assert StringHelper.startsWith(scratch.get(), FIELDPAYLOADS);
       boolean payloads = Boolean.parseBoolean(readString(FIELDPAYLOADS.length, scratch));
       
       readLine();
-      assert StringHelper.startsWith(scratch, FIELDTERMCOUNT);
+      assert StringHelper.startsWith(scratch.get(), FIELDTERMCOUNT);
       int termCount = parseIntAt(FIELDTERMCOUNT.length);
       
       SimpleTVTerms terms = new SimpleTVTerms(offsets, positions, payloads);
       fields.put(fieldName, terms);
       
+      BytesRefBuilder term = new BytesRefBuilder();
       for (int j = 0; j < termCount; j++) {
         readLine();
-        assert StringHelper.startsWith(scratch, TERMTEXT);
-        BytesRef term = new BytesRef();
-        int termLength = scratch.length - TERMTEXT.length;
+        assert StringHelper.startsWith(scratch.get(), TERMTEXT);
+        int termLength = scratch.length() - TERMTEXT.length;
         term.grow(termLength);
-        term.length = termLength;
-        System.arraycopy(scratch.bytes, scratch.offset+TERMTEXT.length, term.bytes, term.offset, termLength);
+        term.setLength(termLength);
+        System.arraycopy(scratch.bytes(), TERMTEXT.length, term.bytes(), 0, termLength);
         
         SimpleTVPostings postings = new SimpleTVPostings();
-        terms.terms.put(term, postings);
+        terms.terms.put(term.toBytesRef(), postings);
         
         readLine();
-        assert StringHelper.startsWith(scratch, TERMFREQ);
+        assert StringHelper.startsWith(scratch.get(), TERMFREQ);
         postings.freq = parseIntAt(TERMFREQ.length);
         
         if (positions || offsets) {
@@ -178,16 +179,16 @@ public class SimpleTextTermVectorsReader
           for (int k = 0; k < postings.freq; k++) {
             if (positions) {
               readLine();
-              assert StringHelper.startsWith(scratch, POSITION);
+              assert StringHelper.startsWith(scratch.get(), POSITION);
               postings.positions[k] = parseIntAt(POSITION.length);
               if (payloads) {
                 readLine();
-                assert StringHelper.startsWith(scratch, PAYLOAD);
-                if (scratch.length - PAYLOAD.length == 0) {
+                assert StringHelper.startsWith(scratch.get(), PAYLOAD);
+                if (scratch.length() - PAYLOAD.length == 0) {
                   postings.payloads[k] = null;
                 } else {
-                  byte payloadBytes[] = new byte[scratch.length - PAYLOAD.length];
-                  System.arraycopy(scratch.bytes, scratch.offset+PAYLOAD.length, payloadBytes, 0, payloadBytes.length);
+                  byte payloadBytes[] = new byte[scratch.length() - PAYLOAD.length];
+                  System.arraycopy(scratch.bytes(), PAYLOAD.length, payloadBytes, 0, payloadBytes.length);
                   postings.payloads[k] = new BytesRef(payloadBytes);
                 }
               }
@@ -195,11 +196,11 @@ public class SimpleTextTermVectorsReader
             
             if (offsets) {
               readLine();
-              assert StringHelper.startsWith(scratch, STARTOFFSET);
+              assert StringHelper.startsWith(scratch.get(), STARTOFFSET);
               postings.startOffsets[k] = parseIntAt(STARTOFFSET.length);
               
               readLine();
-              assert StringHelper.startsWith(scratch, ENDOFFSET);
+              assert StringHelper.startsWith(scratch.get(), ENDOFFSET);
               postings.endOffsets[k] = parseIntAt(ENDOFFSET.length);
             }
           }
@@ -232,12 +233,12 @@ public class SimpleTextTermVectorsReader
   }
   
   private int parseIntAt(int offset) {
-    UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+offset, scratch.length-offset, scratchUTF16);
-    return ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
+    scratchUTF16.copyUTF8Bytes(scratch.bytes(), offset, scratch.length()-offset);
+    return ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
   }
   
-  private String readString(int offset, BytesRef scratch) {
-    UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+offset, scratch.length-offset, scratchUTF16);
+  private String readString(int offset, BytesRefBuilder scratch) {
+    scratchUTF16.copyUTF8Bytes(scratch.bytes(), offset, scratch.length()-offset);
     return scratchUTF16.toString();
   }
   

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java Wed Aug 13 09:36:54 2014
@@ -28,6 +28,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;
 
 /**
@@ -60,7 +61,7 @@ public class SimpleTextTermVectorsWriter
   private final String segment;
   private IndexOutput out;
   private int numDocsWritten = 0;
-  private final BytesRef scratch = new BytesRef();
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
   private boolean offsets;
   private boolean positions;
   private boolean payloads;

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java Wed Aug 13 09:36:54 2014
@@ -26,6 +26,7 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -34,9 +35,9 @@ class SimpleTextUtil {
   public final static byte ESCAPE = 92;
   final static BytesRef CHECKSUM = new BytesRef("checksum ");
   
-  public static void write(DataOutput out, String s, BytesRef scratch) throws IOException {
-    UnicodeUtil.UTF16toUTF8(s, 0, s.length(), scratch);
-    write(out, scratch);
+  public static void write(DataOutput out, String s, BytesRefBuilder scratch) throws IOException {
+    scratch.copyChars(s, 0, s.length());
+    write(out, scratch.get());
   }
 
   public static void write(DataOutput out, BytesRef b) throws IOException {
@@ -53,28 +54,25 @@ class SimpleTextUtil {
     out.writeByte(NEWLINE);
   }
   
-  public static void readLine(DataInput in, BytesRef scratch) throws IOException {
+  public static void readLine(DataInput in, BytesRefBuilder scratch) throws IOException {
     int upto = 0;
     while(true) {
       byte b = in.readByte();
-      if (scratch.bytes.length == upto) {
-        scratch.grow(1+upto);
-      }
+      scratch.grow(1+upto);
       if (b == ESCAPE) {
-        scratch.bytes[upto++] = in.readByte();
+        scratch.setByteAt(upto++, in.readByte());
       } else {
         if (b == NEWLINE) {
           break;
         } else {
-          scratch.bytes[upto++] = b;
+          scratch.setByteAt(upto++, b);
         }
       }
     }
-    scratch.offset = 0;
-    scratch.length = upto;
+    scratch.setLength(upto);
   }
 
-  public static void writeChecksum(IndexOutput out, BytesRef scratch) throws IOException {
+  public static void writeChecksum(IndexOutput out, BytesRefBuilder scratch) throws IOException {
     // Pad with zeros so different checksum values use the
     // same number of bytes
     // (BaseIndexFileFormatTestCase.testMergeStability cares):
@@ -85,13 +83,13 @@ class SimpleTextUtil {
   }
   
   public static void checkFooter(ChecksumIndexInput input) throws IOException {
-    BytesRef scratch = new BytesRef();
+    BytesRefBuilder scratch = new BytesRefBuilder();
     String expectedChecksum = String.format(Locale.ROOT, "%020d", input.getChecksum());
     SimpleTextUtil.readLine(input, scratch);
-    if (StringHelper.startsWith(scratch, CHECKSUM) == false) {
-      throw new CorruptIndexException("SimpleText failure: expected checksum line but got " + scratch.utf8ToString() + " (resource=" + input + ")");
+    if (StringHelper.startsWith(scratch.get(), CHECKSUM) == false) {
+      throw new CorruptIndexException("SimpleText failure: expected checksum line but got " + scratch.get().utf8ToString() + " (resource=" + input + ")");
     }
-    String actualChecksum = new BytesRef(scratch.bytes, CHECKSUM.length, scratch.length - CHECKSUM.length).utf8ToString();
+    String actualChecksum = new BytesRef(scratch.bytes(), CHECKSUM.length, scratch.length() - CHECKSUM.length).utf8ToString();
     if (!expectedChecksum.equals(actualChecksum)) {
       throw new CorruptIndexException("SimpleText checksum failure: " + actualChecksum + " != " + expectedChecksum + " (resource=" + input + ")");
     }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java Wed Aug 13 09:36:54 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.util.AttributeF
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeReflector;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.NumericUtils;
 
 /**
@@ -146,7 +147,7 @@ public final class NumericTokenStream ex
   public static final class NumericTermAttributeImpl extends AttributeImpl implements NumericTermAttribute,TermToBytesRefAttribute {
     private long value = 0L;
     private int valueSize = 0, shift = 0, precisionStep = 0;
-    private BytesRef bytes = new BytesRef();
+    private BytesRefBuilder bytes = new BytesRefBuilder();
     
     /** 
      * Creates, but does not yet initialize this attribute instance
@@ -156,7 +157,7 @@ public final class NumericTokenStream ex
 
     @Override
     public BytesRef getBytesRef() {
-      return bytes;
+      return bytes.get();
     }
     
     @Override
@@ -167,6 +168,7 @@ public final class NumericTokenStream ex
       } else {
         NumericUtils.intToPrefixCoded((int) value, shift, bytes);
       }
+      bytes.get();
     }
 
     @Override
@@ -200,7 +202,7 @@ public final class NumericTokenStream ex
     @Override
     public void reflectWith(AttributeReflector reflector) {
       fillBytesRef();
-      reflector.reflect(TermToBytesRefAttribute.class, "bytes", BytesRef.deepCopyOf(bytes));
+      reflector.reflect(TermToBytesRefAttribute.class, "bytes", bytes.toBytesRef());
       reflector.reflect(NumericTermAttribute.class, "shift", shift);
       reflector.reflect(NumericTermAttribute.class, "rawValue", getRawValue());
       reflector.reflect(NumericTermAttribute.class, "valueSize", valueSize);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java Wed Aug 13 09:36:54 2014
@@ -23,8 +23,8 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeReflector;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.UnicodeUtil;
 
 /** Default implementation of {@link CharTermAttribute}. */
 public class CharTermAttributeImpl extends AttributeImpl implements CharTermAttribute, TermToBytesRefAttribute, Cloneable {
@@ -83,16 +83,17 @@ public class CharTermAttributeImpl exten
   }
   
   // *** TermToBytesRefAttribute interface ***
-  private BytesRef bytes = new BytesRef(MIN_BUFFER_SIZE);
+  private BytesRefBuilder bytes = new BytesRefBuilder();
 
   @Override
   public void fillBytesRef() {
-    UnicodeUtil.UTF16toUTF8(termBuffer, 0, termLength, bytes);
+    bytes.copyChars(termBuffer, 0, termLength);
+    bytes.get();
   }
 
   @Override
   public BytesRef getBytesRef() {
-    return bytes;
+    return bytes.get();
   }
   
   // *** CharSequence interface ***
@@ -228,7 +229,8 @@ public class CharTermAttributeImpl exten
     // Do a deep clone
     t.termBuffer = new char[this.termLength];
     System.arraycopy(this.termBuffer, 0, t.termBuffer, 0, this.termLength);
-    t.bytes = BytesRef.deepCopyOf(bytes);
+    t.bytes = new BytesRefBuilder();
+    t.bytes.copyBytes(bytes.get());
     return t;
   }
   
@@ -271,7 +273,7 @@ public class CharTermAttributeImpl exten
   public void reflectWith(AttributeReflector reflector) {
     reflector.reflect(CharTermAttribute.class, "term", toString());
     fillBytesRef();
-    reflector.reflect(TermToBytesRefAttribute.class, "bytes", BytesRef.deepCopyOf(bytes));
+    reflector.reflect(TermToBytesRefAttribute.class, "bytes", bytes.toBytesRef());
   }
   
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java Wed Aug 13 09:36:54 2014
@@ -34,6 +34,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 
 /**
  * Codec API for writing term vectors:
@@ -125,7 +126,7 @@ public abstract class TermVectorsWriter 
   public void addProx(int numProx, DataInput positions, DataInput offsets) throws IOException {
     int position = 0;
     int lastOffset = 0;
-    BytesRef payload = null;
+    BytesRefBuilder payload = null;
 
     for (int i = 0; i < numProx; i++) {
       final int startOffset;
@@ -143,15 +144,13 @@ public abstract class TermVectorsWriter 
           final int payloadLength = positions.readVInt();
 
           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);
 
-          positions.readBytes(payload.bytes, 0, payloadLength);
-          payload.length = payloadLength;
-          thisPayload = payload;
+          positions.readBytes(payload.bytes(), 0, payloadLength);
+          payload.setLength(payloadLength);
+          thisPayload = payload.get();
         } else {
           thisPayload = null;
         }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java Wed Aug 13 09:36:54 2014
@@ -42,9 +42,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.ByteSequenceOutputs;
@@ -445,7 +447,7 @@ public final class BlockTreeTermsWriter 
       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);
@@ -510,7 +512,7 @@ public final class BlockTreeTermsWriter 
     // TODO: maybe we could add bulk-add method to
     // Builder?  Takes FST and unions it w/ current
     // FST.
-    private void append(Builder<BytesRef> builder, FST<BytesRef> subIndex, IntsRef scratchIntsRef) throws IOException {
+    private void append(Builder<BytesRef> builder, FST<BytesRef> subIndex, IntsRefBuilder scratchIntsRef) throws IOException {
       final BytesRefFSTEnum<BytesRef> subIndexEnum = new BytesRefFSTEnum<>(subIndex);
       BytesRefFSTEnum.InputOutput<BytesRef> indexEnt;
       while((indexEnt = subIndexEnum.next()) != null) {
@@ -523,7 +525,7 @@ public final class BlockTreeTermsWriter 
   }
 
   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;
@@ -540,7 +542,7 @@ public final class BlockTreeTermsWriter 
     // 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;
@@ -676,7 +678,7 @@ public final class BlockTreeTermsWriter 
       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.get().bytes, 0, prefix.bytes, 0, prefixLength);
       prefix.length = prefixLength;
 
       // Write block header:
@@ -920,18 +922,18 @@ public final class BlockTreeTermsWriter 
 
     /** 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/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java Wed Aug 13 09:36:54 2014
@@ -32,10 +32,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.RamUsageEstimator;
-import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.fst.Outputs;
 import org.apache.lucene.util.fst.Util;
 
 /** Iterates through terms in this field */
@@ -63,7 +62,7 @@ final class SegmentTermsEnum extends Ter
   // 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<BytesRef>[] arcs = new FST.Arc[1];
@@ -174,7 +173,7 @@ final class SegmentTermsEnum extends Ter
       while(true) {
         if (currentFrame.next()) {
           // Push to new block:
-          currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+          currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length());
           currentFrame.fpOrig = currentFrame.fp;
           // This is a "next" frame -- even if it's
           // floor'd we must pretend it isn't so we don't
@@ -184,7 +183,7 @@ final class SegmentTermsEnum extends Ter
           currentFrame.loadBlock();
           stats.startBlock(currentFrame, !currentFrame.isLastInFloor);
         } else {
-          stats.term(term);
+          stats.term(term.get());
           break;
         }
       }
@@ -205,7 +204,7 @@ final class SegmentTermsEnum extends Ter
     currentFrame.rewind();
     currentFrame.loadBlock();
     validIndexPrefix = 0;
-    term.length = 0;
+    term.clear();
 
     return stats;
   }
@@ -323,9 +322,7 @@ final class SegmentTermsEnum extends Ter
       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();
 
@@ -359,7 +356,7 @@ final class SegmentTermsEnum extends Ter
       targetUpto = 0;
           
       SegmentTermsEnumFrame lastFrame = stack[0];
-      assert validIndexPrefix <= term.length;
+      assert validIndexPrefix <= term.length();
 
       final int targetLimit = Math.min(target.length, validIndexPrefix);
 
@@ -370,7 +367,7 @@ final class SegmentTermsEnum extends Ter
 
       // 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);
         // }
@@ -395,9 +392,9 @@ final class SegmentTermsEnum extends Ter
         // 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]) + ")");
           // }
@@ -408,7 +405,7 @@ final class SegmentTermsEnum extends Ter
         }
 
         if (cmp == 0) {
-          cmp = term.length - target.length;
+          cmp = term.length() - target.length;
         }
         targetUpto = targetUptoMid;
       }
@@ -435,7 +432,7 @@ final class SegmentTermsEnum extends Ter
         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");
@@ -498,8 +495,8 @@ final class SegmentTermsEnum extends Ter
 
         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));
           // }
@@ -523,7 +520,7 @@ final class SegmentTermsEnum extends Ter
       } 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 != BlockTreeTermsWriter.NO_OUTPUT) {
@@ -551,7 +548,7 @@ final class SegmentTermsEnum extends Ter
     // 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));
       // }
@@ -580,10 +577,8 @@ final class SegmentTermsEnum extends Ter
     if (fr.index == null) {
       throw new IllegalStateException("terms index was not loaded");
     }
-   
-    if (term.bytes.length <= target.length) {
-      term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
-    }
+
+    term.grow(1 + target.length);
 
     assert clearEOF();
 
@@ -617,7 +612,7 @@ final class SegmentTermsEnum extends Ter
       targetUpto = 0;
           
       SegmentTermsEnumFrame lastFrame = stack[0];
-      assert validIndexPrefix <= term.length;
+      assert validIndexPrefix <= term.length();
 
       final int targetLimit = Math.min(target.length, validIndexPrefix);
 
@@ -628,7 +623,7 @@ final class SegmentTermsEnum extends Ter
 
       // 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);
         //}
@@ -656,9 +651,9 @@ final class SegmentTermsEnum extends Ter
         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]) + ")");
           //}
@@ -669,7 +664,7 @@ final class SegmentTermsEnum extends Ter
         }
 
         if (cmp == 0) {
-          cmp = term.length - target.length;
+          cmp = term.length() - target.length;
         }
         targetUpto = targetUptoMid;
       }
@@ -696,7 +691,7 @@ final class SegmentTermsEnum extends Ter
         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");
@@ -780,7 +775,7 @@ final class SegmentTermsEnum extends Ter
         }
       } 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;
@@ -840,7 +835,7 @@ final class SegmentTermsEnum extends Ter
       while(true) {
         SegmentTermsEnumFrame f = getFrame(ord);
         assert f != null;
-        final BytesRef prefix = new BytesRef(term.bytes, 0, f.prefix);
+        final BytesRef prefix = new BytesRef(term.get().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=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
         } else {
@@ -848,8 +843,8 @@ final class SegmentTermsEnum extends Ter
         }
         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");
           }
           BytesRef output = Util.get(fr.index, prefix);
@@ -912,7 +907,7 @@ final class SegmentTermsEnum extends Ter
       // 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;
     }
 
@@ -925,7 +920,7 @@ final class SegmentTermsEnum extends Ter
         if (currentFrame.ord == 0) {
           //if (DEBUG) System.out.println("  return null");
           assert setEOF();
-          term.length = 0;
+          term.clear();
           validIndexPrefix = 0;
           currentFrame.rewind();
           termExists = false;
@@ -937,7 +932,7 @@ final class SegmentTermsEnum extends Ter
         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);
         }
@@ -955,7 +950,7 @@ final class SegmentTermsEnum extends Ter
       if (currentFrame.next()) {
         // Push to new block:
         //if (DEBUG) System.out.println("  push frame");
-        currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+        currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length());
         // This is a "next" frame -- even if it's
         // floor'd we must pretend it isn't so we don't
         // try to scan to the right floor frame:
@@ -964,7 +959,7 @@ final class SegmentTermsEnum extends Ter
         currentFrame.loadBlock();
       } else {
         //if (DEBUG) System.out.println("  return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord);
-        return term;
+        return term.get();
       }
     }
   }
@@ -972,7 +967,7 @@ final class SegmentTermsEnum extends Ter
   @Override
   public BytesRef term() {
     assert !eof;
-    return term;
+    return term.get();
   }
 
   @Override
@@ -1022,7 +1017,7 @@ final class SegmentTermsEnum extends Ter
     //   System.out.println("BTTR.seekExact termState seg=" + segment + " target=" + target.utf8ToString() + " " + target + " state=" + otherState);
     // }
     assert clearEOF();
-    if (target.compareTo(term) != 0 || !termExists) {
+    if (target.compareTo(term.get()) != 0 || !termExists) {
       assert otherState != null && otherState instanceof BlockTermState;
       currentFrame = staticFrame;
       currentFrame.state.copyFrom(otherState);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java?rev=1617695&r1=1617694&r2=1617695&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java Wed Aug 13 09:36:54 2014
@@ -273,11 +273,9 @@ final class SegmentTermsEnumFrame {
     nextEnt++;
     suffix = suffixesReader.readVInt();
     startBytePos = suffixesReader.getPosition();
-    ste.term.length = prefix + suffix;
-    if (ste.term.bytes.length < ste.term.length) {
-      ste.term.grow(ste.term.length);
-    }
-    suffixesReader.readBytes(ste.term.bytes, prefix, suffix);
+    ste.term.setLength(prefix + suffix);
+    ste.term.grow(ste.term.length());
+    suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
     // A normal term
     ste.termExists = true;
     return false;
@@ -290,11 +288,9 @@ final class SegmentTermsEnumFrame {
     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;
@@ -426,7 +422,7 @@ final class SegmentTermsEnumFrame {
   // 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;
       }
     }
@@ -681,7 +677,7 @@ final class SegmentTermsEnumFrame {
             ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, termLen);
             ste.currentFrame.loadBlock();
             while (ste.currentFrame.next()) {
-              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length);
+              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length());
               ste.currentFrame.loadBlock();
             }
           }
@@ -725,10 +721,8 @@ final class SegmentTermsEnumFrame {
 
   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(termLength);
+    ste.term.grow(termLength);
+    System.arraycopy(suffixBytes, startBytePos, ste.term.bytes(), prefix, suffix);
   }
 }