[
https://issues.apache.org/jira/browse/TEZ-3813?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Muhammad Samir Khan updated TEZ-3813:
-------------------------------------
Attachment: TEZ-3813.002.patch
Some more optimizations related to enums, similar to TEZ-3732.
*JOL Dump:*
+Before:+
Internals:
{code}
# Running 64-bit HotSpot VM.
# Using compressed oop with 3-bit shift.
# Using compressed klass with 3-bit shift.
# Objects are 8 bytes aligned.
# Field sizes by type: 4, 1, 1, 2, 2, 4, 4, 8, 8 [bytes]
# Array element sizes: 4, 1, 1, 2, 2, 4, 4, 8, 8 [bytes]
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) 7a 12 01 f8 (01111010 00010010
00000001 11111000) (-134147462)
12 4
int FetchedInput.id 0
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}
Footprint:
{code}
# Running 64-bit HotSpot VM.
# Using compressed oop with 3-bit shift.
# Using compressed klass with 3-bit shift.
# Objects are 8 bytes aligned.
# Field sizes by type: 4, 1, 1, 2, 2, 4, 4, 8, 8 [bytes]
# Array element sizes: 4, 1, 1, 2, 2, 4, 4, 8, 8 [bytes]
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@215be6bbd
footprint:
COUNT AVG SUM DESCRIPTION
1 16 16 [B
2 32 64 [C
2 24 48 java.lang.String
1 32 32
org.apache.hadoop.io.BoundedByteArrayOutputStream
1 24 24
org.apache.tez.runtime.library.common.shuffle.FetchedInput$State
1 24 24
org.apache.tez.runtime.library.common.shuffle.FetchedInput$Type
1 56 56
org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput
9 264 (total)
{code}
+After:+
Internals:
{code}
# Running 64-bit HotSpot VM.
# Using compressed oop with 3-bit shift.
# Using compressed klass with 3-bit shift.
# Objects are 8 bytes aligned.
# Field sizes by type: 4, 1, 1, 2, 2, 4, 4, 8, 8 [bytes]
# Array element sizes: 4, 1, 1, 2, 2, 4, 4, 8, 8 [bytes]
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) 7e 12 01 f8 (01111110 00010010
00000001 11111000) (-134147458)
12 4
int FetchedInput.id 0
16 8
long FetchedInput.actualSize 0
24 8
long FetchedInput.compressedSize 0
32 1
byte FetchedInput.state 0
33 3
(alignment/padding gap)
36 4
org.apache.tez.runtime.library.common.InputAttemptIdentifier
FetchedInput.inputAttemptIdentifier null
40 4
org.apache.tez.runtime.library.common.shuffle.FetchedInputCallback
FetchedInput.callback null
44 4
byte[] MemoryFetchedInput.byteArray []
Instance size: 48 bytes
Space losses: 3 bytes internal + 0 bytes external = 3 bytes total
{code}
Footprint:
{code}
# Running 64-bit HotSpot VM.
# Using compressed oop with 3-bit shift.
# Using compressed klass with 3-bit shift.
# Objects are 8 bytes aligned.
# Field sizes by type: 4, 1, 1, 2, 2, 4, 4, 8, 8 [bytes]
# Array element sizes: 4, 1, 1, 2, 2, 4, 4, 8, 8 [bytes]
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@215be6bbd
footprint:
COUNT AVG SUM DESCRIPTION
1 16 16 [B
1 48 48
org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput
2 64 (total)
{code}
> Reduce Object size of MemoryFetchedInput for large jobs
> -------------------------------------------------------
>
> Key: TEZ-3813
> URL: https://issues.apache.org/jira/browse/TEZ-3813
> Project: Apache Tez
> Issue Type: Bug
> Reporter: Muhammad Samir Khan
> Assignee: Muhammad Samir Khan
> Attachments: TEZ-3813.001.patch, TEZ-3813.002.patch
>
>
> Same as TEZ-3752 for the unordered case. MemoryFetchedInput has a
> BoundedByteArrayOutputStream that is not used (only the underlying byte[] is
> used).
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)