[
https://issues.apache.org/jira/browse/TEZ-3155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15183553#comment-15183553
]
Hitesh Shah commented on TEZ-3155:
----------------------------------
Thanks for addressing the previous comments. Some more comments based on patch
4:
{code}
<Match>
47 <Class name="org.apache.tez.dag.api.records.DAGProtos$DAGPlan"/>
48 <Field name="credentialsBinary_"/>
49 <Bug pattern="SE_BAD_FIELD"/>
50 </Match>
{code}
- this addition seems to have no relation to the proto being modified - why
was this needed?
TezClient:
{code} private FileSystem fs = null; {code}
- rename this to something like stagingFs. Also, this should be initialized
once in init() and re-used.
{code}
137 private static final int gapToMaxIPCSize = 5 * 1024 * 1024;
138 private AtomicInteger serializedDAGPlanCounter = new AtomicInteger(0);
{code}
- above need code comments to describe that the vars are.
- might be good to make gapToMaxIPCSize configurable with default as 5 MB ).
Mark the new config property as Private though
{code}
dagClientConf.getInt(CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH,
530 CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT)) {
{code}
- this should be a class member var and initialized once. Also it should use
the main tezconf and not dagclientconf
{code} TezConfiguration tezConf = amConfig.getTezConfiguration(); {code}
- no need to create an extra local var. Just use
"amConfig.getTezConfiguration()" directly
{code}
/* we need manually delete the serialized dagplan since staging path here
won't be destroyed */
190 Path dagPlanPath = new Path(request.getSerializedDagPlanPath());
191 FileSystem fs = dagPlanPath.getFileSystem(conf);
192 fs.delete(dagPlanPath, false);
{code}
- this is not reliable if there is a test failure or an exception is thrown
- staging dir should be set to target and also use the local fs
- Using local fs could be done by having a package private method to override
the stagingFs in TezClient with the value of FileSystem::getLocal
- For the dag plan file, use deleteOnExit()
TestTezClient:
{code}
int maxIPCMsgSize = 1024;
173 conf.setInt(CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH,
maxIPCMsgSize);
174
processorDescriptor.setUserPayload(UserPayload.create(ByteBuffer.allocate(2*maxIPCMsgSize)));
{code}
- processorDescriptor.setUserPayload() is not being invoked for the
largeDagPlan false case? - shouldnt it always be set to say 2 MB in both
scenarios and the max limit changed to 1 MB in one scenario and say 8 ( +5 for
the overhead check ) MB in the other scenario? This can played around with to
address my following comments on the buffer and additional resources checks.
- how is the 5 MB buffer check being tested?
- Also, there is no test if additionalResources ( or a combination of dag
plan + additional rsrcs ) exceeds ipc limits?
DAGClientAMProtocolBlockingPBServerImpl:
- fs can be initialized in the ctor itself
{code}
try (FSDataInputStream fsDataInputStream = fs.open(requestPath)) {
173 dagPlan = DAGPlan.parseFrom(fsDataInputStream);
174 } catch (IOException e) {
175 throw wrapException(e);
176 }
{code}
- wont the exception thrown in line 173 be caught be the catch in line 186 ?
testSubmitDagInSessionWithLargeDagPlan
- test could be enhanced to verify the payload contents after deserialization
> 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
>
>
> 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)