HBASE-14520 Optimize the number of calls for tags creation in bulk load (Bhupendra Kumar Jain)

This commit is contained in:
tedyu 2015-10-06 06:43:30 -07:00
parent ed4c734b15
commit 23079c02bf
2 changed files with 29 additions and 21 deletions

View File

@ -158,7 +158,18 @@ public class TextSortReducer extends
ts = parsed.getTimestamp(ts);
cellVisibilityExpr = parsed.getCellVisibility();
ttl = parsed.getCellTTL();
// create tags for the parsed line
List<Tag> tags = new ArrayList<Tag>();
if (cellVisibilityExpr != null) {
tags.addAll(kvCreator.getVisibilityExpressionResolver().createVisibilityExpTags(
cellVisibilityExpr));
}
// Add TTL directly to the KV so we can vary them when packing more than one KV
// into puts
if (ttl > 0) {
tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
}
for (int i = 0; i < parsed.getColumnCount(); i++) {
if (i == parser.getRowKeyColumnIndex() || i == parser.getTimestampKeyColumnIndex()
|| i == parser.getAttributesKeyColumnIndex() || i == parser.getCellVisibilityColumnIndex()
@ -167,16 +178,6 @@ public class TextSortReducer extends
}
// Creating the KV which needs to be directly written to HFiles. Using the Facade
// KVCreator for creation of kvs.
List<Tag> tags = new ArrayList<Tag>();
if (cellVisibilityExpr != null) {
tags.addAll(kvCreator.getVisibilityExpressionResolver()
.createVisibilityExpTags(cellVisibilityExpr));
}
// Add TTL directly to the KV so we can vary them when packing more than one KV
// into puts
if (ttl > 0) {
tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
}
Cell cell = this.kvCreator.create(lineBytes, parsed.getRowKeyOffset(),
parsed.getRowKeyLength(), parser.getFamily(i), 0, parser.getFamily(i).length,
parser.getQualifier(i), 0, parser.getQualifier(i).length, ts, lineBytes,

View File

@ -71,6 +71,9 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
private String hfileOutPath;
/** List of cell tags */
private List<Tag> tags;
public long getTs() {
return ts;
}
@ -106,6 +109,7 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
throw new RuntimeException("No row key column specified");
}
this.kvCreator = new CellCreator(conf);
tags = new ArrayList<Tag>();
}
/**
@ -155,6 +159,19 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
cellVisibilityExpr = parsed.getCellVisibility();
ttl = parsed.getCellTTL();
// create tags for the parsed line
if (hfileOutPath != null) {
tags.clear();
if (cellVisibilityExpr != null) {
tags.addAll(kvCreator.getVisibilityExpressionResolver().createVisibilityExpTags(
cellVisibilityExpr));
}
// Add TTL directly to the KV so we can vary them when packing more than one KV
// into puts
if (ttl > 0) {
tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
}
}
Put put = new Put(rowKey.copyBytes());
for (int i = 0; i < parsed.getColumnCount(); i++) {
if (i == parser.getRowKeyColumnIndex() || i == parser.getTimestampKeyColumnIndex()
@ -199,16 +216,6 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
} else {
// Creating the KV which needs to be directly written to HFiles. Using the Facade
// KVCreator for creation of kvs.
List<Tag> tags = new ArrayList<Tag>();
if (cellVisibilityExpr != null) {
tags.addAll(kvCreator.getVisibilityExpressionResolver()
.createVisibilityExpTags(cellVisibilityExpr));
}
// Add TTL directly to the KV so we can vary them when packing more than one KV
// into puts
if (ttl > 0) {
tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
}
cell = this.kvCreator.create(lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
parser.getFamily(i), 0, parser.getFamily(i).length, parser.getQualifier(i), 0,
parser.getQualifier(i).length, ts, lineBytes, parsed.getColumnOffset(i),