Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-13625

Use HDFS for HFileOutputFormat2 partitioner's path

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 1.1.0, 1.2.0, 2.0.0
    • 0.98.13, 1.0.2, 1.2.0, 1.1.1, 2.0.0
    • mapreduce
    • None
    • Reviewed
    • Introduces a new config hbase.fs.tmp.dir which is a directory in HDFS (or default file system) to use as a staging directory for HFileOutputFormat2. This is also used as the default for hbase.bulkload.staging.dir

    Description

      HBASE-13010 changed hard-coded '/tmp' in HFileOutputFormat2 partitioner's path to 'hadoop.tmp.dir'. This breaks unit test in Windows.

         static void configurePartitioner(Job job, List<ImmutableBytesWritable> splitPoints)
           ...
           // create the partitions file
      -    FileSystem fs = FileSystem.get(job.getConfiguration());
      -    Path partitionsPath = new Path("/tmp", "partitions_" + UUID.randomUUID());
      +    FileSystem fs = FileSystem.get(conf);
      +    Path partitionsPath = new Path(conf.get("hadoop.tmp.dir"), "partitions_" + UUID.randomUUID());
      

      Here is the exception from 1 of the UTs when running against Windows (from branch-1.1) - The ':' is an invalid character in windows file path:

      java.lang.IllegalArgumentException: Pathname /C:/hbase-server/target/test-data/d25e2228-8959-43ee-b413-4fa69cdb8032/hadoop_tmp/partitions_fb96c0a0-41e6-4964-a391-738cb761ee3e from C:/hbase-server/target/test-data/d25e2228-8959-43ee-b413-4fa69cdb8032/hadoop_tmp/partitions_fb96c0a0-41e6-4964-a391-738cb761ee3e is not a valid DFS filename.
      	at org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:197)
      	at org.apache.hadoop.hdfs.DistributedFileSystem.access$000(DistributedFileSystem.java:106)
      	at org.apache.hadoop.hdfs.DistributedFileSystem$7.doCall(DistributedFileSystem.java:448)
      	at org.apache.hadoop.hdfs.DistributedFileSystem$7.doCall(DistributedFileSystem.java:444)
      	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
      	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:444)
      	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:387)
      	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:909)
      	at org.apache.hadoop.io.SequenceFile$Writer.<init>(SequenceFile.java:1074)
      	at org.apache.hadoop.io.SequenceFile$RecordCompressWriter.<init>(SequenceFile.java:1374)
      	at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:275)
      	at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:297)
      	at org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2.writePartitions(HFileOutputFormat2.java:335)
      	at org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2.configurePartitioner(HFileOutputFormat2.java:593)
      	at org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2.configureIncrementalLoad(HFileOutputFormat2.java:440)
      	at org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2.configureIncrementalLoad(HFileOutputFormat2.java:405)
      	at org.apache.hadoop.hbase.mapreduce.ImportTsv.createSubmittableJob(ImportTsv.java:539)
      	at org.apache.hadoop.hbase.mapreduce.ImportTsv.run(ImportTsv.java:720)
      	at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
      	at org.apache.hadoop.hbase.mapreduce.TestImportTsv.doMROnTableTest(TestImportTsv.java:313)
      	at org.apache.hadoop.hbase.mapreduce.TestImportTsv.testBulkOutputWithoutAnExistingTable(TestImportTsv.java:168)
      

      The proposed fix is to use a config to point to a hdfs directory.

      Attachments

        1. HBASE-13625.patch
          2 kB
          Stephen Yuan Jiang
        2. HBASE-13625-v2.patch
          5 kB
          Stephen Yuan Jiang

        Issue Links

          Activity

            People

              syuanjiang Stephen Yuan Jiang
              syuanjiang Stephen Yuan Jiang
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: