hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Ted Yu (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (HBASE-15966) Bulk load unable to read HFiles from different filesystem type than fs.defaultFS
Date Wed, 08 Jun 2016 21:25:21 GMT

     [ https://issues.apache.org/jira/browse/HBASE-15966?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Ted Yu updated HBASE-15966:
---------------------------
    Attachment: 15966.v1.txt

Something like this ?

> Bulk load unable to read HFiles from different filesystem type than fs.defaultFS
> --------------------------------------------------------------------------------
>
>                 Key: HBASE-15966
>                 URL: https://issues.apache.org/jira/browse/HBASE-15966
>             Project: HBase
>          Issue Type: Bug
>          Components: hbase, HFile
>    Affects Versions: 0.98.4
>         Environment: Microsoft Azure HDInsight 3.2 cluster with eight hosts
> - Ubuntu 12.04.5
> - HDP 2.2
> - Hadoop 2.6.0
> - HBase 0.98.4
>            Reporter: Dustin Christmann
>         Attachments: 15966.v1.txt
>
>
> In a YARN job, I am creating HFiles with code that has been cribbed from the TableOutputFormat
class and bulkloading them with LoadIncrementalHFiles.doBulkLoad.
> On other clusters, where fs.defaultFS is set to an hdfs: URI, and my HFiles are placed
in an hdfs: URI, the bulkload works as intended.
> On this particular cluster, where fs.defaultFS is set to a wasb: URI and my HFiles are
placed in a wasb: URI, the bulkload also works as intended.
> However, on this same cluster, whenever I place the HFiles in an hdfs: URI, I get the
following logs in my application from the HBase client logging repeatedly:
> [02 Jun 23:23:26.002](20259/140062246807296) Info2:org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles:
Trying to load hfile=hdfs://[my cluster]/[my path] first=\x00\x00\x11\x06 last=;\x8B\x85\x18
> [02 Jun 23:23:26.002](20259/140062245754624) Info3:org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles:
Going to connect to server region=[my namespace]:[my table],,1464909723920.00eafdb73989312bd8864f0913255f50.,
hostname=10.0.1.6,16020,1464698786237, seqNum=2 for row  with hfile group [{[B@4d0409e7,hdfs://[my
cluster]/[my path]}]
> [02 Jun 23:23:26.012](20259/140062245754624) Info1:org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles:
Attempt to bulk load region containing  into table [my namespace]:[my table] with files [family:[my
family] path:hdfs://[my cluster]/[my path]] failed.  This is recoverable and they will be
retried.
> [02 Jun 23:23:26.019](20259/140061634982912) Info2:org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles:
Split occured while grouping HFiles, retry attempt 2 with 1 files remaining to group or split
> And when I look at the appropriate region server's log, I find the following exception
repeatedly:
> 2016-06-02 20:22:50,771 ERROR [B.DefaultRpcServer.handler=22,queue=2,port=16020] access.SecureBulkLoadEndpoint:
Failed to complete bulk load
> java.io.FileNotFoundException: File doesn't exist: hdfs://[my cluster]/[my path]    
 at org.apache.hadoop.fs.azure.NativeAzureFileSystem.setPermission(NativeAzureFileSystem.java:2192)
>        at org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint$1.run(SecureBulkLoadEndpoint.java:280)
>        at org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint$1.run(SecureBulkLoadEndpoint.java:270)
>        at java.security.AccessController.doPrivileged(Native Method)
>        at javax.security.auth.Subject.doAs(Subject.java:356)
>        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1651)
>        at org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint.secureBulkLoadHFiles(SecureBulkLoadEndpoint.java:270)
>        at org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos$SecureBulkLoadService.callMethod(SecureBulkLoadProtos.java:4631)
>        at org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:6986)
>        at org.apache.hadoop.hbase.regionserver.HRegionServer.execServiceOnRegion(HRegionServer.java:3456)
>        at org.apache.hadoop.hbase.regionserver.HRegionServer.execService(HRegionServer.java:3438)
>        at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:29998)
>        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2080)
>        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:108)
>        at org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:114)
>        at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:94)
>        at java.lang.Thread.run(Thread.java:745)
> Looking at the appropriate code in SecureBulkLoadEndpoint.java, I'm finding the following:
>         public Boolean run() {
>           FileSystem fs = null;
>          try {
>             Configuration conf = env.getConfiguration();
>             fs = FileSystem.get(conf);
>             for(Pair<byte[], String> el: familyPaths) {
>               Path p = new Path(el.getSecond());
>               Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
>               if(!fs.exists(stageFamily)) {
>                 fs.mkdirs(stageFamily);
>                 fs.setPermission(stageFamily, PERM_ALL_ACCESS);
>               }
>             }
> The call to FileSystem.get is obviously the culprit, since it gets the FileSystem object
based on fs.defaultFS, which is suboptimal in this case and other cases where the HFiles are
located on a different type of filesystem than the defaultFS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message