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

smarthan commented on HADOOP-11794:
-----------------------------------

Hi [~yzhangal], @I have a doubt that whether there is a bug in 
RetriableFileCopyCommand.copyBytes( ) when enable blocksperchunk .

 
{code:java}
 int bytesRead = readBytes(inStream, buf, sourceOffset);
 while (bytesRead >= 0) {
 if (chunkLength > 0 &&
     (totalBytesRead + bytesRead) >= chunkLength) {
   bytesRead = (int)(chunkLength - totalBytesRead);
   finished = true;
 }
 totalBytesRead += bytesRead;
 //action == FileAction.APPEND always false when blocksperchunk >= 1,
 //the sourceOffset will never been change
 //readBytes(inStream, buf, sourceOffset) will always start from the same 
position 'sourceOffset'
 //so the buf[] always read same data, and result in wrong split copy finally
 if (action == FileAction.APPEND) {
   sourceOffset += bytesRead;
 }
 outStream.write(buf, 0, bytesRead);
 updateContextStatus(totalBytesRead, context, source2);
 if (finished) {
   break;
 }
 bytesRead = readBytes(inStream, buf, sourceOffset);
 }
{code}
 


I merge this patch to branch cdh5.10, and when I set blocksperchunk >= 1, the 
copy of data will be different form source.
I debug and find this problem, and when I modify the condition as follow, it 
works. 
{code:java}
if (action == FileAction.APPEND || (source2.isSplit() && sourceOffset > 0)) {
  sourceOffset += bytesRead;
}{code}
 

I am not sure whether it is a bug or just caused by version, please look at it 
if possible, thanks.

 

> Enable distcp to copy blocks in parallel
> ----------------------------------------
>
>                 Key: HADOOP-11794
>                 URL: https://issues.apache.org/jira/browse/HADOOP-11794
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: tools/distcp
>    Affects Versions: 0.21.0
>            Reporter: dhruba borthakur
>            Assignee: Yongjun Zhang
>            Priority: Major
>             Fix For: 2.9.0, 3.0.0-alpha4
>
>         Attachments: HADOOP-11794.001.patch, HADOOP-11794.002.patch, 
> HADOOP-11794.003.patch, HADOOP-11794.004.patch, HADOOP-11794.005.patch, 
> HADOOP-11794.006.patch, HADOOP-11794.007.patch, HADOOP-11794.008.patch, 
> HADOOP-11794.009.patch, HADOOP-11794.010.branch2.002.patch, 
> HADOOP-11794.010.branch2.patch, HADOOP-11794.010.patch, MAPREDUCE-2257.patch
>
>
> The minimum unit of work for a distcp task is a file. We have files that are 
> greater than 1 TB with a block size of  1 GB. If we use distcp to copy these 
> files, the tasks either take a long long long time or finally fails. A better 
> way for distcp would be to copy all the source blocks in parallel, and then 
> stich the blocks back to files at the destination via the HDFS Concat API 
> (HDFS-222)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to