[ https://issues.apache.org/jira/browse/HBASE-10902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13960879#comment-13960879 ]
Jerry He commented on HBASE-10902: ---------------------------------- Attached a patch for 0.96. Tested to work in both local cluster and across secure clusters. Please review. I will do more cleanup if necessary and add patch for trunk. > Make Secure Bulk Load work across remote secure clusters > -------------------------------------------------------- > > Key: HBASE-10902 > URL: https://issues.apache.org/jira/browse/HBASE-10902 > Project: HBase > Issue Type: Improvement > Affects Versions: 0.96.1 > Reporter: Jerry He > Assignee: Jerry He > Attachments: HBASE-10902-v0-0.96.patch > > > Two secure clusters, both with kerberos enabled. > Run bulk load on one cluster to load files from another cluster. > biadmin@hdtest249:~> hbase > org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles > hdfs://bdvm197.svl.ibm.com:9000/user/biadmin/mybackups/TestTable/0709e79bb131af13ed088bf1afd5649c > TestTable_rr > Bulk load failed. In the region server log: > {code} > 2014-04-02 20:04:56,361 ERROR > org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint: Failed to > complete bulk load > java.lang.IllegalArgumentException: Wrong FS: > hdfs://bdvm197.svl.ibm.com:9000/user/biadmin/mybackups/TestTable/0709e79bb131af13ed088bf1afd5649c/info/6b44ca48aebf48d98cb3491f512c41a7, > expected: hdfs://hdtest249.svl.ibm.com:9000 > at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:651) > at > org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:181) > at > org.apache.hadoop.hdfs.DistributedFileSystem.access$000(DistributedFileSystem.java:92) > at > org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1248) > at > org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1244) > at > org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) > at > org.apache.hadoop.hdfs.DistributedFileSystem.setPermission(DistributedFileSystem.java:1244) > at > org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint$1.run(SecureBulkLoadEndpoint.java:233) > at > org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint$1.run(SecureBulkLoadEndpoint.java:223) > at > java.security.AccessController.doPrivileged(AccessController.java:300) > at javax.security.auth.Subject.doAs(Subject.java:494) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1482) > at > org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint.secureBulkLoadHFiles(SecureBulkLoadEndpoint.java:223) > at > org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos$SecureBulkLoadService.callMethod(SecureBulkLoadProtos.java:4631) > at > org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:5088) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.execService(HRegionServer.java:3219) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:26933) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at > org.apache.hadoop.hbase.ipc.RpcServer$Handler.run(RpcServer.java:1854) > {code} -- This message was sent by Atlassian JIRA (v6.2#6252)