You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2010/01/21 12:54:51 UTC

svn commit: r901662 - in /lucene/java/trunk: contrib/analyzers/common/src/java/org/tartarus/snowball/ contrib/snowball/ contrib/snowball/src/java/org/tartarus/snowball/ src/java/org/apache/lucene/analysis/ src/java/org/apache/lucene/analysis/tokenattri...

Author: mikemccand
Date: Thu Jan 21 11:54:50 2010
New Revision: 901662

URL: http://svn.apache.org/viewvc?rev=901662&view=rev
Log:
LUCENE-2213: rename ArrayUtil.getNextSize -> oversize; tweak how it picks the next size

Added:
    lucene/java/trunk/contrib/snowball/
      - copied from r900774, lucene/java/trunk/contrib/snowball/
    lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java   (with props)
Modified:
    lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowball/SnowballProgram.java
    lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/SnowballProgram.java
    lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilter.java
    lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java
    lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/TermAttributeImpl.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsConsumer.java
    lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.java
    lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java
    lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java
    lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java
    lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java
    lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/TestTermAttributeImpl.java
    lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java

Modified: lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowball/SnowballProgram.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowball/SnowballProgram.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowball/SnowballProgram.java (original)
+++ lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowball/SnowballProgram.java Thu Jan 21 11:54:50 2010
@@ -35,6 +35,7 @@
 import java.lang.reflect.InvocationTargetException;
 
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * This is the rev 502 of the Snowball SVN trunk,
@@ -432,7 +433,7 @@
 	final int newLength = limit + adjustment;
 	//resize if necessary
 	if (newLength > current.length) {
-	  char newBuffer[] = new char[ArrayUtil.getNextSize(newLength)];
+	  char newBuffer[] = new char[ArrayUtil.oversize(newLength, RamUsageEstimator.NUM_BYTES_CHAR)];
 	  System.arraycopy(current, 0, newBuffer, 0, limit);
 	  current = newBuffer;
 	}

Modified: lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/SnowballProgram.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/SnowballProgram.java?rev=901662&r1=900774&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/SnowballProgram.java (original)
+++ lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/SnowballProgram.java Thu Jan 21 11:54:50 2010
@@ -35,6 +35,7 @@
 import java.lang.reflect.InvocationTargetException;
 
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * This is the rev 502 of the Snowball SVN trunk,
@@ -432,7 +433,7 @@
 	final int newLength = limit + adjustment;
 	//resize if necessary
 	if (newLength > current.length) {
-	  char newBuffer[] = new char[ArrayUtil.getNextSize(newLength)];
+	  char newBuffer[] = new char[ArrayUtil.oversize(newLength, RamUsageEstimator.NUM_BYTES_CHAR)];
 	  System.arraycopy(current, 0, newBuffer, 0, limit);
 	  current = newBuffer;
 	}

Modified: lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilter.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilter.java Thu Jan 21 11:54:50 2010
@@ -1,10 +1,5 @@
 package org.apache.lucene.analysis;
 
-import java.io.IOException;
-
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-import org.apache.lucene.util.ArrayUtil;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -22,6 +17,12 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
+
+import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
 /**
  * This class converts alphabetic, numeric, and symbolic Unicode characters
  * which are not in the first 127 ASCII characters (the "Basic Latin" Unicode
@@ -101,7 +102,7 @@
     // Worst-case length required:
     final int maxSizeNeeded = 4 * length;
     if (output.length < maxSizeNeeded) {
-      output = new char[ArrayUtil.getNextSize(maxSizeNeeded)];
+      output = new char[ArrayUtil.oversize(maxSizeNeeded, RamUsageEstimator.NUM_BYTES_CHAR)];
     }
 
     outputPos = 0;

Modified: lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java Thu Jan 21 11:54:50 2010
@@ -29,6 +29,7 @@
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /** 
   A Token is an occurrence of a term from the text of a field.  It consists of
@@ -347,12 +348,12 @@
   public char[] resizeTermBuffer(int newSize) {
     if (termBuffer == null) {
       // The buffer is always at least MIN_BUFFER_SIZE
-      termBuffer = new char[ArrayUtil.getNextSize(newSize < MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize)]; 
+      termBuffer = new char[ArrayUtil.oversize(newSize < MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize, RamUsageEstimator.NUM_BYTES_CHAR)]; 
     } else {
       if(termBuffer.length < newSize){
         // Not big enough; create a new array with slight
         // over allocation and preserve content
-        final char[] newCharBuffer = new char[ArrayUtil.getNextSize(newSize)];
+        final char[] newCharBuffer = new char[ArrayUtil.oversize(newSize, RamUsageEstimator.NUM_BYTES_CHAR)];
         System.arraycopy(termBuffer, 0, newCharBuffer, 0, termBuffer.length);
         termBuffer = newCharBuffer;
       }
@@ -367,19 +368,19 @@
   private void growTermBuffer(int newSize) {
     if (termBuffer == null) {
       // The buffer is always at least MIN_BUFFER_SIZE    
-      termBuffer = new char[ArrayUtil.getNextSize(newSize < MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize)];   
+      termBuffer = new char[ArrayUtil.oversize(newSize < MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize, RamUsageEstimator.NUM_BYTES_CHAR)];
     } else {
       if(termBuffer.length < newSize){
         // Not big enough; create a new array with slight
         // over allocation:
-        termBuffer = new char[ArrayUtil.getNextSize(newSize)];
+        termBuffer = new char[ArrayUtil.oversize(newSize, RamUsageEstimator.NUM_BYTES_CHAR)];
       }
     } 
   }
   
   private void initTermBuffer() {
     if (termBuffer == null) {
-      termBuffer = new char[ArrayUtil.getNextSize(MIN_BUFFER_SIZE)];
+      termBuffer = new char[ArrayUtil.oversize(MIN_BUFFER_SIZE, RamUsageEstimator.NUM_BYTES_CHAR)];
       termLength = 0;
     }
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/TermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/TermAttributeImpl.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/TermAttributeImpl.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/TermAttributeImpl.java Thu Jan 21 11:54:50 2010
@@ -21,6 +21,7 @@
 
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * The term text of a Token.
@@ -106,12 +107,12 @@
   public char[] resizeTermBuffer(int newSize) {
     if (termBuffer == null) {
       // The buffer is always at least MIN_BUFFER_SIZE
-      termBuffer = new char[ArrayUtil.getNextSize(newSize < MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize)]; 
+      termBuffer = new char[ArrayUtil.oversize(newSize < MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize, RamUsageEstimator.NUM_BYTES_CHAR)]; 
     } else {
       if(termBuffer.length < newSize){
         // Not big enough; create a new array with slight
         // over allocation and preserve content
-        final char[] newCharBuffer = new char[ArrayUtil.getNextSize(newSize)];
+        final char[] newCharBuffer = new char[ArrayUtil.oversize(newSize, RamUsageEstimator.NUM_BYTES_CHAR)];
         System.arraycopy(termBuffer, 0, newCharBuffer, 0, termBuffer.length);
         termBuffer = newCharBuffer;
       }
@@ -127,19 +128,19 @@
   private void growTermBuffer(int newSize) {
     if (termBuffer == null) {
       // The buffer is always at least MIN_BUFFER_SIZE
-      termBuffer = new char[ArrayUtil.getNextSize(newSize < MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize)];   
+      termBuffer = new char[ArrayUtil.oversize(newSize < MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize, RamUsageEstimator.NUM_BYTES_CHAR)];   
     } else {
       if(termBuffer.length < newSize){
         // Not big enough; create a new array with slight
         // over allocation:
-        termBuffer = new char[ArrayUtil.getNextSize(newSize)];
+        termBuffer = new char[ArrayUtil.oversize(newSize, RamUsageEstimator.NUM_BYTES_CHAR)];
       }
     } 
   }
   
   private void initTermBuffer() {
     if (termBuffer == null) {
-      termBuffer = new char[ArrayUtil.getNextSize(MIN_BUFFER_SIZE)];
+      termBuffer = new char[ArrayUtil.oversize(MIN_BUFFER_SIZE, RamUsageEstimator.NUM_BYTES_CHAR)];
       termLength = 0;
     }
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java Thu Jan 21 11:54:50 2010
@@ -25,6 +25,7 @@
 import java.io.IOException;
 
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /** This is just a "splitter" class: it lets you wrap two
  *  DocFieldConsumer instances as a single consumer. */
