[ 
https://issues.apache.org/jira/browse/TEZ-3732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16014305#comment-16014305
 ] 

Jonathan Eagles commented on TEZ-3732:
--------------------------------------

Before for FetchInput
{code}
Failed to find matching constructor, falling back to class-only introspection.

org.apache.tez.runtime.library.common.shuffle.DiskFetchedInput object internals:
 OFFSET  SIZE                                                                 
TYPE DESCRIPTION                               VALUE
      0    12                                                                   
   (object header)                           N/A
     12     4                                                                  
int FetchedInput.id                           N/A
     16     8                                                                 
long FetchedInput.actualSize                   N/A
     24     8                                                                 
long FetchedInput.compressedSize               N/A
     32     4         
org.apache.tez.runtime.library.common.InputAttemptIdentifier 
FetchedInput.inputAttemptIdentifier       N/A
     36     4      
org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type 
FetchedInput.type                         N/A
     40     4   
org.apache.tez.runtime.library.common.shuffle.FetchedInputCallback 
FetchedInput.callback                     N/A
     44     4     
org.apache.tez.runtime.library.common.shuffle.FetchedInput.State 
FetchedInput.state                        N/A
     48     4                                      
org.apache.hadoop.fs.FileSystem DiskFetchedInput.localFS                  N/A
     52     4                                            
org.apache.hadoop.fs.Path DiskFetchedInput.tmpOutputPath            N/A
     56     4                                            
org.apache.hadoop.fs.Path DiskFetchedInput.outputPath               N/A
     60     4                                                                   
   (loss due to the next object alignment)
Instance size: 64 bytes
Space losses: 0 bytes internal + 4 bytes external = 4 bytes total
{code}

{code}
Failed to find matching constructor, falling back to class-only introspection.

org.apache.tez.runtime.library.common.shuffle.LocalDiskFetchedInput object 
internals:
 OFFSET  SIZE                                                                 
TYPE DESCRIPTION                               VALUE
      0    12                                                                   
   (object header)                           N/A
     12     4                                                                  
int FetchedInput.id                           N/A
     16     8                                                                 
long FetchedInput.actualSize                   N/A
     24     8                                                                 
long FetchedInput.compressedSize               N/A
     32     4         
org.apache.tez.runtime.library.common.InputAttemptIdentifier 
FetchedInput.inputAttemptIdentifier       N/A
     36     4      
org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type 
FetchedInput.type                         N/A
     40     4   
org.apache.tez.runtime.library.common.shuffle.FetchedInputCallback 
FetchedInput.callback                     N/A
     44     4     
org.apache.tez.runtime.library.common.shuffle.FetchedInput.State 
FetchedInput.state                        N/A
     48     8                                                                 
long LocalDiskFetchedInput.startOffset         N/A
     56     4                                            
org.apache.hadoop.fs.Path LocalDiskFetchedInput.inputFile           N/A
     60     4                                      
org.apache.hadoop.fs.FileSystem LocalDiskFetchedInput.localFS             N/A
Instance size: 64 bytes
Space losses: 0 bytes internal + 0 bytes external = 0 bytes total
{code}

{code}
Instantiated the sample instance via public 
org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput(long,long,org.apache.tez.runtime.library.common.InputAttemptIdentifier,org.apache.tez.runtime.library.common.shuffle.FetchedInputCallback)

org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput object 
internals:
 OFFSET  SIZE                                                                 
TYPE DESCRIPTION                               VALUE
      0     4                                                                   
   (object header)                           01 00 00 00 (00000001 00000000 
00000000 00000000) (1)
      4     4                                                                   
   (object header)                           00 00 00 00 (00000000 00000000 
00000000 00000000) (0)
      8     4                                                                   
   (object header)                           18 ba 0d 00 (00011000 10111010 
00001101 00000000) (899608)
     12     4                                                                  
int FetchedInput.id                           2
     16     8                                                                 
long FetchedInput.actualSize                   0
     24     8                                                                 
long FetchedInput.compressedSize               0
     32     4         
org.apache.tez.runtime.library.common.InputAttemptIdentifier 
FetchedInput.inputAttemptIdentifier       null
     36     4      
org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type 
FetchedInput.type                         (object)
     40     4   
org.apache.tez.runtime.library.common.shuffle.FetchedInputCallback 
FetchedInput.callback                     null
     44     4     
org.apache.tez.runtime.library.common.shuffle.FetchedInput.State 
FetchedInput.state                        (object)
     48     4                    
org.apache.hadoop.io.BoundedByteArrayOutputStream MemoryFetchedInput.byteStream 
            (object)
     52     4                                                                   
   (loss due to the next object alignment)
Instance size: 56 bytes
Space losses: 0 bytes internal + 4 bytes external = 4 bytes total
{code}

> Reduce Object size of InputAttemptIdentifier and MapOutput for large jobs
> -------------------------------------------------------------------------
>
>                 Key: TEZ-3732
>                 URL: https://issues.apache.org/jira/browse/TEZ-3732
>             Project: Apache Tez
>          Issue Type: Bug
>            Reporter: Jonathan Eagles
>            Assignee: Jonathan Eagles
>         Attachments: TEZ-3732.1.patch
>
>
> Objects in 64bit java are 12bytes + member size aligned to 8 bytes
> InputAttemptIdentifier -> 33Bytes gets aligned up to 40 bytes
> This class is just one byte over the 32 byte alignment. Reducing object size 
> by one byte can save 8 bytes per object.
> This is ~8MB savings for 1,000,000 inputs and ~80 MB savings for tasks with 
> 10,000,000 inputs to fetch (Yes this is a real job)
> MapOutput -> 45 bytes gets aligned to 48 bytes
> This class can be sub-classed to avoid all sub-classes paying the object size 
> cost for the other sub-classes
> Wait InMemory and DiskDirect -> 32 bytes
> Disk -> 40 bytes
> Total savings is harder to account for but more than the above case.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to