[
https://issues.apache.org/jira/browse/HADOOP-15850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16656170#comment-16656170
]
Wei-Chiu Chuang commented on HADOOP-15850:
------------------------------------------
I think the fix makes sense. In addition, you should remove the following line
in TestCopyCommitter in order to test the fix. (Comment out the line, the tests
fail without the fix)
{code:title=TestCopyCommitter.java}
// Unset listing file path since the config is shared by
// multiple tests, and some test doesn't set it, such as
// testNoCommitAction, but the distcp code will check it.
config.set(DistCpConstants.CONF_LABEL_LISTING_FILE_PATH, "");
{code}
> CopyCommitter#concatFileChunks should check that the blocks per chunk is not 0
> ------------------------------------------------------------------------------
>
> Key: HADOOP-15850
> URL: https://issues.apache.org/jira/browse/HADOOP-15850
> Project: Hadoop Common
> Issue Type: Bug
> Components: tools/distcp
> Affects Versions: 3.1.1
> Reporter: Ted Yu
> Assignee: Ted Yu
> Priority: Major
> Attachments: HADOOP-15850.v2.patch, HADOOP-15850.v3.patch,
> HADOOP-15850.v4.patch, testIncrementalBackupWithBulkLoad-output.txt
>
>
> I was investigating test failure of TestIncrementalBackupWithBulkLoad from
> hbase against hadoop 3.1.1
> hbase MapReduceBackupCopyJob$BackupDistCp would create listing file:
> {code}
> LOG.debug("creating input listing " + listing + " , totalRecords=" +
> totalRecords);
> cfg.set(DistCpConstants.CONF_LABEL_LISTING_FILE_PATH, listing);
> cfg.setLong(DistCpConstants.CONF_LABEL_TOTAL_NUMBER_OF_RECORDS,
> totalRecords);
> {code}
> For the test case, two bulk loaded hfiles are in the listing:
> {code}
> 2018-10-13 14:09:24,123 DEBUG [Time-limited test]
> mapreduce.MapReduceBackupCopyJob$BackupDistCp(195): BackupDistCp :
> hdfs://localhost:42796/user/hbase/test-data/160aeab5-6bca-9f87-465e-2517a0c43119/data/default/test-1539439707496/96b5a3613d52f4df1ba87a1cef20684c/f/394e6d39a9b94b148b9089c4fb967aad_SeqId_205_
> 2018-10-13 14:09:24,125 DEBUG [Time-limited test]
> mapreduce.MapReduceBackupCopyJob$BackupDistCp(195): BackupDistCp :
> hdfs://localhost:42796/user/hbase/test-data/160aeab5-6bca-9f87-465e-2517a0c43119/data/default/test-1539439707496/96b5a3613d52f4df1ba87a1cef20684c/f/a7599081e835440eb7bf0dd3ef4fd7a5_SeqId_205_
> 2018-10-13 14:09:24,125 DEBUG [Time-limited test]
> mapreduce.MapReduceBackupCopyJob$BackupDistCp(197): BackupDistCp execute for
> 2 files of 10242
> {code}
> Later on, CopyCommitter#concatFileChunks would throw the following exception:
> {code}
> 2018-10-13 14:09:25,351 WARN [Thread-936] mapred.LocalJobRunner$Job(590):
> job_local1795473782_0004
> java.io.IOException: Inconsistent sequence file: current chunk file
> org.apache.hadoop.tools.CopyListingFileStatus@bb8826ee{hdfs://localhost:42796/user/hbase/test-data/
>
> 160aeab5-6bca-9f87-465e-2517a0c43119/data/default/test-1539439707496/96b5a3613d52f4df1ba87a1cef20684c/f/a7599081e835440eb7bf0dd3ef4fd7a5_SeqId_205_
> length = 5100 aclEntries = null, xAttrs = null} doesnt match prior entry
> org.apache.hadoop.tools.CopyListingFileStatus@243d544d{hdfs://localhost:42796/user/hbase/test-data/160aeab5-6bca-9f87-465e-
>
> 2517a0c43119/data/default/test-1539439707496/96b5a3613d52f4df1ba87a1cef20684c/f/394e6d39a9b94b148b9089c4fb967aad_SeqId_205_
> length = 5142 aclEntries = null, xAttrs = null}
> at
> org.apache.hadoop.tools.mapred.CopyCommitter.concatFileChunks(CopyCommitter.java:276)
> at
> org.apache.hadoop.tools.mapred.CopyCommitter.commitJob(CopyCommitter.java:100)
> at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:567)
> {code}
> The above warning shouldn't happen - the two bulk loaded hfiles are
> independent.
> From the contents of the two CopyListingFileStatus instances, we can see that
> their isSplit() return false. Otherwise the following from toString should be
> logged:
> {code}
> if (isSplit()) {
> sb.append(", chunkOffset = ").append(this.getChunkOffset());
> sb.append(", chunkLength = ").append(this.getChunkLength());
> }
> {code}
> From hbase side, we can specify one bulk loaded hfile per job but that
> defeats the purpose of using DistCp.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]