@@ -117,7 +118,7 @@
         // enough space to recycle all outstanding PerDoc
         // instances
         assert allocCount == 1+docFreeList.length;
-        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
+        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       }
       return new PerDoc();
     } else

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java Thu Jan 21 11:54:50 2010
@@ -24,6 +24,7 @@
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Gathers all Fieldables for a document under the same
@@ -340,7 +341,7 @@
         // enough space to recycle all outstanding PerDoc
         // instances
         assert allocCount == 1+docFreeList.length;
-        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
+        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       }
       return new PerDoc();
     } else

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java Thu Jan 21 11:54:50 2010
@@ -40,6 +40,7 @@
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.ThreadInterruptedException;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * This class accepts multiple added documents and directly
@@ -1503,7 +1504,7 @@
         int gap = doc.docID - nextWriteDocID;
         if (gap >= waiting.length) {
           // Grow queue
-          DocWriter[] newArray = new DocWriter[ArrayUtil.getNextSize(gap)];
+          DocWriter[] newArray = new DocWriter[ArrayUtil.oversize(gap, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
           assert nextWriteLoc >= 0;
           System.arraycopy(waiting, nextWriteLoc, newArray, 0, waiting.length-nextWriteLoc);
           System.arraycopy(waiting, 0, newArray, waiting.length-nextWriteLoc, nextWriteLoc);

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsConsumer.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsConsumer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsConsumer.java Thu Jan 21 11:54:50 2010
@@ -20,6 +20,7 @@
 import java.io.IOException;
 
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * NOTE: this API is experimental and will likely change
@@ -35,7 +36,7 @@
   FormatPostingsDocsConsumer addTerm(String text) throws IOException {
     final int len = text.length();
     if (termBuffer == null || termBuffer.length < 1+len)
-      termBuffer = new char[ArrayUtil.getNextSize(1+len)];
+      termBuffer = new char[ArrayUtil.oversize(1+len, RamUsageEstimator.NUM_BYTES_CHAR)];
     text.getChars(0, len, termBuffer, 0);
     termBuffer[len] = 0xffff;
     return addTerm(termBuffer, 0);

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.java Thu Jan 21 11:54:50 2010
@@ -63,11 +63,13 @@
   
   @Override
   void finish() {
-    assert docIDs.length == norms.length;
     if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
       if (docIDs.length <= upto) {
         assert docIDs.length == upto;
         docIDs = ArrayUtil.grow(docIDs, 1+upto);
+      }
+      if (norms.length <= upto) {
+        assert norms.length == upto;
         norms = ArrayUtil.grow(norms, 1+upto);
       }
       final float norm = docState.similarity.computeNorm(fieldInfo.name, fieldState);

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.java Thu Jan 21 11:54:50 2010
@@ -20,6 +20,7 @@
 import java.io.IOException;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /** This is a DocFieldConsumer that writes stored fields. */
 final class StoredFieldsWriter {
@@ -108,7 +109,7 @@
         // enough space to recycle all outstanding PerDoc
         // instances
         assert allocCount == 1+docFreeList.length;
-        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
+        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       }
       return new PerDoc();
     } else

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Thu Jan 21 11:54:50 2010
@@ -20,6 +20,7 @@
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -117,7 +118,7 @@
         // enough space to recycle all outstanding PerDoc
         // instances
         assert allocCount == 1+docFreeList.length;
-        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
+        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       }
       return new PerDoc();
     } else
@@ -266,6 +267,8 @@
     void addField(final int fieldNumber) {
       if (numVectorFields == fieldNumbers.length) {
         fieldNumbers = ArrayUtil.grow(fieldNumbers);
+      }
+      if (numVectorFields == fieldPointers.length) {
         fieldPointers = ArrayUtil.grow(fieldPointers);
       }
       fieldNumbers[numVectorFields] = fieldNumber;

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java Thu Jan 21 11:54:50 2010
@@ -26,6 +26,7 @@
 import java.io.IOException;
 
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /** This class implements {@link InvertedDocConsumer}, which
  *  is passed each token produced by the analyzer on each
@@ -89,7 +90,7 @@
 
     assert postingsFreeCount == postingsAllocCount: Thread.currentThread().getName() + ": postingsFreeCount=" + postingsFreeCount + " postingsAllocCount=" + postingsAllocCount + " consumer=" + consumer;
 
-    final int newSize = ArrayUtil.getShrinkSize(postingsFreeList.length, postingsAllocCount);
+    final int newSize = ArrayUtil.getShrinkSize(postingsFreeList.length, postingsAllocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
     if (newSize != postingsFreeList.length) {
       RawPostingList[] newArray = new RawPostingList[newSize];
       System.arraycopy(postingsFreeList, 0, newArray, 0, postingsFreeCount);
@@ -222,7 +223,7 @@
       if (newPostingsAllocCount > postingsFreeList.length)
         // Pre-allocate the postingsFreeList so it's large
         // enough to hold all postings we've given out
-        postingsFreeList = new RawPostingList[ArrayUtil.getNextSize(newPostingsAllocCount)];
+        postingsFreeList = new RawPostingList[ArrayUtil.oversize(newPostingsAllocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
     }
 
     postingsFreeCount -= numToCopy;

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java Thu Jan 21 11:54:50 2010
@@ -7,9 +7,9 @@
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -122,20 +122,95 @@
  END APACHE HARMONY CODE
   */
 
+  /** Returns an array size >= minTargetSize, generally
+   *  over-allocating exponentially to achieve amortized
+   *  linear-time cost as the array grows.
+   *
+   *  NOTE: this was originally borrowed from Python 2.4.2
+   *  listobject.c sources (attribution in LICENSE.txt), but
+   *  has now been substantially changed based on
+   *  discussions from java-dev thread with subject "Dynamic
+   *  array reallocation algorithms", started on Jan 12
+   *  2010.
+   *
+   * @param minTargetSize Minimum required value to be returned.
+   * @param bytesPerElement Bytes used by each element of
+   * the array.  See constants in {@link RamUsageEstimator}.
+   *
+   * @lucene.internal
+   */
+
+  public static int oversize(int minTargetSize, int bytesPerElement) {
+
+    if (minTargetSize < 0) {
+      // catch usage that accidentally overflows int
+      throw new IllegalArgumentException("invalid array size " + minTargetSize);
+    }
+
+    if (minTargetSize == 0) {
+      // wait until at least one element is requested
+      return 0;
+    }
 
-  public static int getNextSize(int targetSize) {
-    /* This over-allocates proportional to the list size, making room
-     * for additional growth.  The over-allocation is mild, but is
-     * enough to give linear-time amortized behavior over a long
-     * sequence of appends() in the presence of a poorly-performing
-     * system realloc().
-     * The growth pattern is:  0, 4, 8, 16, 25, 35, 46, 58, 72, 88, ...
-     */
-    return (targetSize >> 3) + (targetSize < 9 ? 3 : 6) + targetSize;
+    // asymptotic exponential growth by 1/8th, favors
+    // spending a bit more CPU to not tye up too much wasted
+    // RAM:
+    int extra = minTargetSize >> 3;
+
+    if (extra < 3) {
+      // for very small arrays, where constant overhead of
+      // realloc is presumably relatively high, we grow
+      // faster
+      extra = 3;
+    }
+
+    int newSize = minTargetSize + extra;
+
+    // add 7 to allow for worst case byte alignment addition below:
+    if (newSize+7 < 0) {
+      // int overflowed -- return max allowed array size
+      return Integer.MAX_VALUE;
+    }
+
+    if (Constants.JRE_IS_64BIT) {
+      // round up to 8 byte alignment in 64bit env
+      switch(bytesPerElement) {
+      case 4:
+        // round up to multiple of 2
+        return (newSize + 1) & 0x7ffffffe;
+      case 2:
+        // round up to multiple of 4
+        return (newSize + 3) & 0x7ffffffc;
+      case 1:
+        // round up to multiple of 8
+        return (newSize + 7) & 0x7ffffff8;
+      case 8:
+        // no rounding
+      default:
+        // odd (invalid?) size
+        return newSize;
+      }
+    } else {
+      // round up to 4 byte alignment in 64bit env
+      switch(bytesPerElement) {
+      case 2:
+        // round up to multiple of 2
+        return (newSize + 1) & 0x7ffffffe;
+      case 1:
+        // round up to multiple of 4
+        return (newSize + 3) & 0x7ffffffc;
+      case 4:
+      case 8:
+        // no rounding
+      default:
+        // odd (invalid?) size
+        return newSize;
+      }
+    }
   }
 
-  public static int getShrinkSize(int currentSize, int targetSize) {
-    final int newSize = getNextSize(targetSize);
+  public static int getShrinkSize(int currentSize, int targetSize, int bytesPerElement) {
+    final int newSize = oversize(targetSize, bytesPerElement);
     // Only reallocate if we are "substantially" smaller.
     // This saves us from "running hot" (constantly making a
     // bit bigger then a bit smaller, over and over):
@@ -147,7 +222,7 @@
 
   public static int[] grow(int[] array, int minSize) {
     if (array.length < minSize) {
-      int[] newArray = new int[getNextSize(minSize)];
+      int[] newArray = new int[oversize(minSize, RamUsageEstimator.NUM_BYTES_INT)];
       System.arraycopy(array, 0, newArray, 0, array.length);
       return newArray;
     } else
@@ -159,7 +234,7 @@
   }
 
   public static int[] shrink(int[] array, int targetSize) {
-    final int newSize = getShrinkSize(array.length, targetSize);
+    final int newSize = getShrinkSize(array.length, targetSize, RamUsageEstimator.NUM_BYTES_INT);
     if (newSize != array.length) {
       int[] newArray = new int[newSize];
       System.arraycopy(array, 0, newArray, 0, newSize);
@@ -170,7 +245,7 @@
 
   public static long[] grow(long[] array, int minSize) {
     if (array.length < minSize) {
-      long[] newArray = new long[getNextSize(minSize)];
+      long[] newArray = new long[oversize(minSize, RamUsageEstimator.NUM_BYTES_LONG)];
       System.arraycopy(array, 0, newArray, 0, array.length);
       return newArray;
     } else
@@ -182,7 +257,7 @@
   }
 
   public static long[] shrink(long[] array, int targetSize) {
-    final int newSize = getShrinkSize(array.length, targetSize);
+    final int newSize = getShrinkSize(array.length, targetSize, RamUsageEstimator.NUM_BYTES_LONG);
     if (newSize != array.length) {
       long[] newArray = new long[newSize];
       System.arraycopy(array, 0, newArray, 0, newSize);
@@ -193,7 +268,7 @@
 
   public static byte[] grow(byte[] array, int minSize) {
     if (array.length < minSize) {
-      byte[] newArray = new byte[getNextSize(minSize)];
+      byte[] newArray = new byte[oversize(minSize, 1)];
       System.arraycopy(array, 0, newArray, 0, array.length);
       return newArray;
     } else
@@ -205,7 +280,7 @@
   }
 
   public static byte[] shrink(byte[] array, int targetSize) {
-    final int newSize = getShrinkSize(array.length, targetSize);
+    final int newSize = getShrinkSize(array.length, targetSize, 1);
     if (newSize != array.length) {
       byte[] newArray = new byte[newSize];
       System.arraycopy(array, 0, newArray, 0, newSize);

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java Thu Jan 21 11:54:50 2010
@@ -43,6 +43,14 @@
   private int arraySize;
   private int classSize;
 
+  public final static int NUM_BYTES_OBJECT_REF = Constants.JRE_IS_64BIT ? 8 : 4;
+  public final static int NUM_BYTES_CHAR = 2;
+  public final static int NUM_BYTES_SHORT = 2;
+  public final static int NUM_BYTES_INT = 4;
+  public final static int NUM_BYTES_LONG = 8;
+  public final static int NUM_BYTES_FLOAT = 4;
+  public final static int NUM_BYTES_DOUBLE = 8;
+
   private boolean checkInterned;
 
   /**

Modified: lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java Thu Jan 21 11:54:50 2010
@@ -85,7 +85,6 @@
       buf.append(buf.toString());
     }
     assertEquals(1048576, t.termLength());
-    assertEquals(1179654, t.termBuffer().length);
 
     // now as a string, first variant
     t = new Token();
@@ -99,7 +98,6 @@
       buf.append(content);
     }
     assertEquals(1048576, t.termLength());
-    assertEquals(1179654, t.termBuffer().length);
 
     // now as a string, second variant
     t = new Token();
@@ -113,7 +111,6 @@
       buf.append(content);
     }
     assertEquals(1048576, t.termLength());
-    assertEquals(1179654, t.termBuffer().length);
 
     // Test for slow growth to a long term
     t = new Token();
@@ -127,7 +124,6 @@
       buf.append("a");
     }
     assertEquals(20000, t.termLength());
-    assertEquals(20167, t.termBuffer().length);
 
     // Test for slow growth to a long term
     t = new Token();
@@ -141,7 +137,6 @@
       buf.append("a");
     }
     assertEquals(20000, t.termLength());
-    assertEquals(20167, t.termBuffer().length);
   }
 
   public void testToString() throws Exception {

Modified: lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/TestTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/TestTermAttributeImpl.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/TestTermAttributeImpl.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/TestTermAttributeImpl.java Thu Jan 21 11:54:50 2010
@@ -49,7 +49,6 @@
       buf.append(buf.toString());
     }
     assertEquals(1048576, t.termLength());
-    assertEquals(1179654, t.termBuffer().length);
 
     // now as a string, first variant
     t = new TermAttributeImpl();
@@ -63,7 +62,6 @@
       buf.append(content);
     }
     assertEquals(1048576, t.termLength());
-    assertEquals(1179654, t.termBuffer().length);
 
     // now as a string, second variant
     t = new TermAttributeImpl();
@@ -77,7 +75,6 @@
       buf.append(content);
     }
     assertEquals(1048576, t.termLength());
-    assertEquals(1179654, t.termBuffer().length);
 
     // Test for slow growth to a long term
     t = new TermAttributeImpl();
@@ -91,7 +88,6 @@
       buf.append("a");
     }
     assertEquals(20000, t.termLength());
-    assertEquals(20167, t.termBuffer().length);
 
     // Test for slow growth to a long term
     t = new TermAttributeImpl();
@@ -105,7 +101,6 @@
       buf.append("a");
     }
     assertEquals(20000, t.termLength());
-    assertEquals(20167, t.termBuffer().length);
   }
 
   public void testToString() throws Exception {

Added: lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java?rev=901662&view=auto
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java (added)
+++ lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java Thu Jan 21 11:54:50 2010
@@ -0,0 +1,59 @@
+package org.apache.lucene.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Random;
+
+public class TestArrayUtil extends LuceneTestCase {
+
+  // Ensure ArrayUtil.getNextSize gives linear amortized cost of realloc/copy
+  public void testGrowth() {
+    int currentSize = 0;
+    long copyCost = 0;
+
+    // Make sure ArrayUtil hits Integer.MAX_VALUE, if we insist:
+    while(currentSize != Integer.MAX_VALUE) {
+      int nextSize = ArrayUtil.oversize(1+currentSize, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
+      assertTrue(nextSize > currentSize);
+      if (currentSize > 0) {
+        copyCost += currentSize;
+        double copyCostPerElement = ((double) copyCost)/currentSize;
+        assertTrue("cost " + copyCostPerElement, copyCostPerElement < 10.0);
+      }
+      currentSize = nextSize;
+    }
+  }
+
+  public void testMaxSize() {
+    // intentionally pass invalid elemSizes:
+    for(int elemSize=0;elemSize<10;elemSize++) {
+      assertEquals(Integer.MAX_VALUE, ArrayUtil.oversize(Integer.MAX_VALUE, elemSize));
+      assertEquals(Integer.MAX_VALUE, ArrayUtil.oversize(Integer.MAX_VALUE-1, elemSize));
+    }
+  }
+
+  public void testInvalidElementSizes() {
+    final Random r = newRandom();
+    for(int iter=0;iter<10000;iter++) {
+      final int minTargetSize = r.nextInt(Integer.MAX_VALUE);
+      final int elemSize = r.nextInt(11);
+      final int v = ArrayUtil.oversize(minTargetSize, elemSize);
+      assertTrue(v >= minTargetSize);
+    }
+  }
+}

Propchange: lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java?rev=901662&r1=901661&r2=901662&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java Thu Jan 21 11:54:50 2010
@@ -164,14 +164,14 @@
       int encodedLen1 = IndexableBinaryStringTools.getEncodedLength(
           originalArray1, 0, numBytes1);
       if (encodedLen1 > encoded1.length)
-        encoded1 = new char[ArrayUtil.getNextSize(encodedLen1)];
+        encoded1 = new char[ArrayUtil.oversize(encodedLen1, RamUsageEstimator.NUM_BYTES_CHAR)];
       IndexableBinaryStringTools.encode(originalArray1, 0, numBytes1, encoded1,
           0, encodedLen1);
 
       int encodedLen2 = IndexableBinaryStringTools.getEncodedLength(original2,
           0, numBytes2);
       if (encodedLen2 > encoded2.length)
-        encoded2 = new char[ArrayUtil.getNextSize(encodedLen2)];
+        encoded2 = new char[ArrayUtil.oversize(encodedLen2, RamUsageEstimator.NUM_BYTES_CHAR)];
       IndexableBinaryStringTools.encode(original2, 0, numBytes2, encoded2, 0,
           encodedLen2);
 
@@ -308,7 +308,7 @@
       int encodedLen = IndexableBinaryStringTools.getEncodedLength(binary, 0,
           numBytes);
       if (encoded.length < encodedLen)
-        encoded = new char[ArrayUtil.getNextSize(encodedLen)];
+        encoded = new char[ArrayUtil.oversize(encodedLen, RamUsageEstimator.NUM_BYTES_CHAR)];
       IndexableBinaryStringTools.encode(binary, 0, numBytes, encoded, 0,
           encodedLen);
 



RE: svn commit: r901662 - in /lucene/java/trunk: contrib/analyzers/common/src/java/org/tartarus/snowball/ contrib/snowball/ contrib/snowball/src/java/org/tartarus/snowball/ src/java/org/apache/lucene/analysis/ src/java/org/apache/lucene/analysis/tokenattr

Posted by Uwe Schindler <uw...@thetaphi.de>.
Somehow we have now both ArrayUtilTest and TestArrayUtil? I think they should be merged and TestArrayUtil as name preferred (as all other tests are named like this).

Should I provide a patch?

-----
Uwe Schindler
H.-H.-Meier-Allee 63, D-28213 Bremen
http://www.thetaphi.de
eMail: uwe@thetaphi.de


> -----Original Message-----
> From: mikemccand@apache.org [mailto:mikemccand@apache.org]
> Sent: Thursday, January 21, 2010 12:55 PM
> To: java-commits@lucene.apache.org
> Subject: svn commit: r901662 - in /lucene/java/trunk:
> contrib/analyzers/common/src/java/org/tartarus/snowball/
> contrib/snowball/ contrib/snowball/src/java/org/tartarus/snowball/
> src/java/org/apache/lucene/analysis/
> src/java/org/apache/lucene/analysis/tokenattri...
> 
> Author: mikemccand
> Date: Thu Jan 21 11:54:50 2010
> New Revision: 901662
> 
> URL: http://svn.apache.org/viewvc?rev=901662&view=rev
> Log:
> LUCENE-2213: rename ArrayUtil.getNextSize -> oversize; tweak how it
> picks the next size
> 
> Added:
>     lucene/java/trunk/contrib/snowball/
>       - copied from r900774, lucene/java/trunk/contrib/snowball/
> 
> lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java
> (with props)
> Modified:
> 
> lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowba
> ll/SnowballProgram.java
> 
> lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/Snowb
> allProgram.java
> 
> lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilte
> r.java
>     lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java
> 
> lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/T
> ermAttributeImpl.java
> 
> lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.ja
> va
> 
> lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPer
> Thread.java
> 
> lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
> 
> lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsC
> onsumer.java
> 
> lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.
> java
> 
> lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.j
> ava
> 
> lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWrit
> er.java
>     lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java
>     lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java
> 
> lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.jav
> a
> 
> lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java
> 
> lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/T
> estTermAttributeImpl.java
> 
> lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinarySt
> ringTools.java
> 
> Modified:
> lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowba
> ll/SnowballProgram.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/common
> /src/java/org/tartarus/snowball/SnowballProgram.java?rev=901662&r1=9016
> 61&r2=901662&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowba
> ll/SnowballProgram.java (original)
> +++
> lucene/java/trunk/contrib/analyzers/common/src/java/org/tartarus/snowba
> ll/SnowballProgram.java Thu Jan 21 11:54:50 2010
> @@ -35,6 +35,7 @@
>  import java.lang.reflect.InvocationTargetException;
> 
>  import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /**
>   * This is the rev 502 of the Snowball SVN trunk,
> @@ -432,7 +433,7 @@
>  	final int newLength = limit + adjustment;
>  	//resize if necessary
>  	if (newLength > current.length) {
> -	  char newBuffer[] = new char[ArrayUtil.getNextSize(newLength)];
> +	  char newBuffer[] = new char[ArrayUtil.oversize(newLength,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>  	  System.arraycopy(current, 0, newBuffer, 0, limit);
>  	  current = newBuffer;
>  	}
> 
> Modified:
> lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/Snowb
> allProgram.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/contrib/snowball/src/jav
> a/org/tartarus/snowball/SnowballProgram.java?rev=901662&r1=900774&r2=90
> 1662&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/Snowb
> allProgram.java (original)
> +++
> lucene/java/trunk/contrib/snowball/src/java/org/tartarus/snowball/Snowb
> allProgram.java Thu Jan 21 11:54:50 2010
> @@ -35,6 +35,7 @@
>  import java.lang.reflect.InvocationTargetException;
> 
>  import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /**
>   * This is the rev 502 of the Snowball SVN trunk,
> @@ -432,7 +433,7 @@
>  	final int newLength = limit + adjustment;
>  	//resize if necessary
>  	if (newLength > current.length) {
> -	  char newBuffer[] = new char[ArrayUtil.getNextSize(newLength)];
> +	  char newBuffer[] = new char[ArrayUtil.oversize(newLength,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>  	  System.arraycopy(current, 0, newBuffer, 0, limit);
>  	  current = newBuffer;
>  	}
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilte
> r.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/analysis/ASCIIFoldingFilter.java?rev=901662&r1=901661&r2=901662&view
> =diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilte
> r.java (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/analysis/ASCIIFoldingFilte
> r.java Thu Jan 21 11:54:50 2010
> @@ -1,10 +1,5 @@
>  package org.apache.lucene.analysis;
> 
> -import java.io.IOException;
> -
> -import org.apache.lucene.analysis.tokenattributes.TermAttribute;
> -import org.apache.lucene.util.ArrayUtil;
> -
>  /**
>   * Licensed to the Apache Software Foundation (ASF) under one or more
>   * contributor license agreements.  See the NOTICE file distributed
> with
> @@ -22,6 +17,12 @@
>   * limitations under the License.
>   */
> 
> +import java.io.IOException;
> +
> +import org.apache.lucene.analysis.tokenattributes.TermAttribute;
> +import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> +
>  /**
>   * This class converts alphabetic, numeric, and symbolic Unicode
> characters
>   * which are not in the first 127 ASCII characters (the "Basic Latin"
> Unicode
> @@ -101,7 +102,7 @@
>      // Worst-case length required:
>      final int maxSizeNeeded = 4 * length;
>      if (output.length < maxSizeNeeded) {
> -      output = new char[ArrayUtil.getNextSize(maxSizeNeeded)];
> +      output = new char[ArrayUtil.oversize(maxSizeNeeded,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>      }
> 
>      outputPos = 0;
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/analysis/Token.java?rev=901662&r1=901661&r2=901662&view=diff
> =======================================================================
> =======
> --- lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java
> (original)
> +++ lucene/java/trunk/src/java/org/apache/lucene/analysis/Token.java
> Thu Jan 21 11:54:50 2010
> @@ -29,6 +29,7 @@
>  import org.apache.lucene.util.Attribute;
>  import org.apache.lucene.util.AttributeSource;
>  import org.apache.lucene.util.AttributeImpl;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /**
>    A Token is an occurrence of a term from the text of a field.  It
> consists of
> @@ -347,12 +348,12 @@
>    public char[] resizeTermBuffer(int newSize) {
>      if (termBuffer == null) {
>        // The buffer is always at least MIN_BUFFER_SIZE
> -      termBuffer = new char[ArrayUtil.getNextSize(newSize <
> MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize)];
> +      termBuffer = new char[ArrayUtil.oversize(newSize <
> MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>      } else {
>        if(termBuffer.length < newSize){
>          // Not big enough; create a new array with slight
>          // over allocation and preserve content
> -        final char[] newCharBuffer = new
> char[ArrayUtil.getNextSize(newSize)];
> +        final char[] newCharBuffer = new
> char[ArrayUtil.oversize(newSize, RamUsageEstimator.NUM_BYTES_CHAR)];
>          System.arraycopy(termBuffer, 0, newCharBuffer, 0,
> termBuffer.length);
>          termBuffer = newCharBuffer;
>        }
> @@ -367,19 +368,19 @@
>    private void growTermBuffer(int newSize) {
>      if (termBuffer == null) {
>        // The buffer is always at least MIN_BUFFER_SIZE
> -      termBuffer = new char[ArrayUtil.getNextSize(newSize <
> MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize)];
> +      termBuffer = new char[ArrayUtil.oversize(newSize <
> MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>      } else {
>        if(termBuffer.length < newSize){
>          // Not big enough; create a new array with slight
>          // over allocation:
> -        termBuffer = new char[ArrayUtil.getNextSize(newSize)];
> +        termBuffer = new char[ArrayUtil.oversize(newSize,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>        }
>      }
>    }
> 
>    private void initTermBuffer() {
>      if (termBuffer == null) {
> -      termBuffer = new char[ArrayUtil.getNextSize(MIN_BUFFER_SIZE)];
> +      termBuffer = new char[ArrayUtil.oversize(MIN_BUFFER_SIZE,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>        termLength = 0;
>      }
>    }
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/T
> ermAttributeImpl.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/analysis/tokenattributes/TermAttributeImpl.java?rev=901662&r1=901661
> &r2=901662&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/T
> ermAttributeImpl.java (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/analysis/tokenattributes/T
> ermAttributeImpl.java Thu Jan 21 11:54:50 2010
> @@ -21,6 +21,7 @@
> 
>  import org.apache.lucene.util.ArrayUtil;
>  import org.apache.lucene.util.AttributeImpl;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /**
>   * The term text of a Token.
> @@ -106,12 +107,12 @@
>    public char[] resizeTermBuffer(int newSize) {
>      if (termBuffer == null) {
>        // The buffer is always at least MIN_BUFFER_SIZE
> -      termBuffer = new char[ArrayUtil.getNextSize(newSize <
> MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize)];
> +      termBuffer = new char[ArrayUtil.oversize(newSize <
> MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>      } else {
>        if(termBuffer.length < newSize){
>          // Not big enough; create a new array with slight
>          // over allocation and preserve content
> -        final char[] newCharBuffer = new
> char[ArrayUtil.getNextSize(newSize)];
> +        final char[] newCharBuffer = new
> char[ArrayUtil.oversize(newSize, RamUsageEstimator.NUM_BYTES_CHAR)];
>          System.arraycopy(termBuffer, 0, newCharBuffer, 0,
> termBuffer.length);
>          termBuffer = newCharBuffer;
>        }
> @@ -127,19 +128,19 @@
>    private void growTermBuffer(int newSize) {
>      if (termBuffer == null) {
>        // The buffer is always at least MIN_BUFFER_SIZE
> -      termBuffer = new char[ArrayUtil.getNextSize(newSize <
> MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize)];
> +      termBuffer = new char[ArrayUtil.oversize(newSize <
> MIN_BUFFER_SIZE ? MIN_BUFFER_SIZE : newSize,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>      } else {
>        if(termBuffer.length < newSize){
>          // Not big enough; create a new array with slight
>          // over allocation:
> -        termBuffer = new char[ArrayUtil.getNextSize(newSize)];
> +        termBuffer = new char[ArrayUtil.oversize(newSize,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>        }
>      }
>    }
> 
>    private void initTermBuffer() {
>      if (termBuffer == null) {
> -      termBuffer = new char[ArrayUtil.getNextSize(MIN_BUFFER_SIZE)];
> +      termBuffer = new char[ArrayUtil.oversize(MIN_BUFFER_SIZE,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>        termLength = 0;
>      }
>    }
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.ja
> va
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/index/DocFieldConsumers.java?rev=901662&r1=901661&r2=901662&view=dif
> f
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.ja
> va (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.ja
> va Thu Jan 21 11:54:50 2010
> @@ -25,6 +25,7 @@
>  import java.io.IOException;
> 
>  import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /** This is just a "splitter" class: it lets you wrap two
>   *  DocFieldConsumer instances as a single consumer. */
> @@ -117,7 +118,7 @@
>          // enough space to recycle all outstanding PerDoc
>          // instances
>          assert allocCount == 1+docFreeList.length;
> -        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
> +        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount,
> RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
>        }
>        return new PerDoc();
>      } else
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPer
> Thread.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/index/DocFieldProcessorPerThread.java?rev=901662&r1=901661&r2=901662
> &view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPer
> Thread.java (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPer
> Thread.java Thu Jan 21 11:54:50 2010
> @@ -24,6 +24,7 @@
>  import org.apache.lucene.document.Document;
>  import org.apache.lucene.document.Fieldable;
>  import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /**
>   * Gathers all Fieldables for a document under the same
> @@ -340,7 +341,7 @@
>          // enough space to recycle all outstanding PerDoc
>          // instances
>          assert allocCount == 1+docFreeList.length;
> -        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
> +        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount,
> RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
>        }
>        return new PerDoc();
>      } else
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/index/DocumentsWriter.java?rev=901662&r1=901661&r2=901662&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
> (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
> Thu Jan 21 11:54:50 2010
> @@ -40,6 +40,7 @@
>  import org.apache.lucene.util.ArrayUtil;
>  import org.apache.lucene.util.Constants;
>  import org.apache.lucene.util.ThreadInterruptedException;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /**
>   * This class accepts multiple added documents and directly
> @@ -1503,7 +1504,7 @@
>          int gap = doc.docID - nextWriteDocID;
>          if (gap >= waiting.length) {
>            // Grow queue
> -          DocWriter[] newArray = new
> DocWriter[ArrayUtil.getNextSize(gap)];
> +          DocWriter[] newArray = new DocWriter[ArrayUtil.oversize(gap,
> RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
>            assert nextWriteLoc >= 0;
>            System.arraycopy(waiting, nextWriteLoc, newArray, 0,
> waiting.length-nextWriteLoc);
>            System.arraycopy(waiting, 0, newArray, waiting.length-
> nextWriteLoc, nextWriteLoc);
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsC
> onsumer.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/index/FormatPostingsTermsConsumer.java?rev=901662&r1=901661&r2=90166
> 2&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsC
> onsumer.java (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/index/FormatPostingsTermsC
> onsumer.java Thu Jan 21 11:54:50 2010
> @@ -20,6 +20,7 @@
>  import java.io.IOException;
> 
>  import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /**
>   * NOTE: this API is experimental and will likely change
> @@ -35,7 +36,7 @@
>    FormatPostingsDocsConsumer addTerm(String text) throws IOException {
>      final int len = text.length();
>      if (termBuffer == null || termBuffer.length < 1+len)
> -      termBuffer = new char[ArrayUtil.getNextSize(1+len)];
> +      termBuffer = new char[ArrayUtil.oversize(1+len,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>      text.getChars(0, len, termBuffer, 0);
>      termBuffer[len] = 0xffff;
>      return addTerm(termBuffer, 0);
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.
> java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/index/NormsWriterPerField.java?rev=901662&r1=901661&r2=901662&view=d
> iff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.
> java (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.
> java Thu Jan 21 11:54:50 2010
> @@ -63,11 +63,13 @@
> 
>    @Override
>    void finish() {
> -    assert docIDs.length == norms.length;
>      if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
>        if (docIDs.length <= upto) {
>          assert docIDs.length == upto;
>          docIDs = ArrayUtil.grow(docIDs, 1+upto);
> +      }
> +      if (norms.length <= upto) {
> +        assert norms.length == upto;
>          norms = ArrayUtil.grow(norms, 1+upto);
>        }
>        final float norm =
> docState.similarity.computeNorm(fieldInfo.name, fieldState);
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.j
> ava
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/index/StoredFieldsWriter.java?rev=901662&r1=901661&r2=901662&view=di
> ff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.j
> ava (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.j
> ava Thu Jan 21 11:54:50 2010
> @@ -20,6 +20,7 @@
>  import java.io.IOException;
>  import org.apache.lucene.store.RAMOutputStream;
>  import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /** This is a DocFieldConsumer that writes stored fields. */
>  final class StoredFieldsWriter {
> @@ -108,7 +109,7 @@
>          // enough space to recycle all outstanding PerDoc
>          // instances
>          assert allocCount == 1+docFreeList.length;
> -        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
> +        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount,
> RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
>        }
>        return new PerDoc();
>      } else
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWrit
> er.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/index/TermVectorsTermsWriter.java?rev=901662&r1=901661&r2=901662&vie
> w=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWrit
> er.java (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWrit
> er.java Thu Jan 21 11:54:50 2010
> @@ -20,6 +20,7 @@
>  import org.apache.lucene.store.IndexOutput;
>  import org.apache.lucene.store.RAMOutputStream;
>  import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  import java.io.IOException;
>  import java.util.Collection;
> @@ -117,7 +118,7 @@
>          // enough space to recycle all outstanding PerDoc
>          // instances
>          assert allocCount == 1+docFreeList.length;
> -        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
> +        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount,
> RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
>        }
>        return new PerDoc();
>      } else
> @@ -266,6 +267,8 @@
>      void addField(final int fieldNumber) {
>        if (numVectorFields == fieldNumbers.length) {
>          fieldNumbers = ArrayUtil.grow(fieldNumbers);
> +      }
> +      if (numVectorFields == fieldPointers.length) {
>          fieldPointers = ArrayUtil.grow(fieldPointers);
>        }
>        fieldNumbers[numVectorFields] = fieldNumber;
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/index/TermsHash.java?rev=901662&r1=901661&r2=901662&view=diff
> =======================================================================
> =======
> --- lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java
> (original)
> +++ lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java
> Thu Jan 21 11:54:50 2010
> @@ -26,6 +26,7 @@
>  import java.io.IOException;
> 
>  import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.RamUsageEstimator;
> 
>  /** This class implements {@link InvertedDocConsumer}, which
>   *  is passed each token produced by the analyzer on each
> @@ -89,7 +90,7 @@
> 
>      assert postingsFreeCount == postingsAllocCount:
> Thread.currentThread().getName() + ": postingsFreeCount=" +
> postingsFreeCount + " postingsAllocCount=" + postingsAllocCount + "
> consumer=" + consumer;
> 
> -    final int newSize =
> ArrayUtil.getShrinkSize(postingsFreeList.length, postingsAllocCount);
> +    final int newSize =
> ArrayUtil.getShrinkSize(postingsFreeList.length, postingsAllocCount,
> RamUsageEstimator.NUM_BYTES_OBJECT_REF);
>      if (newSize != postingsFreeList.length) {
>        RawPostingList[] newArray = new RawPostingList[newSize];
>        System.arraycopy(postingsFreeList, 0, newArray, 0,
> postingsFreeCount);
> @@ -222,7 +223,7 @@
>        if (newPostingsAllocCount > postingsFreeList.length)
>          // Pre-allocate the postingsFreeList so it's large
>          // enough to hold all postings we've given out
> -        postingsFreeList = new
> RawPostingList[ArrayUtil.getNextSize(newPostingsAllocCount)];
> +        postingsFreeList = new
> RawPostingList[ArrayUtil.oversize(newPostingsAllocCount,
> RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
>      }
> 
>      postingsFreeCount -= numToCopy;
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/util/ArrayUtil.java?rev=901662&r1=901661&r2=901662&view=diff
> =======================================================================
> =======
> --- lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java
> (original)
> +++ lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java
> Thu Jan 21 11:54:50 2010
> @@ -7,9 +7,9 @@
>   * The ASF licenses this file to You under the Apache License, Version
> 2.0
>   * (the "License"); you may not use this file except in compliance
> with
>   * the License.  You may obtain a copy of the License at
> - * <p/>
> - * http://www.apache.org/licenses/LICENSE-2.0
> - * <p/>
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
>   * Unless required by applicable law or agreed to in writing, software
>   * distributed under the License is distributed on an "AS IS" BASIS,
>   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> @@ -122,20 +122,95 @@
>   END APACHE HARMONY CODE
>    */
> 
> +  /** Returns an array size >= minTargetSize, generally
> +   *  over-allocating exponentially to achieve amortized
> +   *  linear-time cost as the array grows.
> +   *
> +   *  NOTE: this was originally borrowed from Python 2.4.2
> +   *  listobject.c sources (attribution in LICENSE.txt), but
> +   *  has now been substantially changed based on
> +   *  discussions from java-dev thread with subject "Dynamic
> +   *  array reallocation algorithms", started on Jan 12
> +   *  2010.
> +   *
> +   * @param minTargetSize Minimum required value to be returned.
> +   * @param bytesPerElement Bytes used by each element of
> +   * the array.  See constants in {@link RamUsageEstimator}.
> +   *
> +   * @lucene.internal
> +   */
> +
> +  public static int oversize(int minTargetSize, int bytesPerElement) {
> +
> +    if (minTargetSize < 0) {
> +      // catch usage that accidentally overflows int
> +      throw new IllegalArgumentException("invalid array size " +
> minTargetSize);
> +    }
> +
> +    if (minTargetSize == 0) {
> +      // wait until at least one element is requested
> +      return 0;
> +    }
> 
> -  public static int getNextSize(int targetSize) {
> -    /* This over-allocates proportional to the list size, making room
> -     * for additional growth.  The over-allocation is mild, but is
> -     * enough to give linear-time amortized behavior over a long
> -     * sequence of appends() in the presence of a poorly-performing
> -     * system realloc().
> -     * The growth pattern is:  0, 4, 8, 16, 25, 35, 46, 58, 72, 88,
> ...
> -     */
> -    return (targetSize >> 3) + (targetSize < 9 ? 3 : 6) + targetSize;
> +    // asymptotic exponential growth by 1/8th, favors
> +    // spending a bit more CPU to not tye up too much wasted
> +    // RAM:
> +    int extra = minTargetSize >> 3;
> +
> +    if (extra < 3) {
> +      // for very small arrays, where constant overhead of
> +      // realloc is presumably relatively high, we grow
> +      // faster
> +      extra = 3;
> +    }
> +
> +    int newSize = minTargetSize + extra;
> +
> +    // add 7 to allow for worst case byte alignment addition below:
> +    if (newSize+7 < 0) {
> +      // int overflowed -- return max allowed array size
> +      return Integer.MAX_VALUE;
> +    }
> +
> +    if (Constants.JRE_IS_64BIT) {
> +      // round up to 8 byte alignment in 64bit env
> +      switch(bytesPerElement) {
> +      case 4:
> +        // round up to multiple of 2
> +        return (newSize + 1) & 0x7ffffffe;
> +      case 2:
> +        // round up to multiple of 4
> +        return (newSize + 3) & 0x7ffffffc;
> +      case 1:
> +        // round up to multiple of 8
> +        return (newSize + 7) & 0x7ffffff8;
> +      case 8:
> +        // no rounding
> +      default:
> +        // odd (invalid?) size
> +        return newSize;
> +      }
> +    } else {
> +      // round up to 4 byte alignment in 64bit env
> +      switch(bytesPerElement) {
> +      case 2:
> +        // round up to multiple of 2
> +        return (newSize + 1) & 0x7ffffffe;
> +      case 1:
> +        // round up to multiple of 4
> +        return (newSize + 3) & 0x7ffffffc;
> +      case 4:
> +      case 8:
> +        // no rounding
> +      default:
> +        // odd (invalid?) size
> +        return newSize;
> +      }
> +    }
>    }
> 
> -  public static int getShrinkSize(int currentSize, int targetSize) {
> -    final int newSize = getNextSize(targetSize);
> +  public static int getShrinkSize(int currentSize, int targetSize, int
> bytesPerElement) {
> +    final int newSize = oversize(targetSize, bytesPerElement);
>      // Only reallocate if we are "substantially" smaller.
>      // This saves us from "running hot" (constantly making a
>      // bit bigger then a bit smaller, over and over):
> @@ -147,7 +222,7 @@
> 
>    public static int[] grow(int[] array, int minSize) {
>      if (array.length < minSize) {
> -      int[] newArray = new int[getNextSize(minSize)];
> +      int[] newArray = new int[oversize(minSize,
> RamUsageEstimator.NUM_BYTES_INT)];
>        System.arraycopy(array, 0, newArray, 0, array.length);
>        return newArray;
>      } else
> @@ -159,7 +234,7 @@
>    }
> 
>    public static int[] shrink(int[] array, int targetSize) {
> -    final int newSize = getShrinkSize(array.length, targetSize);
> +    final int newSize = getShrinkSize(array.length, targetSize,
> RamUsageEstimator.NUM_BYTES_INT);
>      if (newSize != array.length) {
>        int[] newArray = new int[newSize];
>        System.arraycopy(array, 0, newArray, 0, newSize);
> @@ -170,7 +245,7 @@
> 
>    public static long[] grow(long[] array, int minSize) {
>      if (array.length < minSize) {
> -      long[] newArray = new long[getNextSize(minSize)];
> +      long[] newArray = new long[oversize(minSize,
> RamUsageEstimator.NUM_BYTES_LONG)];
>        System.arraycopy(array, 0, newArray, 0, array.length);
>        return newArray;
>      } else
> @@ -182,7 +257,7 @@
>    }
> 
>    public static long[] shrink(long[] array, int targetSize) {
> -    final int newSize = getShrinkSize(array.length, targetSize);
> +    final int newSize = getShrinkSize(array.length, targetSize,
> RamUsageEstimator.NUM_BYTES_LONG);
>      if (newSize != array.length) {
>        long[] newArray = new long[newSize];
>        System.arraycopy(array, 0, newArray, 0, newSize);
> @@ -193,7 +268,7 @@
> 
>    public static byte[] grow(byte[] array, int minSize) {
>      if (array.length < minSize) {
> -      byte[] newArray = new byte[getNextSize(minSize)];
> +      byte[] newArray = new byte[oversize(minSize, 1)];
>        System.arraycopy(array, 0, newArray, 0, array.length);
>        return newArray;
>      } else
> @@ -205,7 +280,7 @@
>    }
> 
>    public static byte[] shrink(byte[] array, int targetSize) {
> -    final int newSize = getShrinkSize(array.length, targetSize);
> +    final int newSize = getShrinkSize(array.length, targetSize, 1);
>      if (newSize != array.length) {
>        byte[] newArray = new byte[newSize];
>        System.arraycopy(array, 0, newArray, 0, newSize);
> 
> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.jav
> a
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/luce
> ne/util/RamUsageEstimator.java?rev=901662&r1=901661&r2=901662&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.jav
> a (original)
> +++
> lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.jav
> a Thu Jan 21 11:54:50 2010
> @@ -43,6 +43,14 @@
>    private int arraySize;
>    private int classSize;
> 
> +  public final static int NUM_BYTES_OBJECT_REF =
> Constants.JRE_IS_64BIT ? 8 : 4;
> +  public final static int NUM_BYTES_CHAR = 2;
> +  public final static int NUM_BYTES_SHORT = 2;
> +  public final static int NUM_BYTES_INT = 4;
> +  public final static int NUM_BYTES_LONG = 8;
> +  public final static int NUM_BYTES_FLOAT = 4;
> +  public final static int NUM_BYTES_DOUBLE = 8;
> +
>    private boolean checkInterned;
> 
>    /**
> 
> Modified:
> lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/luce
> ne/analysis/TestToken.java?rev=901662&r1=901661&r2=901662&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java
> (original)
> +++
> lucene/java/trunk/src/test/org/apache/lucene/analysis/TestToken.java
> Thu Jan 21 11:54:50 2010
> @@ -85,7 +85,6 @@
>        buf.append(buf.toString());
>      }
>      assertEquals(1048576, t.termLength());
> -    assertEquals(1179654, t.termBuffer().length);
> 
>      // now as a string, first variant
>      t = new Token();
> @@ -99,7 +98,6 @@
>        buf.append(content);
>      }
>      assertEquals(1048576, t.termLength());
> -    assertEquals(1179654, t.termBuffer().length);
> 
>      // now as a string, second variant
>      t = new Token();
> @@ -113,7 +111,6 @@
>        buf.append(content);
>      }
>      assertEquals(1048576, t.termLength());
> -    assertEquals(1179654, t.termBuffer().length);
> 
>      // Test for slow growth to a long term
>      t = new Token();
> @@ -127,7 +124,6 @@
>        buf.append("a");
>      }
>      assertEquals(20000, t.termLength());
> -    assertEquals(20167, t.termBuffer().length);
> 
>      // Test for slow growth to a long term
>      t = new Token();
> @@ -141,7 +137,6 @@
>        buf.append("a");
>      }
>      assertEquals(20000, t.termLength());
> -    assertEquals(20167, t.termBuffer().length);
>    }
> 
>    public void testToString() throws Exception {
> 
> Modified:
> lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/T
> estTermAttributeImpl.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/luce
> ne/analysis/tokenattributes/TestTermAttributeImpl.java?rev=901662&r1=90
> 1661&r2=901662&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/T
> estTermAttributeImpl.java (original)
> +++
> lucene/java/trunk/src/test/org/apache/lucene/analysis/tokenattributes/T
> estTermAttributeImpl.java Thu Jan 21 11:54:50 2010
> @@ -49,7 +49,6 @@
>        buf.append(buf.toString());
>      }
>      assertEquals(1048576, t.termLength());
> -    assertEquals(1179654, t.termBuffer().length);
> 
>      // now as a string, first variant
>      t = new TermAttributeImpl();
> @@ -63,7 +62,6 @@
>        buf.append(content);
>      }
>      assertEquals(1048576, t.termLength());
> -    assertEquals(1179654, t.termBuffer().length);
> 
>      // now as a string, second variant
>      t = new TermAttributeImpl();
> @@ -77,7 +75,6 @@
>        buf.append(content);
>      }
>      assertEquals(1048576, t.termLength());
> -    assertEquals(1179654, t.termBuffer().length);
> 
>      // Test for slow growth to a long term
>      t = new TermAttributeImpl();
> @@ -91,7 +88,6 @@
>        buf.append("a");
>      }
>      assertEquals(20000, t.termLength());
> -    assertEquals(20167, t.termBuffer().length);
> 
>      // Test for slow growth to a long term
>      t = new TermAttributeImpl();
> @@ -105,7 +101,6 @@
>        buf.append("a");
>      }
>      assertEquals(20000, t.termLength());
> -    assertEquals(20167, t.termBuffer().length);
>    }
> 
>    public void testToString() throws Exception {
> 
> Added:
> lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/luce
> ne/util/TestArrayUtil.java?rev=901662&view=auto
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java
> (added)
> +++
> lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java
> Thu Jan 21 11:54:50 2010
> @@ -0,0 +1,59 @@
> +package org.apache.lucene.util;
> +
> +/**
> + * Licensed to the Apache Software Foundation (ASF) under one or more
> + * contributor license agreements.  See the NOTICE file distributed
> with
> + * this work for additional information regarding copyright ownership.
> + * The ASF licenses this file to You under the Apache License, Version
> 2.0
> + * (the "License"); you may not use this file except in compliance
> with
> + * the License.  You may obtain a copy of the License at
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +import java.util.Random;
> +
> +public class TestArrayUtil extends LuceneTestCase {
> +
> +  // Ensure ArrayUtil.getNextSize gives linear amortized cost of
> realloc/copy
> +  public void testGrowth() {
> +    int currentSize = 0;
> +    long copyCost = 0;
> +
> +    // Make sure ArrayUtil hits Integer.MAX_VALUE, if we insist:
> +    while(currentSize != Integer.MAX_VALUE) {
> +      int nextSize = ArrayUtil.oversize(1+currentSize,
> RamUsageEstimator.NUM_BYTES_OBJECT_REF);
> +      assertTrue(nextSize > currentSize);
> +      if (currentSize > 0) {
> +        copyCost += currentSize;
> +        double copyCostPerElement = ((double) copyCost)/currentSize;
> +        assertTrue("cost " + copyCostPerElement, copyCostPerElement <
> 10.0);
> +      }
> +      currentSize = nextSize;
> +    }
> +  }
> +
> +  public void testMaxSize() {
> +    // intentionally pass invalid elemSizes:
> +    for(int elemSize=0;elemSize<10;elemSize++) {
> +      assertEquals(Integer.MAX_VALUE,
> ArrayUtil.oversize(Integer.MAX_VALUE, elemSize));
> +      assertEquals(Integer.MAX_VALUE,
> ArrayUtil.oversize(Integer.MAX_VALUE-1, elemSize));
> +    }
> +  }
> +
> +  public void testInvalidElementSizes() {
> +    final Random r = newRandom();
> +    for(int iter=0;iter<10000;iter++) {
> +      final int minTargetSize = r.nextInt(Integer.MAX_VALUE);
> +      final int elemSize = r.nextInt(11);
> +      final int v = ArrayUtil.oversize(minTargetSize, elemSize);
> +      assertTrue(v >= minTargetSize);
> +    }
> +  }
> +}
> 
> Propchange:
> lucene/java/trunk/src/test/org/apache/lucene/util/TestArrayUtil.java
> -----------------------------------------------------------------------
> -------
>     svn:eol-style = native
> 
> Modified:
> lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinarySt
> ringTools.java
> URL:
> http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/luce
> ne/util/TestIndexableBinaryStringTools.java?rev=901662&r1=901661&r2=901
> 662&view=diff
> =======================================================================
> =======
> ---
> lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinarySt
> ringTools.java (original)
> +++
> lucene/java/trunk/src/test/org/apache/lucene/util/TestIndexableBinarySt
> ringTools.java Thu Jan 21 11:54:50 2010
> @@ -164,14 +164,14 @@
>        int encodedLen1 = IndexableBinaryStringTools.getEncodedLength(
>            originalArray1, 0, numBytes1);
>        if (encodedLen1 > encoded1.length)
> -        encoded1 = new char[ArrayUtil.getNextSize(encodedLen1)];
> +        encoded1 = new char[ArrayUtil.oversize(encodedLen1,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>        IndexableBinaryStringTools.encode(originalArray1, 0, numBytes1,
> encoded1,
>            0, encodedLen1);
> 
>        int encodedLen2 =
> IndexableBinaryStringTools.getEncodedLength(original2,
>            0, numBytes2);
>        if (encodedLen2 > encoded2.length)
> -        encoded2 = new char[ArrayUtil.getNextSize(encodedLen2)];
> +        encoded2 = new char[ArrayUtil.oversize(encodedLen2,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>        IndexableBinaryStringTools.encode(original2, 0, numBytes2,
> encoded2, 0,
>            encodedLen2);
> 
> @@ -308,7 +308,7 @@
>        int encodedLen =
> IndexableBinaryStringTools.getEncodedLength(binary, 0,
>            numBytes);
>        if (encoded.length < encodedLen)
> -        encoded = new char[ArrayUtil.getNextSize(encodedLen)];
> +        encoded = new char[ArrayUtil.oversize(encodedLen,
> RamUsageEstimator.NUM_BYTES_CHAR)];
>        IndexableBinaryStringTools.encode(binary, 0, numBytes, encoded,
> 0,
>            encodedLen);
> 
> 



---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org