[ https://issues.apache.org/jira/browse/HIVE-19248?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Sankar Hariappan updated HIVE-19248: ------------------------------------ Description: Hive replication uses Hadoop distcp to copy files from primary to replica warehouse. If the HDFS block size is different across clusters, it cause file copy failures. {code:java} 2018-04-09 14:32:06,690 ERROR [main] org.apache.hadoop.tools.mapred.CopyMapper: Failure in copying hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 to hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 java.io.IOException: File copy failed: hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 --> hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 at org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:299) at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:266) at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:52) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:146) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:787) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341) at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:170) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1869) at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:164) Caused by: java.io.IOException: Couldn't run retriable-command: Copying hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 to hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 at org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:101) at org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:296) ... 10 more Caused by: java.io.IOException: Check-sum mismatch between hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 and hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/.distcp.tmp.attempt_1522833620762_4416_m_000000_0. Source and target differ in block-size. Use -pb to preserve block-sizes during copy. Alternatively, skip checksum-checks altogether, using -skipCrc. (NOTE: By skipping checksums, one runs the risk of masking data-corruption during file-transfer.) at org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.compareCheckSums(RetriableFileCopyCommand.java:212) at org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doCopy(RetriableFileCopyCommand.java:130) at org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doExecute(RetriableFileCopyCommand.java:99) at org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:87) ... 11 more {code} Distcp failed as the CM path for the file doesn't point to source file system. So, it is needed to get the qualified cm root URI as part of files listed in dump. Also, REPL LOAD returns success even if distcp jobs failed. CopyUtils.doCopyRetry doesn't throw error if copy failed even after maximum attempts. So, need to perform 2 things. # If copy of multiple files fail for some reason, then retry with same set of files again but need to set CM path if original source file is missing or modified based on checksum. Let distcp to skip the properly copied files. FileUtil.copy will always overwrite the files. # If source path is moved to CM path, then delete the incorrectly copied files. # If copy fails for maximum attempt, then throw error. was: Hive replication uses Hadoop distcp to copy files from primary to replica warehouse. If the HDFS block size is different across clusters, it cause file copy failures. {code:java} 2018-04-09 14:32:06,690 ERROR [main] org.apache.hadoop.tools.mapred.CopyMapper: Failure in copying hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 to hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 java.io.IOException: File copy failed: hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 --> hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 at org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:299) at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:266) at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:52) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:146) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:787) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341) at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:170) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1869) at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:164) Caused by: java.io.IOException: Couldn't run retriable-command: Copying hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 to hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 at org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:101) at org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:296) ... 10 more Caused by: java.io.IOException: Check-sum mismatch between hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 and hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/.distcp.tmp.attempt_1522833620762_4416_m_000000_0. Source and target differ in block-size. Use -pb to preserve block-sizes during copy. Alternatively, skip checksum-checks altogether, using -skipCrc. (NOTE: By skipping checksums, one runs the risk of masking data-corruption during file-transfer.) at org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.compareCheckSums(RetriableFileCopyCommand.java:212) at org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doCopy(RetriableFileCopyCommand.java:130) at org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doExecute(RetriableFileCopyCommand.java:99) at org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:87) ... 11 more {code} But REPL LOAD returns success even if distcp jobs failed. CopyUtils.doCopyRetry doesn't throw error if copy failed even after maximum attempts. So, need to perform 2 things. # If copy of multiple files fail for some reason, need to check if any files completely copied by verifying the checksum and file size and skip those from retry. # If source path is moved to CM path, then delete the incorrectly copied files. # If copy fails for maximum attempt, then throw error. > REPL LOAD doesn't copy file from source CM path and also doesn't throw error > if file copy fails. > ------------------------------------------------------------------------------------------------ > > Key: HIVE-19248 > URL: https://issues.apache.org/jira/browse/HIVE-19248 > Project: Hive > Issue Type: Bug > Components: HiveServer2, repl > Affects Versions: 3.0.0 > Reporter: Sankar Hariappan > Assignee: Sankar Hariappan > Priority: Major > Labels: DR, pull-request-available, replication > Fix For: 3.1.0 > > > Hive replication uses Hadoop distcp to copy files from primary to replica > warehouse. If the HDFS block size is different across clusters, it cause file > copy failures. > {code:java} > 2018-04-09 14:32:06,690 ERROR [main] > org.apache.hadoop.tools.mapred.CopyMapper: Failure in copying > hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 to > hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 > java.io.IOException: File copy failed: > hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 > --> > hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 > at > org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:299) > at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:266) > at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:52) > at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:146) > at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:787) > at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341) > at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:170) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1869) > at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:164) > Caused by: java.io.IOException: Couldn't run retriable-command: Copying > hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 to > hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/000259_0 > at > org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:101) > at > org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:296) > ... 10 more > Caused by: java.io.IOException: Check-sum mismatch between > hdfs://chelsea/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/000259_0 > and > hdfs://marilyn/apps/hive/warehouse/tpch_flat_orc_1000.db/customer/.hive-staging_hive_2018-04-09_14-30-45_723_7153496419225102220-2/-ext-10001/.distcp.tmp.attempt_1522833620762_4416_m_000000_0. > Source and target differ in block-size. Use -pb to preserve block-sizes > during copy. Alternatively, skip checksum-checks altogether, using -skipCrc. > (NOTE: By skipping checksums, one runs the risk of masking data-corruption > during file-transfer.) > at > org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.compareCheckSums(RetriableFileCopyCommand.java:212) > at > org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doCopy(RetriableFileCopyCommand.java:130) > at > org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doExecute(RetriableFileCopyCommand.java:99) > at > org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:87) > ... 11 more > {code} > Distcp failed as the CM path for the file doesn't point to source file > system. So, it is needed to get the qualified cm root URI as part of files > listed in dump. > Also, REPL LOAD returns success even if distcp jobs failed. > CopyUtils.doCopyRetry doesn't throw error if copy failed even after maximum > attempts. > So, need to perform 2 things. > # If copy of multiple files fail for some reason, then retry with same set > of files again but need to set CM path if original source file is missing or > modified based on checksum. Let distcp to skip the properly copied files. > FileUtil.copy will always overwrite the files. > # If source path is moved to CM path, then delete the incorrectly copied > files. > # If copy fails for maximum attempt, then throw error. > -- This message was sent by Atlassian JIRA (v7.6.3#76005)