You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2016/09/29 06:57:00 UTC

[15/50] [abbrv] hbase git commit: HBASE-16665 Check whether KeyValueUtil.createXXX could be replaced by CellUtil without copy

HBASE-16665 Check whether KeyValueUtil.createXXX could be replaced by CellUtil without copy


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

Branch: refs/heads/hbase-14439
Commit: 21969f5159e6e8f93a7b8f9c7cfe2f359f11dd27
Parents: f7bb6fb
Author: chenheng <ch...@apache.org>
Authored: Sun Sep 25 14:06:55 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Sun Sep 25 14:06:55 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Result.java  | 16 ++++++++++++--
 .../java/org/apache/hadoop/hbase/CellUtil.java  | 22 ++++++++++++++++++++
 .../hbase/io/hfile/HFilePrettyPrinter.java      |  2 +-
 .../hbase/mob/mapreduce/MemStoreWrapper.java    |  3 ++-
 .../hbase/mob/mapreduce/SweepReducer.java       |  3 ++-
 .../hbase/regionserver/AbstractMemStore.java    |  2 +-
 .../hbase/regionserver/HRegionFileSystem.java   |  5 +++--
 .../hbase/regionserver/StoreFileReader.java     |  4 +---
 8 files changed, 46 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/21969f51/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index f1e7cc4..98792e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -275,12 +275,24 @@ public class Result implements CellScannable, CellScanner {
     return result;
   }
 
+  private byte[] notNullBytes(final byte[] bytes) {
+    if (bytes == null) {
+      return HConstants.EMPTY_BYTE_ARRAY;
+    } else {
+      return bytes;
+    }
+  }
+
   protected int binarySearch(final Cell [] kvs,
                              final byte [] family,
                              final byte [] qualifier) {
+    byte[] familyNotNull = notNullBytes(family);
+    byte[] qualifierNotNull = notNullBytes(qualifier);
     Cell searchTerm =
-        KeyValueUtil.createFirstOnRow(CellUtil.cloneRow(kvs[0]),
-            family, qualifier);
+        CellUtil.createFirstOnRow(kvs[0].getRowArray(),
+            kvs[0].getRowOffset(), kvs[0].getRowLength(),
+            familyNotNull, 0, (byte)familyNotNull.length,
+            qualifierNotNull, 0, qualifierNotNull.length);
 
     // pos === ( -(insertion point) - 1)
     int pos = Arrays.binarySearch(kvs, searchTerm, CellComparator.COMPARATOR);

http://git-wip-us.apache.org/repos/asf/hbase/blob/21969f51/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 94c7189..2da71fb 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -1735,6 +1735,24 @@ public final class CellUtil {
     return new FirstOnRowCell(row, roffset, rlength);
   }
 
+  public static Cell createFirstOnRow(final byte[] row, final byte[] family, final byte[] col) {
+    return createFirstOnRow(row, 0, (short)row.length,
+        family, 0, (byte)family.length,
+        col, 0, col.length);
+  }
+
+  public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength,
+                                      final byte[] family, int foffset, byte flength,
+                                      final byte[] col, int coffset, int clength) {
+    return new FirstOnRowColCell(row, roffset, rlength,
+        family, foffset, flength,
+        col, coffset, clength);
+  }
+
+  public static Cell createFirstOnRow(final byte[] row) {
+    return createFirstOnRow(row, 0, (short)row.length);
+  }
+
   /**
    * Create a Cell that is smaller than all other possible Cells for the given Cell's row.
    * The family length is considered to be 0
@@ -1824,6 +1842,10 @@ public final class CellUtil {
     return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
 
+  public static Cell createLastOnRow(final byte[] row) {
+    return new LastOnRowCell(row, 0, (short)row.length);
+  }
+
   /**
    * Create a Cell that is larger than all other possible Cells for the given Cell's rk:cf:q. Used
    * in creating "fake keys" for the multi-column Bloom filter optimization to skip the row/column

http://git-wip-us.apache.org/repos/asf/hbase/blob/21969f51/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
index 36067e5..8c34d77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
@@ -317,7 +317,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
       if (this.isSeekToRow) {
         // seek to the first kv on this row
         shouldScanKeysValues =
-          (scanner.seekTo(KeyValueUtil.createFirstOnRow(this.row)) != -1);
+          (scanner.seekTo(CellUtil.createFirstOnRow(this.row)) != -1);
       } else {
         shouldScanKeysValues = scanner.seekTo();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/21969f51/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
index 7997b49..2dea5bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
@@ -155,7 +156,7 @@ public class MemStoreWrapper {
     context.getCounter(SweepCounter.FILE_AFTER_MERGE_OR_CLEAN).increment(1);
     // write reference/fileName back to the store files of HBase.
     scanner = snapshot.getScanner();
-    scanner.seek(KeyValueUtil.createFirstOnRow(HConstants.EMPTY_START_ROW));
+    scanner.seek(CellUtil.createFirstOnRow(HConstants.EMPTY_START_ROW));
     cell = null;
     Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE,
         Bytes.toBytes(this.table.getName().toString()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/21969f51/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java
index d39267b..b6b4f67 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
@@ -348,7 +349,7 @@ public class SweepReducer extends Reducer<Text, KeyValue, Writable, Writable> {
         file.open();
         try {
           scanner = file.getScanner();
-          scanner.seek(KeyValueUtil.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY));
+          scanner.seek(CellUtil.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY));
           Cell cell;
           while (null != (cell = scanner.next())) {
             if (kvs.contains(cell)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/21969f51/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
index 5e9f632..aa6576f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
@@ -270,7 +270,7 @@ public abstract class AbstractMemStore implements MemStore {
 
     // Get the Cells for the row/family/qualifier regardless of timestamp.
     // For this case we want to clean up any other puts
-    Cell firstCell = KeyValueUtil.createFirstOnRow(
+    Cell firstCell = CellUtil.createFirstOnRow(
         cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
         cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
         cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());

http://git-wip-us.apache.org/repos/asf/hbase/blob/21969f51/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 508b4a7..12e93c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -30,6 +30,7 @@ import java.util.UUID;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -598,7 +599,7 @@ public class HRegionFileSystem {
       try {
         if (top) {
           //check if larger than last key.
-          KeyValue splitKey = KeyValueUtil.createFirstOnRow(splitRow);
+          Cell splitKey = CellUtil.createFirstOnRow(splitRow);
           Cell lastKey = f.getLastKey();
           // If lastKey is null means storefile is empty.
           if (lastKey == null) {
@@ -609,7 +610,7 @@ public class HRegionFileSystem {
           }
         } else {
           //check if smaller than first key
-          KeyValue splitKey = KeyValueUtil.createLastOnRow(splitRow);
+          Cell splitKey = CellUtil.createLastOnRow(splitRow);
           Cell firstKey = f.getFirstKey();
           // If firstKey is null means storefile is empty.
           if (firstKey == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/21969f51/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
index bd9d205..d91e79e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
@@ -236,9 +236,7 @@ public class StoreFileReader {
         if (columns != null && columns.size() == 1) {
           byte[] column = columns.first();
           // create the required fake key
-          Cell kvKey = KeyValueUtil.createFirstOnRow(row, 0, row.length,
-            HConstants.EMPTY_BYTE_ARRAY, 0, 0, column, 0,
-            column.length);
+          Cell kvKey = CellUtil.createFirstOnRow(row, HConstants.EMPTY_BYTE_ARRAY, column);
           return passesGeneralRowColBloomFilter(kvKey);
         }