You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "zhengyang jia (Jira)" <ji...@apache.org> on 2022/11/01 08:09:00 UTC

[jira] [Created] (HBASE-27457) Tags cannot be written to the hfile during bulkload

zhengyang jia created HBASE-27457:
-------------------------------------

             Summary: Tags cannot be written to the hfile during bulkload
                 Key: HBASE-27457
                 URL: https://issues.apache.org/jira/browse/HBASE-27457
             Project: HBase
          Issue Type: Bug
            Reporter: zhengyang jia


When I store hdfs file data into hbase through bulkload, I want to give different data different ttl, so I use tags in mapper to create per cell ttl for each cell, but I found that this ttl does not take effect.

I have the bulkload driver

 
{code:java}
job.setInputFormatClass(TextInputFormat.class);
TextInputFormat.setInputPaths(job, inputFile);
job.setOutputFormatClass(HFileOutputFormat2.class);
HFileOutputFormat2.setOutputPath(job, outputPath);
HFileOutputFormat2.configureIncrementalLoad(job, table, conn.getRegionLocator(TableName.valueOf(tableName)));
job.waitForCompletion(true);
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
loader.doBulkLoad(outputPath, admin, table, conn.getRegionLocator(TableName.valueOf(tableName))); {code}
 

and the mapper

 
{code:java}
public class BulkloadTTLTestMapper extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put> {
  public void map(LongWritable key, Text value, Mapper.Context context) throws IOException, InterruptedException {
    String values = value.toString();
    String[] lines = values.split(",");
    byte[] rowkey = Bytes.toBytes(lines[0]);
    ImmutableBytesWritable putRowKey = new ImmutableBytesWritable(rowkey);
    Put put = new Put(rowkey);
    Cell cell1 = new KeyValue(Bytes.toBytes(lines[0]), Bytes.toBytes("d"), Bytes.toBytes("field1"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes(lines[1]),new Tag[]{new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(100000L))});
    put.add(cell1);
    Cell cell2 = new KeyValue(Bytes.toBytes(lines[0]), Bytes.toBytes("d"), Bytes.toBytes("field2"), HConstants.LATEST_TIMESTAMP, generateValue(lines[2]),new Tag[]{new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(200000L))});
    put.add(cell2);
    context.write(putRowKey, put);
  }
} {code}
 

I checked the execution steps through debug, and I found that the tags were not written into hfile. By reading the source code, I found that in org.apache.hadoop.hbase.protobuf.ProtobufUtil.toMutation()

 
{code:java}
if (cell.getTagsLength() > 0) {
   valueBuilder.setTags(ByteStringer.wrap(cell.getTagsArray(),
       cell.getTagsOffset(), cell.getTagsLength()));
}{code}
 

This code was deleted at [issue15198|https://issues.apache.org/jira/browse/HBASE-15198], which made it impossible for me to write tags into hfile.

If i just add 
{code:java}
conf.set("hbase.client.rpc.codec", "org.apache.hadoop.hbase.codec.CellCodecWithTags"){code}
It is Useless, because bulkload does not use these Codecs.

Only when I add back those two lines of code that were deleted at issue15198, I could write tags into hfile and make per cell ttl take effect.

So I want to know is there any other way to help me carry tags in bulkload besides adding these two lines of code back.

Thanks.

 

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)