[
https://issues.apache.org/jira/browse/TEZ-3155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15185961#comment-15185961
]
Hitesh Shah commented on TEZ-3155:
----------------------------------
Comments:
Maybe
s/TEZ_CLIENT_REQUEST_GAP_TO_MAX_IPC_SIZE_BYTE/TEZ_IPC_PAYLOAD_RESERVED_BYTES
- tez.ipc.payload.reserved.bytes
( as the value is indicate how of the much of the ipc payload is being
reserved for overheads )
- also removed CLIENT as this could be leveraged in other ipc calls later if
needed
{code}
this.stagingFs =
TezCommonUtils.getTezSystemStagingPath(amConfig.getTezConfiguration(),
sessionAppId.toString()).getFileSystem(amConfig.getTezConfiguration());
{code}
- this can be simplified. Look at Filesystem.get(Config)
{code}
// TODO: if additionalAMResources itself exceeds the limit, we still have
problems. This can be solved by serializing
...
try {
546 fsDataOutputStream = stagingFs.create(dagPlanPath, false);
547 dagPlan.writeTo(fsDataOutputStream);
548
requestBuilder.setSerializedDagPlanPath(stagingFs.resolvePath(dagPlanPath).toString());
549 }
{code}
- Sorry - missed this in the earlier review. Any reason why this is not
addressed as part of this patch? As part of the max len check, both dag plan
and local resources could be written to the same file using writeDelimitedTo.
{code}
testTezClientSessionLargeDAGPlan(10*1024*1024, 10, 10, false);
164 testTezClientSessionLargeDAGPlan(10*1024*1024, 5*1024*1024, 10,
true);
165 testTezClientSessionLargeDAGPlan(10*1024*1024, 10, 5*1024*1024,
true);
166 testTezClientSessionLargeDAGPlan(10*1024*1024, 1*1024*1024,
5*1024*1024, true);
167 testTezClientSessionLargeDAGPlan(10*1024*1024, 10*1024*1024,
5*1024*1024, true);
168 // The case that amResourceSize exeeds the maxIPCMsgSize is not
tested because our code won't work in this case
{code}
- the tests belong in a separate @Test function say
testTezClientSessionLargeDAGPlan and rename the current
testTezClientSessionLargeDAGPlan to _testTezClientSessionLargeDAGPlan to
indicate a private function. Also, a comment for what permutation of the test
represents would be good.
{code}
conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, "target");
{code}
- dir should target/TestTezClient/ to ensure that it does not clash with
other test code.
"new DAGClientAMProtocolBlockingPBServerImpl(realInstance, conf);" - should be
changed to "new DAGClientAMProtocolBlockingPBServerImpl(realInstance, fs);" by
appropriate code changes in DAGAppMaster to re-use recoveryFs
{code}
assertTrue(dagPlan.getName().equals(dagPlanName));
92 assertTrue(dagPlan.getVertexCount() == 1);
...
for (int i = 0; i < randomBytes.length; i++) {
98 assertTrue(randomBytes[i] == randomBytesDeserialized[i]);
99 }
{code}
- I think Assert supports array equals which can be leveraged here. Also,
instead of assertTrue it is better to use assertEquals
Minor nit:
{code}
(dagPlan.getSerializedSize()+requestBuilder.getAdditionalAmResources().getSerializedSize()
>
540 maxSubmitDAGRequestSizeThroughIPC)
{code}
- please add a parenthesis around the + op for more code clarity
> Support a way to submit DAGs to a session where the DAG plan exceeds hadoop
> ipc limits
> ---------------------------------------------------------------------------------------
>
> Key: TEZ-3155
> URL: https://issues.apache.org/jira/browse/TEZ-3155
> Project: Apache Tez
> Issue Type: Bug
> Reporter: Hitesh Shah
> Assignee: Zhiyuan Yang
> Attachments: TEZ-3155.1.patch, TEZ-3155.2.patch, TEZ-3155.3.patch,
> TEZ-3155.4.patch, TEZ-3155.5.patch
>
>
> Currently, dag submissions fail if the dag plan exceeds the hadoop ipc
> limits. One option would be to fall back to local resources if the dag plan
> is too large.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)