You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "John Doe (JIRA)" <ji...@apache.org> on 2018/04/27 19:06:00 UTC
[jira] [Created] (HADOOP-15425) CopyFilesMapper.doCopyFile hangs
with misconfigured sizeBuf
John Doe created HADOOP-15425:
---------------------------------
Summary: CopyFilesMapper.doCopyFile hangs with misconfigured sizeBuf
Key: HADOOP-15425
URL: https://issues.apache.org/jira/browse/HADOOP-15425
Project: Hadoop Common
Issue Type: Bug
Components: tools
Affects Versions: 2.5.0
Reporter: John Doe
When the sizeBuf is configured to be 0, the for loop in DistCpV1$CopyFilesMapper.doCopyFile function hangs endlessly.
This is because when the buf.size (i.e., sizeBuf) is 0, the bytesRead will always be 0 by invoking bytesRead=in.read(buf).
Here is the code snippet.
{code:java}
sizeBuf = job.getInt("copy.buf.size", 128 * 1024); //when copy.buf.size = 0
buffer = new byte[sizeBuf];
private long doCopyFile(FileStatus srcstat, Path tmpfile, Path absdst, Reporter reporter) throws IOException {
FSDataInputStream in = null;
FSDataOutputStream out = null;
long bytesCopied = 0L;
try {
Path srcPath = srcstat.getPath();
// open src file
in = srcPath.getFileSystem(job).open(srcPath);
reporter.incrCounter(Counter.BYTESEXPECTED, srcstat.getLen());
// open tmp file
out = create(tmpfile, reporter, srcstat);
LOG.info("Copying file " + srcPath + " of size " + srcstat.getLen() + " bytes...");
// copy file
for(int bytesRead; (bytesRead = in.read(buffer)) >= 0; ) {
out.write(buffer, 0, bytesRead);
bytesCopied += bytesRead;
reporter.setStatus(... );
}
} finally {
checkAndClose(in);
checkAndClose(out);
}
return bytesCopied;
}
{code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org