From 23079c02bf40c318fff4f77fa9182ebdfb230e90 Mon Sep 17 00:00:00 2001 From: tedyu Date: Tue, 6 Oct 2015 06:43:30 -0700 Subject: [PATCH] HBASE-14520 Optimize the number of calls for tags creation in bulk load (Bhupendra Kumar Jain) --- .../hbase/mapreduce/TextSortReducer.java | 23 ++++++++-------- .../hbase/mapreduce/TsvImporterMapper.java | 27 ++++++++++++------- 2 files changed, 29 insertions(+), 21 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java index 5056f0bf2a8..46e69cd890d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java @@ -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 tags = new ArrayList(); + 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 tags = new ArrayList(); - 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, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java index 9f1b4c321e3..2c139c9923a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java @@ -71,6 +71,9 @@ extends Mapper private String hfileOutPath; + /** List of cell tags */ + private List tags; + public long getTs() { return ts; } @@ -106,6 +109,7 @@ extends Mapper throw new RuntimeException("No row key column specified"); } this.kvCreator = new CellCreator(conf); + tags = new ArrayList(); } /** @@ -155,6 +159,19 @@ extends Mapper 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 } else { // Creating the KV which needs to be directly written to HFiles. Using the Facade // KVCreator for creation of kvs. - List tags = new ArrayList(); - 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),