You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ab...@apache.org on 2022/04/26 18:50:57 UTC
[tez] branch branch-0.9 updated: TEZ-4406: Wrong FS Exception when warehouse and scratchdir are on different FS
This is an automated email from the ASF dual-hosted git repository.
abstractdog pushed a commit to branch branch-0.9
in repository https://gitbox.apache.org/repos/asf/tez.git
The following commit(s) were added to refs/heads/branch-0.9 by this push:
new 769b21a08 TEZ-4406: Wrong FS Exception when warehouse and scratchdir are on different FS
769b21a08 is described below
commit 769b21a083208982efa7ad55138d7c634a357081
Author: guptanikhil007 <gu...@gmail.com>
AuthorDate: Tue Apr 26 04:29:03 2022 +0530
TEZ-4406: Wrong FS Exception when warehouse and scratchdir are on different FS
Co-authored-by: Nikhil Gupta <gu...@microsoft.com>
---
tez-api/src/main/java/org/apache/tez/client/TezClient.java | 5 +++--
.../dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java | 5 +++--
2 files changed, 6 insertions(+), 4 deletions(-)
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index 949a2ce18..a64519a1d 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -691,14 +691,15 @@ public class TezClient {
sessionAppId.toString()), TezConstants.TEZ_PB_PLAN_BINARY_NAME +
serializedSubmitDAGPlanRequestCounter.incrementAndGet());
- try (FSDataOutputStream fsDataOutputStream = stagingFs.create(dagPlanPath, false)) {
+ FileSystem fs = dagPlanPath.getFileSystem(stagingFs.getConf());
+ try (FSDataOutputStream fsDataOutputStream = fs.create(dagPlanPath, false)) {
LOG.info("Send dag plan using YARN local resources since it's too large"
+ ", dag plan size=" + request.getSerializedSize()
+ ", max dag plan size through IPC=" + maxSubmitDAGRequestSizeThroughIPC
+ ", max IPC message size= " + amConfig.getTezConfiguration().getInt(
CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH, CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT));
request.writeTo(fsDataOutputStream);
- request = requestBuilder.clear().setSerializedRequestPath(stagingFs.resolvePath(dagPlanPath).toString()).build();
+ request = requestBuilder.clear().setSerializedRequestPath(fs.resolvePath(dagPlanPath).toString()).build();
}
}
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
index 72cf0d564..4bdb46885 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
@@ -166,7 +166,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
if (request.hasSerializedRequestPath()) {
// need to deserialize large request from hdfs
Path requestPath = new Path(request.getSerializedRequestPath());
- try (FSDataInputStream fsDataInputStream = stagingFs.open(requestPath)) {
+ FileSystem fs = requestPath.getFileSystem(stagingFs.getConf());
+ try (FSDataInputStream fsDataInputStream = fs.open(requestPath)) {
CodedInputStream in =
CodedInputStream.newInstance(fsDataInputStream);
in.setSizeLimit(Integer.MAX_VALUE);
@@ -183,7 +184,7 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
}
String dagId = real.submitDAG(dagPlan, additionalResources);
return SubmitDAGResponseProto.newBuilder().setDagId(dagId).build();
- } catch(TezException e) {
+ } catch(IOException | TezException e) {
throw wrapException(e);
}
}