Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
2.0.0
-
None
-
Hadoop 3.0.0 + HBase 2.0.0, Kerberos.
-
Reviewed
Description
HBASE-15806 implemented an endpoint based export. It gets caller's HDFS delegation token, and RegionServer is supposed to write out exported files as the caller.
Everything works fine if you use run export as hbase user. However, once you use a different user to export, it fails.
To reproduce,
Add to configuration key hbase.coprocessor.region.classes the coprocessor class org.apache.hadoop.hbase.coprocessor.Export.
create a table t1, assign permission to a user foo:
hbase(main):004:0> user_permission 't1' User Namespace,Table,Family,Qualifier:Permission hbase default,t1,,: [Permission: actions=READ,WRITE,EXEC,CREATE,ADMIN] foo default,t1,,: [Permission: actions=READ,WRITE,EXEC,CREATE,ADMIN]
As user foo, execute the following command:
$ hdfs dfs -mkdir /tmp/export_hbase2 $ hbase org.apache.hadoop.hbase.coprocessor.Export t1 /tmp/export_hbase2/t2/ .... 18/07/10 14:03:59 INFO client.RpcRetryingCallerImpl: Call exception, tries=6, retries=6, started=4457 ms ago, cancelled=false, msg=org.apache.hadoop.security.AccessControlException: Permission denied: user=hbase, access=WRITE, inode="/tmp/export_hbase2/t2":foo:supergroup:drwxr-xr-x at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:400) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:256) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:194) at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkPermission(FSDirectory.java:1846) at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkPermission(FSDirectory.java:1830) at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkAncestorAccess(FSDirectory.java:1789) at org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.resolvePathForStartFile(FSDirWriteFileOp.java:316) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2411) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:2343) at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:764) at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:451) at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java) at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:523) at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:991) at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:869) at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:815) 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:1685) at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2675) at sun.reflect.GeneratedConstructorAccessor25.newInstance(Unknown Source) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:121) at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:88) at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:278) at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1195) at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1174) at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1112) at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:462) at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:459) at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:473) at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:400) at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1103) at org.apache.hadoop.io.SequenceFile$Writer.<init>(SequenceFile.java:1168) at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:285) at org.apache.hadoop.hbase.coprocessor.Export$SecureWriter.<init>(Export.java:445) at org.apache.hadoop.hbase.coprocessor.Export.processData(Export.java:214) at org.apache.hadoop.hbase.coprocessor.Export.export(Export.java:338) at org.apache.hadoop.hbase.protobuf.generated.ExportProtos$ExportService.callMethod(ExportProtos.java:2030) at org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:8007) at org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(RSRpcServices.java:2409) at org.apache.hadoop.hbase.regionserver.RSRpcServices.execService(RSRpcServices.java:2391) at org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:42010) at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:409) at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:130) at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:324) at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:304)
This is not a problem in unit test, because HBase runs as the same user as HDFS superuser, so it always has the permission to write.
I have a fix and a test code. chia7712 could you help review?
The doc could use some help too. Not very easy to follow for first time user. I'll file a doc Jira for that.
Attachments
Attachments
Issue Links
- relates to
-
HBASE-20873 Update doc for Endpoint-based Export
- Resolved