hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject hbase git commit: HBASE-14520 Optimize the number of calls for tags creation in bulk load (Bhupendra Kumar Jain)
Date Tue, 06 Oct 2015 13:43:39 GMT
Repository: hbase
Updated Branches:
  refs/heads/master ed4c734b1 -> 23079c02b


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


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/23079c02
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/23079c02
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/23079c02

Branch: refs/heads/master
Commit: 23079c02bf40c318fff4f77fa9182ebdfb230e90
Parents: ed4c734
Author: tedyu <yuzhihong@gmail.com>
Authored: Tue Oct 6 06:43:30 2015 -0700
Committer: tedyu <yuzhihong@gmail.com>
Committed: Tue Oct 6 06:43:30 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/mapreduce/TextSortReducer.java | 23 +++++++++--------
 .../hbase/mapreduce/TsvImporterMapper.java      | 27 ++++++++++++--------
 2 files changed, 29 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/23079c02/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
----------------------------------------------------------------------
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 5056f0b..46e69cd 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<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,

http://git-wip-us.apache.org/repos/asf/hbase/blob/23079c02/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
----------------------------------------------------------------------
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 9f1b4c3..2c139c9 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<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),


Mime
View raw message