You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/01/06 03:21:22 UTC

svn commit: r1227936 - in /lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src: java/org/apache/lucene/analysis/kuromoji/trie/ resources/org/apache/lucene/analysis/kuromoji/trie/ tools/java/org/apache/lucene/analysis/kuromoji/util/

Author: uschindler
Date: Fri Jan  6 02:21:21 2012
New Revision: 1227936

URL: http://svn.apache.org/viewvc?rev=1227936&view=rev
Log:
LUCENE-3305: Optimize reallocation strategy in DoubleArrayTrie and TokenInfoDictionaryWriter. For DoubleArrayTrie also record the largest index, so the file does not contain useless tailing 00s (unfortunately the effect is not so big). Also remove useless copying of ByteBuffers in DoubleArrayTrie.write() by keeping a reference to the backing ByteBuffers.

Modified:
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrie.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrie.dat
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrie.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrie.java?rev=1227936&r1=1227935&r2=1227936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrie.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrie.java Fri Jan  6 02:21:21 2012
@@ -38,6 +38,8 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.InputStreamDataInput;
 import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.IOUtils;
 
@@ -55,6 +57,9 @@ public final class DoubleArrayTrie {
   
   private static final int TAIL_OFFSET = 10000000;
   
+  // the underlying raw byte buffers
+  private ByteBuffer baseByteBuffer, checkByteBuffer, tailByteBuffer;
+  
   private IntBuffer baseBuffer;
   
   private IntBuffer checkBuffer;
@@ -62,6 +67,7 @@ public final class DoubleArrayTrie {
   private CharBuffer tailBuffer;
   
   private int tailIndex = TAIL_OFFSET;
+  private int lastIndex = 0;
   
   /**
    * Construct double array trie which is equivalent to input trie
@@ -69,10 +75,10 @@ public final class DoubleArrayTrie {
    * TODO: maybe remove write access
    */
   public DoubleArrayTrie(Trie trie) {
-    baseBuffer = ByteBuffer.allocate(BASE_CHECK_INITILAL_SIZE * 4).asIntBuffer();
+    baseBuffer = (baseByteBuffer = ByteBuffer.allocate(BASE_CHECK_INITILAL_SIZE * 4)).asIntBuffer();
     baseBuffer.put(0, 1);
-    checkBuffer = ByteBuffer.allocate(BASE_CHECK_INITILAL_SIZE * 4).asIntBuffer();
-    tailBuffer = ByteBuffer.allocate(TAIL_INITIAL_SIZE * 2).asCharBuffer();
+    checkBuffer = (checkByteBuffer = ByteBuffer.allocate(BASE_CHECK_INITILAL_SIZE * 4)).asIntBuffer();
+    tailBuffer = (tailByteBuffer = ByteBuffer.allocate(TAIL_INITIAL_SIZE * 2)).asCharBuffer();
     add(-1, 0, trie.getRoot());
   }
   
@@ -85,29 +91,32 @@ public final class DoubleArrayTrie {
     ReadableByteChannel channel = Channels.newChannel(is);
     
     int toRead, read;
-    ByteBuffer tmpBaseBuffer = ByteBuffer.allocateDirect(toRead = baseCheckSize * 4);	// The size is 4 times the baseCheckSize since it is the length of array
-    read = channel.read(tmpBaseBuffer);
+    baseByteBuffer = ByteBuffer.allocateDirect(toRead = baseCheckSize * 4);	// The size is 4 times the baseCheckSize since it is the length of array
+    read = channel.read(baseByteBuffer);
     if (read != toRead) {
       throw new EOFException("Cannot read DoubleArrayTree");
     }
-    tmpBaseBuffer.rewind();
-    baseBuffer = tmpBaseBuffer.asIntBuffer().asReadOnlyBuffer();
+    baseByteBuffer.rewind();
+    baseByteBuffer = baseByteBuffer.asReadOnlyBuffer();
+    baseBuffer = baseByteBuffer.asIntBuffer().asReadOnlyBuffer();
     
-    ByteBuffer tmpCheckBuffer = ByteBuffer.allocateDirect(toRead = baseCheckSize * 4);
-    read = channel.read(tmpCheckBuffer);
+    checkByteBuffer = ByteBuffer.allocateDirect(toRead = baseCheckSize * 4);
+    read = channel.read(checkByteBuffer);
     if (read != toRead) {
       throw new EOFException("Cannot read DoubleArrayTree");
     }
-    tmpCheckBuffer.rewind();
-    checkBuffer = tmpCheckBuffer.asIntBuffer().asReadOnlyBuffer();
+    checkByteBuffer.rewind();
+    checkByteBuffer = checkByteBuffer.asReadOnlyBuffer();
+    checkBuffer = checkByteBuffer.asIntBuffer().asReadOnlyBuffer();
     
-    ByteBuffer tmpTailBuffer = ByteBuffer.allocateDirect(toRead = tailSize * 2);			// The size is 2 times the tailSize since it is the length of array
-    read = channel.read(tmpTailBuffer);
+    tailByteBuffer = ByteBuffer.allocateDirect(toRead = tailSize * 2);			// The size is 2 times the tailSize since it is the length of array
+    read = channel.read(tailByteBuffer);
     if (read != toRead) {
       throw new EOFException("Cannot read DoubleArrayTree");
     }
-    tmpTailBuffer.rewind();
-    tailBuffer = tmpTailBuffer.asCharBuffer().asReadOnlyBuffer();
+    tailByteBuffer.rewind();
+    tailByteBuffer = tailByteBuffer.asReadOnlyBuffer();
+    tailBuffer = tailByteBuffer.asCharBuffer().asReadOnlyBuffer();
   }  
   
   /**
@@ -127,36 +136,30 @@ public final class DoubleArrayTrie {
   }
   
   public void write(OutputStream os) throws IOException {
-    baseBuffer.rewind();
-    checkBuffer.rewind();
-    tailBuffer.rewind();
+    baseByteBuffer.rewind();
+    checkByteBuffer.rewind();
+    tailByteBuffer.rewind();
   
+    final int baseSize = lastIndex + 1,
+      tailSize = tailIndex - TAIL_OFFSET;
+    
     final DataOutput out = new OutputStreamDataOutput(os);
     CodecUtil.writeHeader(out, HEADER, VERSION);
-    out.writeVInt(baseBuffer.capacity());
-    out.writeVInt(tailBuffer.capacity());
+    out.writeVInt(baseSize);
+    out.writeVInt(tailSize);
     final WritableByteChannel channel = Channels.newChannel(os);
     
-    ByteBuffer tmpBuffer = ByteBuffer.allocate(baseBuffer.capacity() * 4);
-    IntBuffer tmpIntBuffer = tmpBuffer.asIntBuffer();
-    tmpIntBuffer.put(baseBuffer);
-    tmpBuffer.rewind();
-    channel.write(tmpBuffer);
-    assert tmpBuffer.remaining() == 0L;
-    
-    tmpBuffer = ByteBuffer.allocate(checkBuffer.capacity() * 4);
-    tmpIntBuffer = tmpBuffer.asIntBuffer();
-    tmpIntBuffer.put(checkBuffer);
-    tmpBuffer.rewind();
-    channel.write(tmpBuffer);
-    assert tmpBuffer.remaining() == 0L;
-    
-    tmpBuffer = ByteBuffer.allocate(tailBuffer.capacity() * 2);
-    CharBuffer tmpCharBuffer = tmpBuffer.asCharBuffer();
-    tmpCharBuffer.put(tailBuffer);
-    tmpBuffer.rewind();
-    channel.write(tmpBuffer);
-    assert tmpBuffer.remaining() == 0L;
+    channel.write((ByteBuffer) baseByteBuffer.limit(baseSize * 4));
+    assert baseByteBuffer.remaining() == 0L;
+    baseByteBuffer.limit(baseByteBuffer.capacity());
+    
+    channel.write((ByteBuffer) checkByteBuffer.limit(baseSize * 4));
+    assert checkByteBuffer.remaining() == 0L;
+    checkByteBuffer.limit(checkByteBuffer.capacity());
+    
+    channel.write((ByteBuffer) tailByteBuffer.limit(tailSize * 2));
+    assert tailByteBuffer.remaining() == 0L;
+    tailByteBuffer.limit(tailByteBuffer.capacity());
   }
   
   /**
@@ -168,6 +171,9 @@ public final class DoubleArrayTrie {
   private void add(int previous, int index, Node node) {
     Node[] children = node.getChildren();	// nodes following current node
     
+    // record the greatest index, so we can shrink the buffer later
+    lastIndex = Math.max(lastIndex, index);
+    
     if(node.getChildren().length > 0 && node.hasSinglePath() && node.getChildren()[0].getKey() != TERMINATING_CHARACTER) {	// If node has only one path, put the rest in tail array
       baseBuffer.put(index, tailIndex);	// current index of tail array
       addToTail(node.children[0]);
@@ -273,12 +279,12 @@ public final class DoubleArrayTrie {
         int nextIndex = index + base + node.getKey();
         
         if(baseBuffer.capacity() <= nextIndex) {
-          int newLength = nextIndex + 1;
-          IntBuffer newBaseBuffer = ByteBuffer.allocate(newLength * 4).asIntBuffer();
+          int newLength = ArrayUtil.oversize(nextIndex + 1, RamUsageEstimator.NUM_BYTES_INT);
+          IntBuffer newBaseBuffer = (baseByteBuffer = ByteBuffer.allocate(newLength * 4)).asIntBuffer();
           baseBuffer.rewind();
           newBaseBuffer.put(baseBuffer);
           baseBuffer = newBaseBuffer;
-          IntBuffer newCheckBuffer = ByteBuffer.allocate(newLength * 4).asIntBuffer();
+          IntBuffer newCheckBuffer = (checkByteBuffer = ByteBuffer.allocate(newLength * 4)).asIntBuffer();
           checkBuffer.rewind();
           newCheckBuffer.put(checkBuffer);
           checkBuffer = newCheckBuffer;
@@ -298,7 +304,10 @@ public final class DoubleArrayTrie {
     }
     
     for(Node node : nodes) {
-      baseBuffer.put(index + base + node.getKey(), node.getKey() == TERMINATING_CHARACTER ? -1 : 1);	// Set -1 if key is terminating character. Set default base value 1 if not.
+      final int i = index + base + node.getKey();
+      // record the greatest index, so we can shrink the buffer later
+      lastIndex = Math.max(lastIndex, i);
+      baseBuffer.put(i, node.getKey() == TERMINATING_CHARACTER ? -1 : 1);	// Set -1 if key is terminating character. Set default base value 1 if not.
     }
     
     return base;
@@ -310,13 +319,15 @@ public final class DoubleArrayTrie {
    */
   private void addToTail(Node node) {
     while(true) {
-      if(tailBuffer.capacity() < tailIndex - TAIL_OFFSET + 1){
-        CharBuffer newTailBuffer = ByteBuffer.allocate((tailBuffer.capacity() + TAIL_INITIAL_SIZE / 100) * 2).asCharBuffer();
+      if(tailBuffer.capacity() <= tailIndex - TAIL_OFFSET){
+        int newLength = ArrayUtil.oversize(tailIndex - TAIL_OFFSET + 1, RamUsageEstimator.NUM_BYTES_CHAR);
+        CharBuffer newTailBuffer = (tailByteBuffer = ByteBuffer.allocate(newLength * 2)).asCharBuffer();
         tailBuffer.rewind();
         newTailBuffer.put(tailBuffer);
         tailBuffer = newTailBuffer;
       }
-      tailBuffer.put(tailIndex++ - TAIL_OFFSET, node.getKey());// set character of current node
+      tailBuffer.put(tailIndex - TAIL_OFFSET, node.getKey());// set character of current node
+      tailIndex++;
       
       if(node.getChildren().length == 0) {	// if it reached the end of input, break.
         break;

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrie.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrie.dat?rev=1227936&r1=1227935&r2=1227936&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java?rev=1227936&r1=1227935&r2=1227936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java Fri Jan  6 02:21:21 2012
@@ -28,6 +28,7 @@ import java.nio.channels.WritableByteCha
 
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.CodecUtil;
 
 import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
@@ -60,9 +61,9 @@ public class TokenInfoDictionaryWriter {
     int featuresSize = features.length()* 2;
     
     // extend buffer if necessary
-    int left = buffer.limit() - buffer.position();
+    int left = buffer.remaining();
     if (8 + featuresSize > left) { // four short and features
-      ByteBuffer newBuffer = ByteBuffer.allocate(buffer.limit() * 2);
+      ByteBuffer newBuffer = ByteBuffer.allocate(ArrayUtil.oversize(buffer.limit() + 1, 1));
       buffer.flip();
       newBuffer.put(buffer);
       buffer = newBuffer;