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

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

Some general overal comments:
   - patch looks decent for the most part
   - please do not change code lines if they are not needed ( minor edits, new 
lines, etc). Small changes are sometimes ok but numerous modifications make it 
hard for a reviewer. 

Some comments on the patch: 

TezClient
  - why 0.8 for largeDAGPlanRatio ? Why does it need to be so high for a 64 MB 
max limit? Shoudnt it be more like a fixed number to account for the overhead?
  - please consider using DAGPlan.writeTo* to write the output stream instead 
of the objectoutputstream  
  - output stream needs to be closed?
  
{code}
        FileSystem fs = dagPlanPath.getFileSystem(dagClientConf);
{code}
  - the above should use amCOnfig.getTezConf() and should be done once per 
TezClient and the fs object re-used. 

{code}
      SubmitDAGRequestProto request = requestBuilder.build();
     SubmitDAGResponseProto response = proxy.submitDAG(null, request);
{code}
  - any reason why this code change was needed? 

  - requestBuilder.build() is getting called twice - should be fixed to be 
called once. 
  - SERIALIZED_DAGPLAN_NAME_PREFIX can be replaced by TEZ_PB_PLAN_BINARY_NAME

{code}
      requestBuilder.setSerializedDagPlanPath(dagPlanPath.toString());
{code}
  - this should be a fully resolved path ( check fs.resolve...) 

TestTezClient:
  - there should be a test for the negative case i.e. one where the dag plan is 
within the defined threshold 

DAGClientServer:
  
{code}
      DAGClientAMProtocolBlockingPBServerImpl service =
DAGClientAMProtocolBlockingPBServerImpl(realInstance, conf);
{code}
  - why was the above needed?

DAGClientAMProtocolBlockingPBServerImpl:
   - it might be better to move this aspect of the code into DAGAppMaster where 
the staging fs object can be reused. 
   - also the stream needs to be closed in a finally block

 


 

> 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
>
>
> 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