Author: thejas
Date: Wed Jul 21 17:33:42 2010
New Revision: 966325

URL: http://svn.apache.org/viewvc?rev=966325&view=rev
Log:
PIG-1492 : DefaultTuple and DefaultMemory understimate their memory footprint

Modified:
    hadoop/pig/trunk/CHANGES.txt
    hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java
    hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java
    hadoop/pig/trunk/test/org/apache/pig/test/TestTuple.java

Modified: hadoop/pig/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=966325&r1=966324&r2=966325&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Wed Jul 21 17:33:42 2010
@@ -100,6 +100,8 @@ PIG-1309: Map-side Cogroup (ashutoshc)
 
 BUG FIXES
 
+PIG-1492: DefaultTuple and DefaultMemory understimate their memory footprint 
(thejas)
+
 PIG-1409: Fix up javadocs for org.apache.pig.builtin (gates)
 
 PIG-1490: Make Pig storers work with remote HDFS in secure mode (rding)

Modified: hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java
URL: 
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java?rev=966325&r1=966324&r2=966325&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java Wed Jul 21 
17:33:42 2010
@@ -140,20 +140,49 @@ public abstract class DefaultAbstractBag
             used *= numInMem;
         }
 
-        // add up the overhead for this object, mContents object, references 
to tuples,
-        // and other object variables
-        used += 12 + 12 + numInMem*4 + 8 + 4 + 8;
+        // add up the overhead for this object and other object variables
+        int bag_fix_size = 8 /* object header */ 
+        + 4 + 8 + 8 /* mLastContentsSize + mMemSize + mSize */
+        + 8 + 8 /* mContents ref  + mSpillFiles ref*/
+        + 4 /* +4 to round it to eight*/
+        + 36 /* mContents fixed */
+        ;
+        long mFields_size =   roundToEight(4 + numInMem*4); /* mContents fixed 
+ per entry */
+        // in java hotspot 32bit vm, there seems to be a minimum bag size of 
188 bytes
+        // some of the extra bytes is probably from a minimum size of this 
array list
+        mFields_size = Math.max(40, mFields_size); 
         
+        used += bag_fix_size + mFields_size;
+
         // add up overhead for mSpillFiles ArrayList, Object[] inside 
ArrayList,
         // object variable inside ArrayList and references to spill files
         if (mSpillFiles != null) {
-               used += 12 + 12 + 4 + mSpillFiles.size()*4;
+            used += roundToEight(36 /* mSpillFiles fixed overhead*/ + 
mSpillFiles.size()*4);
+            
+            if(mSpillFiles.size() > 0){
+                //a rough estimate of memory used by each file entry
+                // the auto generated files are likely to have same length
+                long approx_per_entry_size =
+                    roundToEight(mSpillFiles.get(0).toString().length() * 2 + 
38);
+                
+                used += mSpillFiles.size() * approx_per_entry_size;
+            }
         }
         
         mMemSize = used;
         return used;
     }
 
+    
+    /**
+     * Memory size of objects are rounded to multiple of 8 bytes
+     * @param i
+     * @return i rounded to a equal of higher multiple of 8 
+     */
+    private long roundToEight(long i) {
+        return 8 * ((i+7)/8); // integer division rounds the result down
+    }
+    
     /**
      * Clear out the contents of the bag, both on disk and in memory.
      * Any attempts to read after this is called will produce undefined

Modified: hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java
URL: 
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java?rev=966325&r1=966324&r2=966325&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java Wed Jul 21 
17:33:42 2010
@@ -31,17 +31,15 @@ import org.apache.pig.backend.executione
 import org.apache.pig.impl.util.TupleFormat;
 
 /**
- * This was the old default implementation of Tuple. The new default is
- * {...@link BinSedesTuple} .   
- * Zebra and BinStorage load/store functions use the .write(..) and 
.readFields(..)
- * functions here for (de)serialization.
+ * A default implementation of Tuple.  This class will be created by the
+ * DefaultTupleFactory.
  */
 public class DefaultTuple implements Tuple {
     
     protected boolean isNull = false;
     private static final long serialVersionUID = 2L;
     protected List<Object> mFields;
-        
+    
     /**
      * Default constructor.  This constructor is public so that hadoop can call
      * it directly.  However, inside pig you should never be calling this
@@ -180,16 +178,38 @@ public class DefaultTuple implements Tup
      */
     public long getMemorySize() {
         Iterator<Object> i = mFields.iterator();
-        // initial memory overhead for Tuple object, ArrayList object
-        // and Object[] inside ArrayList, plus references to each tuple field,
-        // plus other object variables
-        long sum = 12*3 + mFields.size()*4 + 8;
+        //fixed overhead
+        long empty_tuple_size = 8 /* tuple object header*/ 
+        + 8 /* isNull - but rounded to 8 bytes as total obj size needs to be 
multiple of 8 */
+        + 8 /* mFields reference*/
+        + 32 /* mFields array list fixed size*/;
+
+        
+        //rest of the fixed portion of mfields size is accounted within 
empty_tuple_size
+        long mfields_var_size =  roundToEight(4 + 4*mFields.size());
+        // in java hotspot 32bit vm, there seems to be a minimum tuple size of 
96
+        // which is probably from the minimum size of this array list
+        mfields_var_size = Math.max(40, mfields_var_size); 
+        
+        
+        long sum = empty_tuple_size + mfields_var_size;
         while (i.hasNext()) {
             sum += getFieldMemorySize(i.next());
         }
         return sum;
     }
 
+    
+    
+    /**
+     * Memory size of objects are rounded to multiple of 8 bytes
+     * @param i
+     * @return i rounded to a equal of higher multiple of 8 
+     */
+    private long roundToEight(long i) {
+        return 8 * ((i+7)/8); // integer division rounds the result down
+    }
+
     /** 
      * Write a tuple of atomic values into a string.  All values in the
      * tuple must be atomic (no bags, tuples, or maps).
@@ -259,7 +279,12 @@ public class DefaultTuple implements Tup
     }
 
     public void write(DataOutput out) throws IOException {
-        DataReaderWriter.writeDatum(out, this);
+        out.writeByte(DataType.TUPLE);
+        int sz = size();
+        out.writeInt(sz);
+        for (int i = 0; i < sz; i++) {
+            DataReaderWriter.writeDatum(out, mFields.get(i));
+        }
     }
 
     public void readFields(DataInput in) throws IOException {
@@ -292,13 +317,16 @@ public class DefaultTuple implements Tup
         switch (DataType.findType(o)) {
             case DataType.BYTEARRAY: {
                 byte[] bytes = ((DataByteArray)o).get();
-                return bytes.length + 12;
+                // bytearray size including rounding to 8 bytes
+                long byte_array_sz = roundToEight(bytes.length + 12);
+                
+                return byte_array_sz + 16 /*16 is additional size of 
DataByteArray */;
             }
 
             case DataType.CHARARRAY: {
                 String s = (String)o;
                 // See PIG-1443 for a reference for this formula
-                return 8 * (((s.length() * 2) + 45) / 8);
+                return roundToEight((s.length() * 2) + 38);
             }
 
             case DataType.TUPLE: {
@@ -312,10 +340,10 @@ public class DefaultTuple implements Tup
             }
 
             case DataType.INTEGER:
