HDFS-6050. NFS does not handle exceptions correctly in a few places. Contributed by Brandon Li
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1581055 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0febaa1b3d
commit
3bfd18c6b0
|
@ -166,8 +166,7 @@ public class IdUserGroup {
|
|||
try {
|
||||
br.close();
|
||||
} catch (IOException e1) {
|
||||
LOG.error("Can't close BufferedReader of command result");
|
||||
e1.printStackTrace();
|
||||
LOG.error("Can't close BufferedReader of command result", e1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -249,7 +248,7 @@ public class IdUserGroup {
|
|||
uid = getUid(user);
|
||||
} catch (IOException e) {
|
||||
uid = user.hashCode();
|
||||
LOG.info("Can't map user " + user + ". Use its string hashcode:" + uid);
|
||||
LOG.info("Can't map user " + user + ". Use its string hashcode:" + uid, e);
|
||||
}
|
||||
return uid;
|
||||
}
|
||||
|
@ -262,7 +261,7 @@ public class IdUserGroup {
|
|||
gid = getGid(group);
|
||||
} catch (IOException e) {
|
||||
gid = group.hashCode();
|
||||
LOG.debug("Can't map group " + group + ". Use its string hashcode:" + gid);
|
||||
LOG.info("Can't map group " + group + ". Use its string hashcode:" + gid, e);
|
||||
}
|
||||
return gid;
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ public class Nfs3FileAttributes {
|
|||
private long used;
|
||||
private Specdata3 rdev;
|
||||
private long fsid;
|
||||
private long fileid;
|
||||
private long fileId;
|
||||
private NfsTime atime;
|
||||
private NfsTime mtime;
|
||||
private NfsTime ctime;
|
||||
|
@ -76,7 +76,7 @@ public class Nfs3FileAttributes {
|
|||
}
|
||||
|
||||
public Nfs3FileAttributes(NfsFileType nfsType, int nlink, short mode, int uid,
|
||||
int gid, long size, long fsid, long fileid, long mtime, long atime) {
|
||||
int gid, long size, long fsid, long fileId, long mtime, long atime) {
|
||||
this.type = nfsType.toValue();
|
||||
this.mode = mode;
|
||||
this.nlink = (type == NfsFileType.NFSDIR.toValue()) ? (nlink + 2) : 1;
|
||||
|
@ -89,7 +89,7 @@ public class Nfs3FileAttributes {
|
|||
this.used = this.size;
|
||||
this.rdev = new Specdata3();
|
||||
this.fsid = fsid;
|
||||
this.fileid = fileid;
|
||||
this.fileId = fileId;
|
||||
this.mtime = new NfsTime(mtime);
|
||||
this.atime = atime != 0 ? new NfsTime(atime) : this.mtime;
|
||||
this.ctime = this.mtime;
|
||||
|
@ -105,7 +105,7 @@ public class Nfs3FileAttributes {
|
|||
this.used = other.getUsed();
|
||||
this.rdev = new Specdata3();
|
||||
this.fsid = other.getFsid();
|
||||
this.fileid = other.getFileid();
|
||||
this.fileId = other.getFileId();
|
||||
this.mtime = new NfsTime(other.getMtime());
|
||||
this.atime = new NfsTime(other.getAtime());
|
||||
this.ctime = new NfsTime(other.getCtime());
|
||||
|
@ -122,7 +122,7 @@ public class Nfs3FileAttributes {
|
|||
xdr.writeInt(rdev.getSpecdata1());
|
||||
xdr.writeInt(rdev.getSpecdata2());
|
||||
xdr.writeLongAsHyper(fsid);
|
||||
xdr.writeLongAsHyper(fileid);
|
||||
xdr.writeLongAsHyper(fileId);
|
||||
atime.serialize(xdr);
|
||||
mtime.serialize(xdr);
|
||||
ctime.serialize(xdr);
|
||||
|
@ -142,7 +142,7 @@ public class Nfs3FileAttributes {
|
|||
xdr.readInt();
|
||||
attr.rdev = new Specdata3();
|
||||
attr.fsid = xdr.readHyper();
|
||||
attr.fileid = xdr.readHyper();
|
||||
attr.fileId = xdr.readHyper();
|
||||
attr.atime = NfsTime.deserialize(xdr);
|
||||
attr.mtime = NfsTime.deserialize(xdr);
|
||||
attr.ctime = NfsTime.deserialize(xdr);
|
||||
|
@ -154,7 +154,7 @@ public class Nfs3FileAttributes {
|
|||
return String.format("type:%d, mode:%d, nlink:%d, uid:%d, gid:%d, " +
|
||||
"size:%d, used:%d, rdev:%s, fsid:%d, fileid:%d, atime:%s, " +
|
||||
"mtime:%s, ctime:%s",
|
||||
type, mode, nlink, uid, gid, size, used, rdev, fsid, fileid, atime,
|
||||
type, mode, nlink, uid, gid, size, used, rdev, fsid, fileId, atime,
|
||||
mtime, ctime);
|
||||
}
|
||||
|
||||
|
@ -170,8 +170,8 @@ public class Nfs3FileAttributes {
|
|||
return fsid;
|
||||
}
|
||||
|
||||
public long getFileid() {
|
||||
return fileid;
|
||||
public long getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public NfsTime getAtime() {
|
||||
|
@ -194,10 +194,6 @@ public class Nfs3FileAttributes {
|
|||
return new WccAttr(size, mtime, ctime);
|
||||
}
|
||||
|
||||
public long getFileId() {
|
||||
return fileid;
|
||||
}
|
||||
|
||||
public long getSize() {
|
||||
return size;
|
||||
}
|
||||
|
|
|
@ -65,7 +65,7 @@ final class Portmap {
|
|||
pm.start(DEFAULT_IDLE_TIME_MILLISECONDS,
|
||||
new InetSocketAddress(port), new InetSocketAddress(port));
|
||||
} catch (Throwable e) {
|
||||
LOG.fatal("Failed to start the server. Cause:" + e.getMessage());
|
||||
LOG.fatal("Failed to start the server. Cause:", e);
|
||||
pm.shutdown();
|
||||
System.exit(-1);
|
||||
}
|
||||
|
|
|
@ -131,7 +131,7 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
|
|||
|
||||
handle = new FileHandle(exFileStatus.getFileId());
|
||||
} catch (IOException e) {
|
||||
LOG.error("Can't get handle for export:" + path + ", exception:" + e);
|
||||
LOG.error("Can't get handle for export:" + path, e);
|
||||
MountResponse.writeMNTResponse(Nfs3Status.NFS3ERR_NOENT, out, xid, null);
|
||||
return out;
|
||||
}
|
||||
|
|
|
@ -135,8 +135,7 @@ public class AsyncDataService {
|
|||
try {
|
||||
openFileCtx.executeWriteBack();
|
||||
} catch (Throwable t) {
|
||||
LOG.error("Asyn data service got error:"
|
||||
+ ExceptionUtils.getFullStackTrace(t));
|
||||
LOG.error("Asyn data service got error:", t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -94,7 +94,8 @@ public class Nfs3Utils {
|
|||
}
|
||||
|
||||
public static WccAttr getWccAttr(Nfs3FileAttributes attr) {
|
||||
return new WccAttr(attr.getSize(), attr.getMtime(), attr.getCtime());
|
||||
return attr == null ? new WccAttr() : new WccAttr(attr.getSize(),
|
||||
attr.getMtime(), attr.getCtime());
|
||||
}
|
||||
|
||||
// TODO: maybe not efficient
|
||||
|
|
|
@ -363,7 +363,7 @@ class OpenFileCtx {
|
|||
}
|
||||
} catch (Throwable t) {
|
||||
LOG.info("Dumper get Throwable: " + t + ". dumpFilePath: "
|
||||
+ OpenFileCtx.this.dumpFilePath);
|
||||
+ OpenFileCtx.this.dumpFilePath, t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -658,8 +658,8 @@ class OpenFileCtx {
|
|||
// Sync file data and length to avoid partial read failure
|
||||
fos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
||||
} catch (ClosedChannelException closedException) {
|
||||
LOG.info("The FSDataOutputStream has been closed. " +
|
||||
"Continue processing the perfect overwrite.");
|
||||
LOG.info("The FSDataOutputStream has been closed. "
|
||||
+ "Continue processing the perfect overwrite.");
|
||||
} catch (IOException e) {
|
||||
LOG.info("hsync failed when processing possible perfect overwrite, path="
|
||||
+ path + " error:" + e);
|
||||
|
@ -678,7 +678,7 @@ class OpenFileCtx {
|
|||
}
|
||||
} catch (IOException e) {
|
||||
LOG.info("Read failed when processing possible perfect overwrite, path="
|
||||
+ path + " error:" + e);
|
||||
+ path, e);
|
||||
return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow,
|
||||
Nfs3Constant.WRITE_COMMIT_VERF);
|
||||
} finally {
|
||||
|
@ -906,7 +906,7 @@ class OpenFileCtx {
|
|||
/** Invoked by AsynDataService to write back to HDFS */
|
||||
void executeWriteBack() {
|
||||
Preconditions.checkState(asyncStatus,
|
||||
"openFileCtx has false asyncStatus, fileId:" + latestAttr.getFileid());
|
||||
"openFileCtx has false asyncStatus, fileId:" + latestAttr.getFileId());
|
||||
final long startOffset = asyncWriteBackStartOffset;
|
||||
try {
|
||||
while (activeState) {
|
||||
|
@ -932,7 +932,7 @@ class OpenFileCtx {
|
|||
asyncStatus = false;
|
||||
} else {
|
||||
LOG.info("Another asyn task is already started before this one"
|
||||
+ " is finalized. fileId:" + latestAttr.getFileid()
|
||||
+ " is finalized. fileId:" + latestAttr.getFileId()
|
||||
+ " asyncStatus:" + asyncStatus + " original startOffset:"
|
||||
+ startOffset + " new startOffset:" + asyncWriteBackStartOffset
|
||||
+ ". Won't change asyncStatus here.");
|
||||
|
@ -959,11 +959,11 @@ class OpenFileCtx {
|
|||
} catch (ClosedChannelException cce) {
|
||||
if (!pendingWrites.isEmpty()) {
|
||||
LOG.error("Can't sync for fileId: " + latestAttr.getFileId()
|
||||
+ ". Channel closed with writes pending");
|
||||
+ ". Channel closed with writes pending.", cce);
|
||||
}
|
||||
status = Nfs3Status.NFS3ERR_IO;
|
||||
} catch (IOException e) {
|
||||
LOG.error("Got stream error during data sync:" + e);
|
||||
LOG.error("Got stream error during data sync:", e);
|
||||
// Do nothing. Stream will be closed eventually by StreamMonitor.
|
||||
status = Nfs3Status.NFS3ERR_IO;
|
||||
}
|
||||
|
@ -973,7 +973,7 @@ class OpenFileCtx {
|
|||
latestAttr = Nfs3Utils.getFileAttr(client,
|
||||
Nfs3Utils.getFileIdPath(latestAttr.getFileId()), iug);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Can't get new file attr for fileId: " + latestAttr.getFileId());
|
||||
LOG.error("Can't get new file attr, fileId: " + latestAttr.getFileId(), e);
|
||||
status = Nfs3Status.NFS3ERR_IO;
|
||||
}
|
||||
|
||||
|
@ -996,7 +996,7 @@ class OpenFileCtx {
|
|||
new VerifierNone()), commit.getXid());
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("FileId: " + latestAttr.getFileid() + " Service time:"
|
||||
LOG.debug("FileId: " + latestAttr.getFileId() + " Service time:"
|
||||
+ (System.currentTimeMillis() - commit.getStartTime())
|
||||
+ "ms. Sent response for commit:" + commit);
|
||||
}
|
||||
|
@ -1059,7 +1059,7 @@ class OpenFileCtx {
|
|||
fos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("hsync failed with writeCtx:" + writeCtx + " error:" + e);
|
||||
LOG.error("hsync failed with writeCtx:" + writeCtx, e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
@ -1091,7 +1091,7 @@ class OpenFileCtx {
|
|||
}
|
||||
|
||||
LOG.info("Clean up open file context for fileId: "
|
||||
+ latestAttr.getFileid());
|
||||
+ latestAttr.getFileId());
|
||||
cleanup();
|
||||
}
|
||||
}
|
||||
|
@ -1118,7 +1118,7 @@ class OpenFileCtx {
|
|||
fos.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.info("Can't close stream for fileId:" + latestAttr.getFileid()
|
||||
LOG.info("Can't close stream for fileId:" + latestAttr.getFileId()
|
||||
+ ", error:" + e);
|
||||
}
|
||||
|
||||
|
@ -1146,7 +1146,7 @@ class OpenFileCtx {
|
|||
try {
|
||||
dumpOut.close();
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
LOG.error("Failed to close outputstream of dump file" + dumpFilePath, e);
|
||||
}
|
||||
File dumpFile = new File(dumpFilePath);
|
||||
if (dumpFile.exists() && !dumpFile.delete()) {
|
||||
|
@ -1157,7 +1157,7 @@ class OpenFileCtx {
|
|||
try {
|
||||
raf.close();
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
LOG.error("Got exception when closing input stream of dump file.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -268,7 +268,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
try {
|
||||
attrs = writeManager.getFileAttr(dfsClient, handle, iug);
|
||||
} catch (IOException e) {
|
||||
LOG.info("Can't get file attribute, fileId=" + handle.getFileId());
|
||||
LOG.info("Can't get file attribute, fileId=" + handle.getFileId(), e);
|
||||
response.setStatus(Nfs3Status.NFS3ERR_IO);
|
||||
return response;
|
||||
}
|
||||
|
@ -381,7 +381,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
wccData = Nfs3Utils.createWccData(Nfs3Utils.getWccAttr(preOpAttr),
|
||||
dfsClient, fileIdPath, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.info("Can't get postOpAttr for fileIdPath: " + fileIdPath);
|
||||
LOG.info("Can't get postOpAttr for fileIdPath: " + fileIdPath, e1);
|
||||
}
|
||||
if (e instanceof AccessControlException) {
|
||||
return new SETATTR3Response(Nfs3Status.NFS3ERR_ACCES, wccData);
|
||||
|
@ -614,13 +614,11 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
// Only do access check.
|
||||
try {
|
||||
// Don't read from cache. Client may not have read permission.
|
||||
attrs = Nfs3Utils.getFileAttr(
|
||||
dfsClient,
|
||||
Nfs3Utils.getFileIdPath(handle),
|
||||
iug);
|
||||
attrs = Nfs3Utils.getFileAttr(dfsClient,
|
||||
Nfs3Utils.getFileIdPath(handle), iug);
|
||||
} catch (IOException e) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Get error accessing file, fileId:" + handle.getFileId());
|
||||
LOG.debug("Get error accessing file, fileId:" + handle.getFileId(), e);
|
||||
}
|
||||
return new READ3Response(Nfs3Status.NFS3ERR_IO);
|
||||
}
|
||||
|
@ -764,7 +762,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
try {
|
||||
postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
|
||||
LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId(), e1);
|
||||
}
|
||||
WccAttr attr = preOpAttr == null ? null : Nfs3Utils.getWccAttr(preOpAttr);
|
||||
WccData fileWcc = new WccData(attr, postOpAttr);
|
||||
|
@ -881,7 +879,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
fos.close();
|
||||
} catch (IOException e1) {
|
||||
LOG.error("Can't close stream for dirFileId:" + dirHandle.getFileId()
|
||||
+ " filename: " + fileName);
|
||||
+ " filename: " + fileName, e1);
|
||||
}
|
||||
}
|
||||
if (dirWcc == null) {
|
||||
|
@ -890,7 +888,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
dfsClient, dirFileIdPath, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.error("Can't get postOpDirAttr for dirFileId:"
|
||||
+ dirHandle.getFileId());
|
||||
+ dirHandle.getFileId(), e1);
|
||||
}
|
||||
}
|
||||
if (e instanceof AccessControlException) {
|
||||
|
@ -982,7 +980,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
try {
|
||||
postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
|
||||
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e);
|
||||
}
|
||||
}
|
||||
WccData dirWcc = new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
|
||||
|
@ -1064,7 +1062,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
try {
|
||||
postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
|
||||
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e1);
|
||||
}
|
||||
}
|
||||
WccData dirWcc = new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
|
||||
|
@ -1146,7 +1144,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
try {
|
||||
postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
|
||||
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e1);
|
||||
}
|
||||
}
|
||||
WccData dirWcc = new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
|
||||
|
@ -1236,7 +1234,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
dfsClient, toDirFileIdPath, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.info("Can't get postOpDirAttr for " + fromDirFileIdPath + " or"
|
||||
+ toDirFileIdPath);
|
||||
+ toDirFileIdPath, e1);
|
||||
}
|
||||
if (e instanceof AccessControlException) {
|
||||
return new RENAME3Response(Nfs3Status.NFS3ERR_PERM, fromDirWcc,
|
||||
|
@ -1298,7 +1296,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
.setPostOpAttr(Nfs3Utils.getFileAttr(dfsClient, linkDirIdPath, iug));
|
||||
|
||||
return new SYMLINK3Response(Nfs3Status.NFS3_OK, new FileHandle(
|
||||
objAttr.getFileid()), objAttr, dirWcc);
|
||||
objAttr.getFileId()), objAttr, dirWcc);
|
||||
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception:" + e);
|
||||
|
@ -1585,7 +1583,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
|
||||
entries[0] = new READDIRPLUS3Response.EntryPlus3(
|
||||
postOpDirAttr.getFileId(), ".", 0, postOpDirAttr, new FileHandle(
|
||||
postOpDirAttr.getFileid()));
|
||||
postOpDirAttr.getFileId()));
|
||||
entries[1] = new READDIRPLUS3Response.EntryPlus3(dotdotFileId, "..",
|
||||
dotdotFileId, postOpDirAttr, new FileHandle(dotdotFileId));
|
||||
|
||||
|
@ -1596,8 +1594,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
try {
|
||||
attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Can't get file attributes for fileId:" + fileId
|
||||
+ " error:" + e);
|
||||
LOG.error("Can't get file attributes for fileId:" + fileId, e);
|
||||
continue;
|
||||
}
|
||||
entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId,
|
||||
|
@ -1614,8 +1611,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
try {
|
||||
attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Can't get file attributes for fileId:" + fileId
|
||||
+ " error:" + e);
|
||||
LOG.error("Can't get file attributes for fileId:" + fileId, e);
|
||||
continue;
|
||||
}
|
||||
entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId,
|
||||
|
@ -1842,10 +1838,9 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
try {
|
||||
postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
|
||||
LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId(), e1);
|
||||
}
|
||||
WccData fileWcc = new WccData(preOpAttr == null ? null
|
||||
: Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
|
||||
WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
|
||||
return new COMMIT3Response(Nfs3Status.NFS3ERR_IO, fileWcc,
|
||||
Nfs3Constant.WRITE_COMMIT_VERF);
|
||||
}
|
||||
|
|
|
@ -156,7 +156,7 @@ public class WriteManager {
|
|||
}
|
||||
throw e;
|
||||
} catch (IOException e) {
|
||||
LOG.error("Can't apapend to file:" + fileIdPath + ", error:" + e);
|
||||
LOG.error("Can't apapend to file:" + fileIdPath, e);
|
||||
if (fos != null) {
|
||||
fos.close();
|
||||
}
|
||||
|
@ -181,7 +181,7 @@ public class WriteManager {
|
|||
try {
|
||||
fos.close();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Can't close stream for fileId:" + handle.getFileId());
|
||||
LOG.error("Can't close stream for fileId:" + handle.getFileId(), e);
|
||||
}
|
||||
// Notify client to retry
|
||||
WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
|
||||
|
@ -284,10 +284,10 @@ public class WriteManager {
|
|||
// Send out the response
|
||||
Nfs3FileAttributes postOpAttr = null;
|
||||
try {
|
||||
String fileIdPath = Nfs3Utils.getFileIdPath(preOpAttr.getFileid());
|
||||
String fileIdPath = Nfs3Utils.getFileIdPath(preOpAttr.getFileId());
|
||||
postOpAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
|
||||
} catch (IOException e1) {
|
||||
LOG.info("Can't get postOpAttr for fileId: " + preOpAttr.getFileid());
|
||||
LOG.info("Can't get postOpAttr for fileId: " + preOpAttr.getFileId(), e1);
|
||||
}
|
||||
WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
|
||||
COMMIT3Response response = new COMMIT3Response(status, fileWcc,
|
||||
|
|
|
@ -435,6 +435,9 @@ Release 2.4.0 - UNRELEASED
|
|||
|
||||
HDFS-6120. Fix and improve safe mode log messages. (Arpit Agarwal)
|
||||
|
||||
HDFS-6050. NFS does not handle exceptions correctly in a few places
|
||||
(brandonli)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
|
||||
|
|
Loading…
Reference in New Issue