Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
None
-
None
-
None
-
Reviewed
Description
I was trying out restore feature and encountered the following exception:
2016-06-10 16:56:57,533 ERROR [main] impl.RestoreClientImpl: ERROR: restore failed with error: java.io.IOException: Can not restore from backup directory hdfs://hbase-test-rc-rerun-6:8020/user/hbase/backup_1465575766499/WALs (check Hadoop and HBase logs) java.io.IOException: java.io.IOException: Can not restore from backup directory hdfs://hbase-test-rc-rerun-6:8020/user/hbase/backup_1465575766499/WALs (check Hadoop and HBase logs) at org.apache.hadoop.hbase.backup.impl.RestoreClientImpl.restoreStage(RestoreClientImpl.java:257) at org.apache.hadoop.hbase.backup.impl.RestoreClientImpl.restore(RestoreClientImpl.java:112) at org.apache.hadoop.hbase.backup.RestoreDriver.parseAndRun(RestoreDriver.java:169) at org.apache.hadoop.hbase.backup.RestoreDriver.doWork(RestoreDriver.java:198) at org.apache.hadoop.hbase.util.AbstractHBaseTool.run(AbstractHBaseTool.java:112) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at org.apache.hadoop.hbase.backup.RestoreDriver.main(RestoreDriver.java:203) Caused by: java.io.IOException: Can not restore from backup directory hdfs://hbase-test-rc-rerun-6:8020/user/hbase/backup_1465575766499/WALs (check Hadoop and HBase logs) at org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreService.run(MapReduceRestoreService.java:92) at org.apache.hadoop.hbase.backup.util.RestoreServerUtil.incrementalRestoreTable(RestoreServerUtil.java:165) at org.apache.hadoop.hbase.backup.impl.RestoreClientImpl.restoreImage(RestoreClientImpl.java:293) at org.apache.hadoop.hbase.backup.impl.RestoreClientImpl.restoreStage(RestoreClientImpl.java:238) ... 6 more Caused by: java.lang.IllegalArgumentException: Wrong FS: hdfs://hbase-test-rc-rerun-6:8020/user/hbase/backup_1465575766499/WALs, expected: hdfs://hbase-test-rc-rerun-6.openstacklocal:8020 at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:658) at org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:212) at org.apache.hadoop.hdfs.DistributedFileSystem.listStatusInternal(DistributedFileSystem.java:882) at org.apache.hadoop.hdfs.DistributedFileSystem.access$600(DistributedFileSystem.java:112) at org.apache.hadoop.hdfs.DistributedFileSystem$21.doCall(DistributedFileSystem.java:951) at org.apache.hadoop.hdfs.DistributedFileSystem$21.doCall(DistributedFileSystem.java:947) at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) at org.apache.hadoop.hdfs.DistributedFileSystem.listStatus(DistributedFileSystem.java:947) at org.apache.hadoop.hbase.mapreduce.WALInputFormat.getFiles(WALInputFormat.java:266) at org.apache.hadoop.hbase.mapreduce.WALInputFormat.getSplits(WALInputFormat.java:246) at org.apache.hadoop.hbase.mapreduce.WALInputFormat.getSplits(WALInputFormat.java:227) at org.apache.hadoop.mapreduce.JobSubmitter.writeNewSplits(JobSubmitter.java:301) at org.apache.hadoop.mapreduce.JobSubmitter.writeSplits(JobSubmitter.java:318) at org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:196) at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1290) at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1287) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:415) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724) at org.apache.hadoop.mapreduce.Job.submit(Job.java:1287) at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:1308) at org.apache.hadoop.hbase.mapreduce.WALPlayer.run(WALPlayer.java:380) at org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreService.run(MapReduceRestoreService.java:73) ... 9 more
It turned out that the refactoring from HBASE-14140 changed the code:
- FileSystem fs = inputDir.getFileSystem(conf); - List<FileStatus> files = getFiles(fs, inputDir, startTime, endTime); - - List<InputSplit> splits = new ArrayList<InputSplit>(files.size()); - for (FileStatus file : files) { + FileSystem fs = FileSystem.get(conf);
We shouldn't be using the default FileSystem.
Instead, FileSystem should be obtained from specified path.