-                return 4 + 12;
+                return 4 + 8 + 4/*+4 to round to 8 bytes*/;
 
             case DataType.LONG:
-                return 8 + 12;
+                return 8 + 8;
 
             case DataType.MAP: {
                 Map<String, Object> m = (Map<String, Object>)o;
@@ -327,17 +355,27 @@ public class DefaultTuple implements Tup
                     sum += getFieldMemorySize(entry.getKey());
                     sum += getFieldMemorySize(entry.getValue());
                 }
-                return sum + 12;
+                //based on experiments on 32 bit Java HotSpot VM
+                // size of map with 0 entries is 120 bytes
+                // each additional entry have around 24 bytes overhead at 
+                // small number of entries. At larger number of entries, the  
+                // overhead is around 32 bytes, probably because of the 
expanded
+                // data structures in anticapation of more entries being added
+                return sum + m.size()*32  + 120;
             }
 
             case DataType.FLOAT:
-                return 8 + 12;
+                return 4 + 8 + 4/*+4 to round to 8 bytes*/;
 
             case DataType.DOUBLE:
-                return 16 + 12;
+                return 8 + 8;
 
             case DataType.BOOLEAN:
-                return 4 + 12;
+                //boolean takes 1 byte , +7 to round it to 8
+                return 1 + 8 + 7;
+                
+            case DataType.NULL:
+                return 0;
 
             default:
                 // ??

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestTuple.java
URL: 
http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestTuple.java?rev=966325&r1=966324&r2=966325&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestTuple.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestTuple.java Wed Jul 21 
17:33:42 2010
@@ -25,6 +25,9 @@ import junit.framework.TestCase;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DefaultBagFactory;
+import org.apache.pig.data.DefaultDataBag;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.util.TupleFormat;
@@ -60,11 +63,64 @@ public class TestTuple extends TestCase 
         }
 
     }
+
+    public void testEmptyTupleSize() {
+        Tuple t = TupleFactory.getInstance().newTuple();
+        long size = t.getMemorySize();
+        assertEquals("tuple size",size, 96);
+    }
+    
+    public void testEmptyBagSize() {
+        DataBag bag = DefaultBagFactory.getInstance().newDefaultBag();
+        long size = bag.getMemorySize();
+        assertEquals("bag size",size, 124);
+    }
     
     // See PIG-1443
     public void testTupleSizeWithString() {
         Tuple t = Util.createTuple(new String[] {"1234567", "bar"});
         long size = t.getMemorySize();
-        assertTrue(size==156);
+        assertEquals("tuple size",size, 200);
+    }
+    
+    public void testTupleSizeWithByteArrays() {
+        Tuple t = TupleFactory.getInstance().newTuple();
+        t.append(new DataByteArray("1234567"));
+        t.append(new DataByteArray("bar"));
+        long size = t.getMemorySize();
+        assertEquals("tuple size",size, 168);
+    }
+
+    public void testTupleSizeWithDoubles() {
+        Tuple t = TupleFactory.getInstance().newTuple();
+        t.append(new Double(0.1));
+        t.append(new Double(2000.10001));
+        long size = t.getMemorySize();
+        assertEquals("tuple size",size, 128);
+    }
+
+    public void testTupleSizeWithFloats() {
+        Tuple t = TupleFactory.getInstance().newTuple();
+        t.append(new Float(0.1F));
+        t.append(new Float(2000.10001F));
+        long size = t.getMemorySize();
+        assertEquals("tuple size",size, 128);
+    }
+    
+    public void testTupleSizeWithLongs() {
+        Tuple t = TupleFactory.getInstance().newTuple();
+        t.append(new Long(100));
+        t.append(new Long(2000));
+        long size = t.getMemorySize();
+        assertEquals("tuple size",size, 128);
     }
+    
+    public void testTupleSizeWithBooleans() {
+        Tuple t = TupleFactory.getInstance().newTuple();
+        t.append(new Boolean(true));
+        t.append(new Boolean(false));
+        long size = t.getMemorySize();
+        assertEquals("tuple size",size, 128);
+    }    
+    
 }


Reply via email to