From 66b4e36a1e6f08dada7c0dfddcb076eb35a5c021 Mon Sep 17 00:00:00 2001 From: Bhupendra Date: Mon, 5 Oct 2015 14:57:34 +0530 Subject: [PATCH] HBASE-14520: Optimize the number of calls for tags creation in bulk load --- .../hadoop/hbase/mapreduce/TextSortReducer.java | 23 +++++++++--------- .../hadoop/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 8dab547..8fe5cc7 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 @@ -159,7 +159,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() @@ -168,16 +179,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 0891df2..98dc25e 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 @@ -72,6 +72,9 @@ extends Mapper private String hfileOutPath; + /** List of cell tags */ + private List tags; + public long getTs() { return ts; } @@ -107,6 +110,7 @@ extends Mapper throw new RuntimeException("No row key column specified"); } this.kvCreator = new CellCreator(conf); + tags = new ArrayList(); } /** @@ -156,6 +160,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() @@ -201,16 +218,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), -- 1.8.4.msysgit.0