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

Hitesh Shah commented on TEZ-3155:
----------------------------------

Nearly there - thanks for the patience in addressing review comments. 

Some minor comments:

{code}
 // amResources exceeds max IPC size
176         _testTezClientSessionLargeDAGPlan(10*1024*1024, 3*1024*1024, 
3*1024*1024, true);
177         // DAGPlan and amResources together exceed threshold but less than 
IPC size
178         _testTezClientSessionLargeDAGPlan(10*1024*1024, 10*1024*1024, 
5*1024*1024, true);
{code}
  - comments dont seem to match the test params

{code}
8             assertTrue(request.hasSerializedRequestPath());
219           assertFalse(request.hasDAGPlan());
220         } else {
221           assertFalse(request.hasSerializedRequestPath());
222           assertTrue(request.hasDAGPlan());
{code}
   - test should also check the additional resources field to see if it is set 
or unset similar to dagPlan

DAGClientAMProtocolBlockingPBServerImpl:
  - stagingFs should be a final var?
  - " request = SubmitDAGRequestProto.parseFrom(fsDataInputStream);" - is this 
safe? i.e. merging contents from parseFrom() into an already existing object 
with other data set from the rpc call?

DAGAppMaster:
    "clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, 
recoveryFS);" - are you sure that recoveryFS is initialized correctly by this 
point? 

TestDAGClientServer:
"      clientServer = new DAGClientServer(mockDAGClientHander, mockAppAttempId, 
FileSystem.get(conf));" - why not just use a mock of the FS?



  

> 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, TEZ-3155.6.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)

Reply via email to