You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2019/02/15 17:15:46 UTC

[hbase] 02/02: HBASE-21872 Use a call that defaults to UTF-8 charset for string to byte encoding

This is an automated email from the ASF dual-hosted git repository.

elserj pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit ae0198084c3910575fd8c8f7d838421cd754246e
Author: Josh Elser <el...@apache.org>
AuthorDate: Mon Feb 11 13:19:23 2019 -0500

    HBASE-21872 Use a call that defaults to UTF-8 charset for string to byte encoding
    
    Fixed commit message
    
    Signed-off-by: Sean Busbey <bu...@apache.org>
---
 .../hbase/backup/impl/BackupSystemTable.java       |  30 +-
 .../backup/impl/IncrementalTableBackupClient.java  |   6 +-
 .../master/LogRollMasterProcedureManager.java      |   3 +-
 .../hbase/backup/TestBackupDeleteRestore.java      |   3 +-
 .../hbase/backup/TestBackupHFileCleaner.java       |   3 +-
 .../apache/hadoop/hbase/rest/RESTDemoClient.java   |   6 +-
 .../apache/hadoop/hbase/thrift2/DemoClient.java    |  13 +-
 .../org/apache/hadoop/hbase/http/HtmlQuoting.java  |  16 +-
 .../hadoop/hbase/IntegrationTestLazyCfLoading.java |   2 +-
 .../org/apache/hadoop/hbase/mapreduce/Import.java  |   2 +-
 .../apache/hadoop/hbase/mapreduce/ImportTsv.java   |   8 +-
 .../hadoop/hbase/TestPerformanceEvaluation.java    |   4 +-
 .../hadoop/hbase/mapred/TestGroupingTableMap.java  |  24 +-
 .../apache/hadoop/hbase/mapred/TestSplitTable.java |   4 +-
 .../hadoop/hbase/mapred/TestTableInputFormat.java  |  30 +-
 .../hadoop/hbase/mapreduce/TestCopyTable.java      |   2 +-
 .../hadoop/hbase/mapreduce/TestImportExport.java   |   2 +-
 .../hbase/mapreduce/TestRegionSizeCalculator.java  |  20 +-
 .../hbase/mapreduce/TestTableInputFormat.java      |  30 +-
 .../hadoop/hbase/mapreduce/TestTableSplit.java     |  33 +-
 .../TestVerifyReplicationCrossDiffHdfs.java        |   8 +-
 .../org/apache/hadoop/hbase/TestSerialization.java |  22 +-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java |  18 +-
 .../hbase/client/TestAsyncTableAdminApi.java       |   8 +-
 .../hbase/client/TestConnectionImplementation.java |  12 +-
 .../apache/hadoop/hbase/client/TestFastFail.java   |   2 +-
 .../hbase/client/TestGetScanPartialResult.java     |   3 +-
 .../hadoop/hbase/client/TestMetaWithReplicas.java  |   6 +-
 .../hbase/client/TestReplicaWithCluster.java       |   2 +-
 .../hadoop/hbase/client/TestReplicasClient.java    |  18 +-
 .../coprocessor/TestRegionObserverInterface.java   |   4 +-
 .../hadoop/hbase/filter/TestFilterWrapper.java     |   2 +-
 .../filter/TestFuzzyRowAndColumnRangeFilter.java   |   4 +-
 .../hbase/filter/TestFuzzyRowFilterEndToEnd.java   |  16 +-
 .../hadoop/hbase/filter/TestScanRowPrefix.java     |   3 +-
 .../hbase/io/encoding/TestEncodedSeekers.java      |   4 +-
 .../hadoop/hbase/io/hfile/RandomKeyValueUtil.java  |  13 +-
 .../apache/hadoop/hbase/io/hfile/TestHFile.java    |   9 +-
 .../hadoop/hbase/io/hfile/TestHFileBlockIndex.java |   2 +-
 .../hadoop/hbase/io/hfile/TestHFileEncryption.java |   3 +-
 .../hadoop/hbase/io/hfile/TestHFileSeek.java       |   5 +-
 .../master/balancer/TestBaseLoadBalancer.java      |  21 +-
 .../hbase/master/locking/TestLockManager.java      |   4 +-
 .../procedure/MasterProcedureTestingUtility.java   |   6 +-
 .../hbase/procedure/SimpleRSProcedureManager.java  |   3 +-
 .../hbase/procedure/TestProcedureManager.java      |   3 +-
 .../procedure/TestZKProcedureControllers.java      |   3 +-
 .../regionserver/TestCompoundBloomFilter.java      |   4 +-
 .../hbase/regionserver/TestMultiLogThreshold.java  |   2 +-
 .../TestRegionReplicasWithModifyTable.java         |   1 -
 .../TestRegionServerReadRequestMetrics.java        |  22 +-
 .../TestSplitTransactionOnCluster.java             |  22 +-
 .../TestWALMonotonicallyIncreasingSeqId.java       |   4 +-
 .../throttle/TestStoreHotnessProtector.java        |   3 +-
 ...stRegionReplicaReplicationEndpointNoMaster.java |   3 +-
 .../access/TestWithDisabledAuthorization.java      |   4 +-
 .../hbase/tool/TestLoadIncrementalHFiles.java      |  24 +-
 .../hadoop/hbase/util/TestBloomFilterChunk.java    |   2 +-
 .../apache/hadoop/hbase/util/TestRegionMover.java  |   2 +-
 .../hadoop/hbase/util/TestRegionSplitter.java      | 532 ++++++++++-----------
 .../hadoop/hbase/zookeeper/TestZooKeeperACL.java   |  37 +-
 .../hbase/zookeeper/MiniZooKeeperCluster.java      |   6 +-
 .../hbase/zookeeper/TestRecoverableZooKeeper.java  |   4 +-
 .../apache/hadoop/hbase/zookeeper/TestZKUtil.java  |   4 +-
 64 files changed, 579 insertions(+), 542 deletions(-)

diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
index 94ccfe5..89945cb 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -141,12 +141,12 @@ public final class BackupSystemTable implements Closeable {
   /**
    * Stores backup sessions (contexts)
    */
-  final static byte[] SESSIONS_FAMILY = "session".getBytes();
+  final static byte[] SESSIONS_FAMILY = Bytes.toBytes("session");
   /**
    * Stores other meta
    */
-  final static byte[] META_FAMILY = "meta".getBytes();
-  final static byte[] BULK_LOAD_FAMILY = "bulk".getBytes();
+  final static byte[] META_FAMILY = Bytes.toBytes("meta");
+  final static byte[] BULK_LOAD_FAMILY = Bytes.toBytes("bulk");
   /**
    * Connection to HBase cluster, shared among all instances
    */
@@ -154,20 +154,20 @@ public final class BackupSystemTable implements Closeable {
 
   private final static String BACKUP_INFO_PREFIX = "session:";
   private final static String START_CODE_ROW = "startcode:";
-  private final static byte[] ACTIVE_SESSION_ROW = "activesession:".getBytes();
-  private final static byte[] ACTIVE_SESSION_COL = "c".getBytes();
+  private final static byte[] ACTIVE_SESSION_ROW = Bytes.toBytes("activesession:");
+  private final static byte[] ACTIVE_SESSION_COL = Bytes.toBytes("c");
 
-  private final static byte[] ACTIVE_SESSION_YES = "yes".getBytes();
-  private final static byte[] ACTIVE_SESSION_NO = "no".getBytes();
+  private final static byte[] ACTIVE_SESSION_YES = Bytes.toBytes("yes");
+  private final static byte[] ACTIVE_SESSION_NO = Bytes.toBytes("no");
 
   private final static String INCR_BACKUP_SET = "incrbackupset:";
   private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm:";
   private final static String RS_LOG_TS_PREFIX = "rslogts:";
 
   private final static String BULK_LOAD_PREFIX = "bulk:";
-  private final static byte[] BULK_LOAD_PREFIX_BYTES = BULK_LOAD_PREFIX.getBytes();
-  private final static byte[] DELETE_OP_ROW = "delete_op_row".getBytes();
-  private final static byte[] MERGE_OP_ROW = "merge_op_row".getBytes();
+  private final static byte[] BULK_LOAD_PREFIX_BYTES = Bytes.toBytes(BULK_LOAD_PREFIX);
+  private final static byte[] DELETE_OP_ROW = Bytes.toBytes("delete_op_row");
+  private final static byte[] MERGE_OP_ROW = Bytes.toBytes("merge_op_row");
 
   final static byte[] TBL_COL = Bytes.toBytes("tbl");
   final static byte[] FAM_COL = Bytes.toBytes("fam");
@@ -1615,7 +1615,7 @@ public final class BackupSystemTable implements Closeable {
           Bytes.toString(region), BLK_LD_DELIM, filename));
         put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
         put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, entry.getKey());
-        put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
+        put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, Bytes.toBytes(file));
         put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_COMMIT);
         puts.add(put);
         LOG.debug(
@@ -1695,7 +1695,7 @@ public final class BackupSystemTable implements Closeable {
         Bytes.toString(region), BLK_LD_DELIM, filename));
       put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
       put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, family);
-      put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
+      put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, Bytes.toBytes(file));
       put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_PREPARE);
       puts.add(put);
       LOG.debug("writing raw bulk path " + file + " for " + table + " " + Bytes.toString(region));
@@ -1902,7 +1902,7 @@ public final class BackupSystemTable implements Closeable {
     Put put = new Put(rowkey(BULK_LOAD_PREFIX, backupId + BLK_LD_DELIM + ts + BLK_LD_DELIM + idx));
     put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, tn.getName());
     put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, fam);
-    put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, p.getBytes());
+    put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, Bytes.toBytes(p));
     return put;
   }
 
@@ -2006,7 +2006,7 @@ public final class BackupSystemTable implements Closeable {
   }
 
   private byte[] convertToByteArray(String[] tables) {
-    return StringUtils.join(tables, ",").getBytes();
+    return Bytes.toBytes(StringUtils.join(tables, ","));
   }
 
   /**
@@ -2037,6 +2037,6 @@ public final class BackupSystemTable implements Closeable {
     for (String ss : other) {
       sb.append(ss);
     }
-    return sb.toString().getBytes();
+    return Bytes.toBytes(sb.toString());
   }
 }
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
index 43824d7..1e4b266 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
@@ -155,11 +155,11 @@ public class IncrementalTableBackupClient extends TableBackupClient {
           String fam = famEntry.getKey();
           Path famDir = new Path(regionDir, fam);
           List<Path> files;
-          if (!mapForSrc[srcIdx].containsKey(fam.getBytes())) {
+          if (!mapForSrc[srcIdx].containsKey(Bytes.toBytes(fam))) {
             files = new ArrayList<>();
-            mapForSrc[srcIdx].put(fam.getBytes(), files);
+            mapForSrc[srcIdx].put(Bytes.toBytes(fam), files);
           } else {
-            files = mapForSrc[srcIdx].get(fam.getBytes());
+            files = mapForSrc[srcIdx].get(Bytes.toBytes(fam));
           }
           Path archiveDir = HFileArchiveUtil.getStoreArchivePath(conf, srcTable, regionName, fam);
           String tblName = srcTable.getQualifierAsString();
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
index a3d5ce2..32e3e23 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
 import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinationManager;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -130,7 +131,7 @@ public class LogRollMasterProcedureManager extends MasterProcedureManager {
     byte[] data = new byte[0];
     if (conf.size() > 0) {
       // Get backup root path
-      data = conf.get(0).getValue().getBytes();
+      data = Bytes.toBytes(conf.get(0).getValue());
     }
     Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), data, servers);
     if (proc == null) {
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
index a557b9b..74176e3 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -63,7 +64,7 @@ public class TestBackupDeleteRestore extends TestBackupBase {
     HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
     // delete row
     try (Table table = TEST_UTIL.getConnection().getTable(table1)) {
-      Delete delete = new Delete("row0".getBytes());
+      Delete delete = new Delete(Bytes.toBytes("row0"));
       table.delete(delete);
       hba.flush(table1);
     }
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
index 8d22f27..1680dc6 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -132,7 +133,7 @@ public class TestBackupHFileCleaner {
       sTableList.add(tableName);
       Map<byte[], List<Path>>[] maps = new Map[1];
       maps[0] = new HashMap<>();
-      maps[0].put(famName.getBytes(), list);
+      maps[0].put(Bytes.toBytes(famName), list);
       sysTbl.writeBulkLoadedFiles(sTableList, maps, "1");
     }
 
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/rest/RESTDemoClient.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/rest/RESTDemoClient.java
index 19fae47..2e462e5 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/rest/RESTDemoClient.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/rest/RESTDemoClient.java
@@ -87,12 +87,12 @@ public class RESTDemoClient {
     try (RemoteHTable remoteTable = new RemoteHTable(restClient, conf, "example")) {
       // Write data to the table
       String rowKey = "row1";
-      Put p = new Put(rowKey.getBytes());
-      p.addColumn("family1".getBytes(), "qualifier1".getBytes(), "value1".getBytes());
+      Put p = new Put(Bytes.toBytes(rowKey));
+      p.addColumn(Bytes.toBytes("family1"), Bytes.toBytes("qualifier1"), Bytes.toBytes("value1"));
       remoteTable.put(p);
 
       // Get the data from the table
-      Get g = new Get(rowKey.getBytes());
+      Get g = new Get(Bytes.toBytes(rowKey));
       Result result = remoteTable.get(g);
 
       Preconditions.checkArgument(result != null,
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java
index 7fbe2aa..f8d4f64 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.thrift2.generated.TGet;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
 import org.apache.hadoop.hbase.thrift2.generated.TPut;
 import org.apache.hadoop.hbase.thrift2.generated.TResult;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.transport.TFramedTransport;
@@ -116,15 +117,15 @@ public class DemoClient {
     // open the transport
     transport.open();
 
-    ByteBuffer table = ByteBuffer.wrap("example".getBytes());
+    ByteBuffer table = ByteBuffer.wrap(Bytes.toBytes("example"));
 
     TPut put = new TPut();
-    put.setRow("row1".getBytes());
+    put.setRow(Bytes.toBytes("row1"));
 
     TColumnValue columnValue = new TColumnValue();
-    columnValue.setFamily("family1".getBytes());
-    columnValue.setQualifier("qualifier1".getBytes());
-    columnValue.setValue("value1".getBytes());
+    columnValue.setFamily(Bytes.toBytes("family1"));
+    columnValue.setQualifier(Bytes.toBytes("qualifier1"));
+    columnValue.setValue(Bytes.toBytes("value1"));
     List<TColumnValue> columnValues = new ArrayList<>(1);
     columnValues.add(columnValue);
     put.setColumnValues(columnValues);
@@ -132,7 +133,7 @@ public class DemoClient {
     client.put(table, put);
 
     TGet get = new TGet();
-    get.setRow("row1".getBytes());
+    get.setRow(Bytes.toBytes("row1"));
 
     TResult result = client.get(table, get);
 
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java
index e2eebb2..6d1bfbc 100644
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.http;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -27,11 +29,11 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public final class HtmlQuoting {
-  private static final byte[] ampBytes = "&amp;".getBytes();
-  private static final byte[] aposBytes = "&apos;".getBytes();
-  private static final byte[] gtBytes = "&gt;".getBytes();
-  private static final byte[] ltBytes = "&lt;".getBytes();
-  private static final byte[] quotBytes = "&quot;".getBytes();
+  private static final byte[] ampBytes = Bytes.toBytes("&amp;");
+  private static final byte[] aposBytes = Bytes.toBytes("&apos;");
+  private static final byte[] gtBytes = Bytes.toBytes("&gt;");
+  private static final byte[] ltBytes = Bytes.toBytes("&lt;");
+  private static final byte[] quotBytes = Bytes.toBytes("&quot;");
 
   /**
    * Does the given string need to be quoted?
@@ -69,7 +71,7 @@ public final class HtmlQuoting {
     if (str == null) {
       return false;
     }
-    byte[] bytes = str.getBytes();
+    byte[] bytes = Bytes.toBytes(str);
     return needsQuoting(bytes, 0 , bytes.length);
   }
 
@@ -104,7 +106,7 @@ public final class HtmlQuoting {
     if (item == null) {
       return null;
     }
-    byte[] bytes = item.getBytes();
+    byte[] bytes = Bytes.toBytes(item);
     if (needsQuoting(bytes, 0, bytes.length)) {
       ByteArrayOutputStream buffer = new ByteArrayOutputStream();
       try {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
index c2966a3..128767b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
@@ -112,7 +112,7 @@ public class IntegrationTestLazyCfLoading {
 
     @Override
     public byte[] getDeterministicUniqueKey(long keyBase) {
-      return LoadTestKVGenerator.md5PrefixedKey(keyBase).getBytes();
+      return Bytes.toBytes(LoadTestKVGenerator.md5PrefixedKey(keyBase));
     }
 
     @Override
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index 7805bc5..239a12b 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -545,7 +545,7 @@ public class Import extends Configured implements Tool {
         if(srcAndDest.length != 2) {
             continue;
         }
-        cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
+        cfRenameMap.put(Bytes.toBytes(srcAndDest[0]), Bytes.toBytes(srcAndDest[1]));
       }
     }
     return cfRenameMap;
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
index 58d9184..0fd9483 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
@@ -194,11 +194,11 @@ public class ImportTsv extends Configured implements Tool {
         }
         String[] parts = str.split(":", 2);
         if (parts.length == 1) {
-          families[i] = str.getBytes();
+          families[i] = Bytes.toBytes(str);
           qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
         } else {
-          families[i] = parts[0].getBytes();
-          qualifiers[i] = parts[1].getBytes();
+          families[i] = Bytes.toBytes(parts[0]);
+          qualifiers[i] = Bytes.toBytes(parts[1]);
         }
       }
     }
@@ -471,7 +471,7 @@ public class ImportTsv extends Configured implements Tool {
         String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
         if (actualSeparator != null) {
           conf.set(SEPARATOR_CONF_KEY,
-              Bytes.toString(Base64.getEncoder().encode(actualSeparator.getBytes())));
+              Bytes.toString(Base64.getEncoder().encode(Bytes.toBytes(actualSeparator))));
         }
 
         // See if a non-default Mapper was set
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
index 2e1ecd0..dfcf2d6 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
@@ -31,6 +31,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
+import java.nio.charset.StandardCharsets;
 import java.util.LinkedList;
 import java.util.NoSuchElementException;
 import java.util.Queue;
@@ -104,7 +105,8 @@ public class TestPerformanceEvaluation {
     try {
       dis.readFully(content);
       BufferedReader br =
-        new BufferedReader(new InputStreamReader(new ByteArrayInputStream(content)));
+        new BufferedReader(new InputStreamReader(
+              new ByteArrayInputStream(content), StandardCharsets.UTF_8));
       int count = 0;
       while (br.readLine() != null) {
         count++;
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestGroupingTableMap.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestGroupingTableMap.java
index 12db348..76e3c73 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestGroupingTableMap.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestGroupingTableMap.java
@@ -72,9 +72,12 @@ public class TestGroupingTableMap {
 
       byte[] row = {};
       List<Cell> keyValues = ImmutableList.<Cell>of(
-          new KeyValue(row, "familyA".getBytes(), "qualifierA".getBytes(), Bytes.toBytes("1111")),
-          new KeyValue(row, "familyA".getBytes(), "qualifierA".getBytes(), Bytes.toBytes("2222")),
-          new KeyValue(row, "familyB".getBytes(), "qualifierB".getBytes(), Bytes.toBytes("3333")));
+          new KeyValue(row, Bytes.toBytes("familyA"), Bytes.toBytes("qualifierA"),
+              Bytes.toBytes("1111")),
+          new KeyValue(row, Bytes.toBytes("familyA"), Bytes.toBytes("qualifierA"),
+              Bytes.toBytes("2222")),
+          new KeyValue(row, Bytes.toBytes("familyB"), Bytes.toBytes("qualifierB"),
+              Bytes.toBytes("3333")));
       when(result.listCells()).thenReturn(keyValues);
       OutputCollector<ImmutableBytesWritable, Result> outputCollectorMock =
           mock(OutputCollector.class);
@@ -102,9 +105,12 @@ public class TestGroupingTableMap {
 
       byte[] row = {};
       List<Cell> keyValues = ImmutableList.<Cell>of(
-          new KeyValue(row, "familyA".getBytes(), "qualifierA".getBytes(), Bytes.toBytes("1111")),
-          new KeyValue(row, "familyB".getBytes(), "qualifierB".getBytes(), Bytes.toBytes("2222")),
-          new KeyValue(row, "familyC".getBytes(), "qualifierC".getBytes(), Bytes.toBytes("3333")));
+          new KeyValue(row, Bytes.toBytes("familyA"), Bytes.toBytes("qualifierA"),
+              Bytes.toBytes("1111")),
+          new KeyValue(row, Bytes.toBytes("familyB"), Bytes.toBytes("qualifierB"),
+              Bytes.toBytes("2222")),
+          new KeyValue(row, Bytes.toBytes("familyC"), Bytes.toBytes("qualifierC"),
+              Bytes.toBytes("3333")));
       when(result.listCells()).thenReturn(keyValues);
       OutputCollector<ImmutableBytesWritable, Result> outputCollectorMock =
           mock(OutputCollector.class);
@@ -137,8 +143,10 @@ public class TestGroupingTableMap {
       final byte[] secondPartKeyValue = Bytes.toBytes("35245142671437");
       byte[] row = {};
       List<Cell> cells = ImmutableList.<Cell>of(
-          new KeyValue(row, "familyA".getBytes(), "qualifierA".getBytes(), firstPartKeyValue),
-          new KeyValue(row, "familyB".getBytes(), "qualifierB".getBytes(), secondPartKeyValue));
+          new KeyValue(row, Bytes.toBytes("familyA"), Bytes.toBytes("qualifierA"),
+              firstPartKeyValue),
+          new KeyValue(row, Bytes.toBytes("familyB"), Bytes.toBytes("qualifierB"),
+              secondPartKeyValue));
       when(result.listCells()).thenReturn(cells);
 
       final AtomicBoolean outputCollected = new AtomicBoolean();
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestSplitTable.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestSplitTable.java
index 7b097d2..bf46a7a 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestSplitTable.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestSplitTable.java
@@ -106,8 +106,8 @@ public class TestSplitTable {
   @SuppressWarnings("deprecation")
   public void testToString() {
     TableSplit split =
-        new TableSplit(TableName.valueOf(name.getMethodName()), "row-start".getBytes(), "row-end".getBytes(),
-            "location");
+        new TableSplit(TableName.valueOf(name.getMethodName()), Bytes.toBytes("row-start"),
+            Bytes.toBytes("row-end"), "location");
     String str =
         "HBase table split(table name: " + name.getMethodName() + ", start row: row-start, "
             + "end row: row-end, region location: location)";
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
index 2ef8351..e93d36d 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
@@ -124,14 +124,14 @@ public class TestTableInputFormat {
    */
   public static Table createTable(byte[] tableName, byte[][] families) throws IOException {
     Table table = UTIL.createTable(TableName.valueOf(tableName), families);
-    Put p = new Put("aaa".getBytes());
+    Put p = new Put(Bytes.toBytes("aaa"));
     for (byte[] family : families) {
-      p.addColumn(family, null, "value aaa".getBytes());
+      p.addColumn(family, null, Bytes.toBytes("value aaa"));
     }
     table.put(p);
-    p = new Put("bbb".getBytes());
+    p = new Put(Bytes.toBytes("bbb"));
     for (byte[] family : families) {
-      p.addColumn(family, null, "value bbb".getBytes());
+      p.addColumn(family, null, Bytes.toBytes("value bbb"));
     }
     table.put(p);
     return table;
@@ -165,8 +165,8 @@ public class TestTableInputFormat {
   static void runTestMapred(Table table) throws IOException {
     org.apache.hadoop.hbase.mapred.TableRecordReader trr =
         new org.apache.hadoop.hbase.mapred.TableRecordReader();
-    trr.setStartRow("aaa".getBytes());
-    trr.setEndRow("zzz".getBytes());
+    trr.setStartRow(Bytes.toBytes("aaa"));
+    trr.setEndRow(Bytes.toBytes("zzz"));
     trr.setHTable(table);
     trr.setInputColumns(columns);
 
@@ -176,11 +176,11 @@ public class TestTableInputFormat {
 
     boolean more = trr.next(key, r);
     assertTrue(more);
-    checkResult(r, key, "aaa".getBytes(), "value aaa".getBytes());
+    checkResult(r, key, Bytes.toBytes("aaa"), Bytes.toBytes("value aaa"));
 
     more = trr.next(key, r);
     assertTrue(more);
-    checkResult(r, key, "bbb".getBytes(), "value bbb".getBytes());
+    checkResult(r, key, Bytes.toBytes("bbb"), Bytes.toBytes("value bbb"));
 
     // no more data
     more = trr.next(key, r);
@@ -204,7 +204,7 @@ public class TestTableInputFormat {
         if (cnt++ < failCnt) {
           // create mock ResultScanner that always fails.
           Scan scan = mock(Scan.class);
-          doReturn("bogus".getBytes()).when(scan).getStartRow(); // avoid npe
+          doReturn(Bytes.toBytes("bogus")).when(scan).getStartRow(); // avoid npe
           ResultScanner scanner = mock(ResultScanner.class);
           // simulate TimeoutException / IOException
           doThrow(new IOException("Injected exception")).when(scanner).next();
@@ -239,7 +239,7 @@ public class TestTableInputFormat {
         if (cnt++ < failCnt) {
           // create mock ResultScanner that always fails.
           Scan scan = mock(Scan.class);
-          doReturn("bogus".getBytes()).when(scan).getStartRow(); // avoid npe
+          doReturn(Bytes.toBytes("bogus")).when(scan).getStartRow(); // avoid npe
           ResultScanner scanner = mock(ResultScanner.class);
 
           invocation.callRealMethod(); // simulate NotServingRegionException
@@ -266,7 +266,7 @@ public class TestTableInputFormat {
    */
   @Test
   public void testTableRecordReader() throws IOException {
-    Table table = createTable("table1".getBytes());
+    Table table = createTable(Bytes.toBytes("table1"));
     runTestMapred(table);
   }
 
@@ -277,7 +277,7 @@ public class TestTableInputFormat {
    */
   @Test
   public void testTableRecordReaderScannerFail() throws IOException {
-    Table htable = createIOEScannerTable("table2".getBytes(), 1);
+    Table htable = createIOEScannerTable(Bytes.toBytes("table2"), 1);
     runTestMapred(htable);
   }
 
@@ -288,7 +288,7 @@ public class TestTableInputFormat {
    */
   @Test(expected = IOException.class)
   public void testTableRecordReaderScannerFailTwice() throws IOException {
-    Table htable = createIOEScannerTable("table3".getBytes(), 2);
+    Table htable = createIOEScannerTable(Bytes.toBytes("table3"), 2);
     runTestMapred(htable);
   }
 
@@ -299,7 +299,7 @@ public class TestTableInputFormat {
    */
   @Test
   public void testTableRecordReaderScannerTimeout() throws IOException {
-    Table htable = createDNRIOEScannerTable("table4".getBytes(), 1);
+    Table htable = createDNRIOEScannerTable(Bytes.toBytes("table4"), 1);
     runTestMapred(htable);
   }
 
@@ -310,7 +310,7 @@ public class TestTableInputFormat {
    */
   @Test(expected = org.apache.hadoop.hbase.NotServingRegionException.class)
   public void testTableRecordReaderScannerTimeoutTwice() throws IOException {
-    Table htable = createDNRIOEScannerTable("table5".getBytes(), 2);
+    Table htable = createDNRIOEScannerTable(Bytes.toBytes("table5"), 2);
     runTestMapred(htable);
   }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
index 5591e5f..1b94ca8 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
@@ -301,7 +301,7 @@ public class TestCopyTable {
     assertNotNull(t2.get(new Get(ROW1)).getRow());
     Result res = t2.get(new Get(ROW1));
     byte[] b1 = res.getValue(FAMILY_B, QUALIFIER);
-    assertEquals("Data13", new String(b1));
+    assertEquals("Data13", Bytes.toString(b1));
     assertNotNull(t2.get(new Get(ROW2)).getRow());
     res = t2.get(new Get(ROW2));
     b1 = res.getValue(FAMILY_A, QUALIFIER);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
index 8a189c5..9c4da4b 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
@@ -415,7 +415,7 @@ public class TestImportExport {
 
     //Add second version of QUAL
     p = new Put(ROW1);
-    p.addColumn(FAMILYA, QUAL, now + 5, "s".getBytes());
+    p.addColumn(FAMILYA, QUAL, now + 5, Bytes.toBytes("s"));
     exportT.put(p);
 
     //Add second Delete family marker
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
index 85d0f1c..c1562cc 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -66,11 +67,11 @@ public class TestRegionSizeCalculator {
 
     RegionSizeCalculator calculator = new RegionSizeCalculator(regionLocator, admin);
 
-    assertEquals(123 * megabyte, calculator.getRegionSize("region1".getBytes()));
-    assertEquals(54321 * megabyte, calculator.getRegionSize("region2".getBytes()));
-    assertEquals(1232 * megabyte, calculator.getRegionSize("region3".getBytes()));
+    assertEquals(123 * megabyte, calculator.getRegionSize(Bytes.toBytes("region1")));
+    assertEquals(54321 * megabyte, calculator.getRegionSize(Bytes.toBytes("region2")));
+    assertEquals(1232 * megabyte, calculator.getRegionSize(Bytes.toBytes("region3")));
     // if regionCalculator does not know about a region, it should return 0
-    assertEquals(0 * megabyte, calculator.getRegionSize("otherTableRegion".getBytes()));
+    assertEquals(0 * megabyte, calculator.getRegionSize(Bytes.toBytes("otherTableRegion")));
 
     assertEquals(3, calculator.getRegionSizeMap().size());
   }
@@ -91,7 +92,8 @@ public class TestRegionSizeCalculator {
 
     RegionSizeCalculator calculator = new RegionSizeCalculator(regionLocator, admin);
 
-    assertEquals(((long) Integer.MAX_VALUE) * megabyte, calculator.getRegionSize("largeRegion".getBytes()));
+    assertEquals(((long) Integer.MAX_VALUE) * megabyte,
+        calculator.getRegionSize(Bytes.toBytes("largeRegion")));
   }
 
   /** When calculator is disabled, it should return 0 for each request.*/
@@ -106,12 +108,12 @@ public class TestRegionSizeCalculator {
 
     //first request on enabled calculator
     RegionSizeCalculator calculator = new RegionSizeCalculator(table, admin);
-    assertEquals(999 * megabyte, calculator.getRegionSize(regionName.getBytes()));
+    assertEquals(999 * megabyte, calculator.getRegionSize(Bytes.toBytes(regionName)));
 
     //then disabled calculator.
     configuration.setBoolean(RegionSizeCalculator.ENABLE_REGIONSIZECALCULATOR, false);
     RegionSizeCalculator disabledCalculator = new RegionSizeCalculator(table, admin);
-    assertEquals(0 * megabyte, disabledCalculator.getRegionSize(regionName.getBytes()));
+    assertEquals(0 * megabyte, disabledCalculator.getRegionSize(Bytes.toBytes(regionName)));
 
     assertEquals(0, disabledCalculator.getRegionSizeMap().size());
   }
@@ -127,7 +129,7 @@ public class TestRegionSizeCalculator {
 
     for (String regionName : regionNames) {
       HRegionInfo info = Mockito.mock(HRegionInfo.class);
-      when(info.getRegionName()).thenReturn(regionName.getBytes());
+      when(info.getRegionName()).thenReturn(Bytes.toBytes(regionName));
       regionLocations.add(new HRegionLocation(info, sn));
     }
 
@@ -156,7 +158,7 @@ public class TestRegionSizeCalculator {
    * */
   private RegionMetrics mockRegion(String regionName, int fileSizeMb) {
     RegionMetrics region = Mockito.mock(RegionMetrics.class);
-    when(region.getRegionName()).thenReturn(regionName.getBytes());
+    when(region.getRegionName()).thenReturn(Bytes.toBytes(regionName));
     when(region.getNameAsString()).thenReturn(regionName);
     when(region.getStoreFileSize()).thenReturn(new Size(fileSizeMb, Size.Unit.MEGABYTE));
     return region;
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java
index eae8606..0697e72 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java
@@ -121,14 +121,14 @@ public class TestTableInputFormat {
    */
   public static Table createTable(byte[] tableName, byte[][] families) throws IOException {
     Table table = UTIL.createTable(TableName.valueOf(tableName), families);
-    Put p = new Put("aaa".getBytes());
+    Put p = new Put(Bytes.toBytes("aaa"));
     for (byte[] family : families) {
-      p.addColumn(family, null, "value aaa".getBytes());
+      p.addColumn(family, null, Bytes.toBytes("value aaa"));
     }
     table.put(p);
-    p = new Put("bbb".getBytes());
+    p = new Put(Bytes.toBytes("bbb"));
     for (byte[] family : families) {
-      p.addColumn(family, null, "value bbb".getBytes());
+      p.addColumn(family, null, Bytes.toBytes("value bbb"));
     }
     table.put(p);
     return table;
@@ -165,8 +165,8 @@ public class TestTableInputFormat {
     org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl trr =
         new org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl();
     Scan s = new Scan();
-    s.setStartRow("aaa".getBytes());
-    s.setStopRow("zzz".getBytes());
+    s.setStartRow(Bytes.toBytes("aaa"));
+    s.setStopRow(Bytes.toBytes("zzz"));
     s.addFamily(FAMILY);
     trr.setScan(s);
     trr.setHTable(table);
@@ -179,13 +179,13 @@ public class TestTableInputFormat {
     assertTrue(more);
     key = trr.getCurrentKey();
     r = trr.getCurrentValue();
-    checkResult(r, key, "aaa".getBytes(), "value aaa".getBytes());
+    checkResult(r, key, Bytes.toBytes("aaa"), Bytes.toBytes("value aaa"));
 
     more = trr.nextKeyValue();
     assertTrue(more);
     key = trr.getCurrentKey();
     r = trr.getCurrentValue();
-    checkResult(r, key, "bbb".getBytes(), "value bbb".getBytes());
+    checkResult(r, key, Bytes.toBytes("bbb"), Bytes.toBytes("value bbb"));
 
     // no more data
     more = trr.nextKeyValue();
@@ -209,7 +209,7 @@ public class TestTableInputFormat {
         if (cnt++ < failCnt) {
           // create mock ResultScanner that always fails.
           Scan scan = mock(Scan.class);
-          doReturn("bogus".getBytes()).when(scan).getStartRow(); // avoid npe
+          doReturn(Bytes.toBytes("bogus")).when(scan).getStartRow(); // avoid npe
           ResultScanner scanner = mock(ResultScanner.class);
           // simulate TimeoutException / IOException
           doThrow(new IOException("Injected exception")).when(scanner).next();
@@ -244,7 +244,7 @@ public class TestTableInputFormat {
         if (cnt++ < failCnt) {
           // create mock ResultScanner that always fails.
           Scan scan = mock(Scan.class);
-          doReturn("bogus".getBytes()).when(scan).getStartRow(); // avoid npe
+          doReturn(Bytes.toBytes("bogus")).when(scan).getStartRow(); // avoid npe
           ResultScanner scanner = mock(ResultScanner.class);
 
           invocation.callRealMethod(); // simulate NotServingRegionException
@@ -273,7 +273,7 @@ public class TestTableInputFormat {
   @Test
   public void testTableRecordReaderMapreduce() throws IOException,
       InterruptedException {
-    Table table = createTable("table1-mr".getBytes());
+    Table table = createTable(Bytes.toBytes("table1-mr"));
     runTestMapreduce(table);
   }
 
@@ -286,7 +286,7 @@ public class TestTableInputFormat {
   @Test
   public void testTableRecordReaderScannerFailMapreduce() throws IOException,
       InterruptedException {
-    Table htable = createIOEScannerTable("table2-mr".getBytes(), 1);
+    Table htable = createIOEScannerTable(Bytes.toBytes("table2-mr"), 1);
     runTestMapreduce(htable);
   }
 
@@ -299,7 +299,7 @@ public class TestTableInputFormat {
   @Test(expected = IOException.class)
   public void testTableRecordReaderScannerFailMapreduceTwice() throws IOException,
       InterruptedException {
-    Table htable = createIOEScannerTable("table3-mr".getBytes(), 2);
+    Table htable = createIOEScannerTable(Bytes.toBytes("table3-mr"), 2);
     runTestMapreduce(htable);
   }
 
@@ -312,7 +312,7 @@ public class TestTableInputFormat {
   @Test
   public void testTableRecordReaderScannerTimeoutMapreduce()
       throws IOException, InterruptedException {
-    Table htable = createDNRIOEScannerTable("table4-mr".getBytes(), 1);
+    Table htable = createDNRIOEScannerTable(Bytes.toBytes("table4-mr"), 1);
     runTestMapreduce(htable);
   }
 
@@ -325,7 +325,7 @@ public class TestTableInputFormat {
   @Test(expected = org.apache.hadoop.hbase.NotServingRegionException.class)
   public void testTableRecordReaderScannerTimeoutMapreduceTwice()
       throws IOException, InterruptedException {
-    Table htable = createDNRIOEScannerTable("table5-mr".getBytes(), 2);
+    Table htable = createDNRIOEScannerTable(Bytes.toBytes("table5-mr"), 2);
     runTestMapreduce(htable);
   }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSplit.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSplit.java
index 9be1634..4d754fa 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSplit.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSplit.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.Assert;
 import org.junit.ClassRule;
@@ -47,11 +48,11 @@ public class TestTableSplit {
   @Test
   public void testHashCode() {
     TableSplit split1 = new TableSplit(TableName.valueOf(name.getMethodName()),
-        "row-start".getBytes(),
-        "row-end".getBytes(), "location");
+        Bytes.toBytes("row-start"),
+            Bytes.toBytes("row-end"), "location");
     TableSplit split2 = new TableSplit(TableName.valueOf(name.getMethodName()),
-        "row-start".getBytes(),
-        "row-end".getBytes(), "location");
+        Bytes.toBytes("row-start"),
+            Bytes.toBytes("row-end"), "location");
     assertEquals (split1, split2);
     assertTrue   (split1.hashCode() == split2.hashCode());
     HashSet<TableSplit> set = new HashSet<>(2);
@@ -66,11 +67,11 @@ public class TestTableSplit {
   @Test
   public void testHashCode_length() {
     TableSplit split1 = new TableSplit(TableName.valueOf(name.getMethodName()),
-            "row-start".getBytes(),
-            "row-end".getBytes(), "location", 1984);
+        Bytes.toBytes("row-start"),
+            Bytes.toBytes("row-end"), "location", 1984);
     TableSplit split2 = new TableSplit(TableName.valueOf(name.getMethodName()),
-            "row-start".getBytes(),
-            "row-end".getBytes(), "location", 1982);
+        Bytes.toBytes("row-start"),
+            Bytes.toBytes("row-end"), "location", 1982);
 
     assertEquals (split1, split2);
     assertTrue   (split1.hashCode() == split2.hashCode());
@@ -86,12 +87,12 @@ public class TestTableSplit {
   @Test
   public void testLengthIsSerialized() throws Exception {
     TableSplit split1 = new TableSplit(TableName.valueOf(name.getMethodName()),
-            "row-start".getBytes(),
-            "row-end".getBytes(), "location", 666);
+        Bytes.toBytes("row-start"),
+            Bytes.toBytes("row-end"), "location", 666);
 
     TableSplit deserialized = new TableSplit(TableName.valueOf(name.getMethodName()),
-            "row-start2".getBytes(),
-            "row-end2".getBytes(), "location1");
+        Bytes.toBytes("row-start2"),
+            Bytes.toBytes("row-end2"), "location1");
     ReflectionUtils.copy(new Configuration(), split1, deserialized);
 
     Assert.assertEquals(666, deserialized.getLength());
@@ -100,8 +101,8 @@ public class TestTableSplit {
   @Test
   public void testToString() {
     TableSplit split =
-        new TableSplit(TableName.valueOf(name.getMethodName()), "row-start".getBytes(), "row-end".getBytes(),
-            "location");
+        new TableSplit(TableName.valueOf(name.getMethodName()), Bytes.toBytes("row-start"),
+            Bytes.toBytes("row-end"), "location");
     String str =
         "HBase table split(table name: " + name.getMethodName() + ", scan: , start row: row-start, "
             + "end row: row-end, region location: location, "
@@ -109,8 +110,8 @@ public class TestTableSplit {
     Assert.assertEquals(str, split.toString());
 
     split =
-        new TableSplit(TableName.valueOf(name.getMethodName()), null, "row-start".getBytes(),
-            "row-end".getBytes(), "location", "encoded-region-name", 1000L);
+        new TableSplit(TableName.valueOf(name.getMethodName()), null, Bytes.toBytes("row-start"),
+            Bytes.toBytes("row-end"), "location", "encoded-region-name", 1000L);
     str =
         "HBase table split(table name: " + name.getMethodName() + ", scan: , start row: row-start, "
             + "end row: row-end, region location: location, "
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationCrossDiffHdfs.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationCrossDiffHdfs.java
index a07e0a8..aaa5b6c 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationCrossDiffHdfs.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationCrossDiffHdfs.java
@@ -168,15 +168,15 @@ public class TestVerifyReplicationCrossDiffHdfs {
     Path rootDir = FSUtils.getRootDir(conf1);
     FileSystem fs = rootDir.getFileSystem(conf1);
     String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
-    SnapshotTestingUtils.createSnapshotAndValidate(util1.getAdmin(), TABLE_NAME, new String(FAMILY),
-      sourceSnapshotName, rootDir, fs, true);
+    SnapshotTestingUtils.createSnapshotAndValidate(util1.getAdmin(), TABLE_NAME,
+        Bytes.toString(FAMILY), sourceSnapshotName, rootDir, fs, true);
 
     // Take target snapshot
     Path peerRootDir = FSUtils.getRootDir(conf2);
     FileSystem peerFs = peerRootDir.getFileSystem(conf2);
     String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
-    SnapshotTestingUtils.createSnapshotAndValidate(util2.getAdmin(), TABLE_NAME, new String(FAMILY),
-      peerSnapshotName, peerRootDir, peerFs, true);
+    SnapshotTestingUtils.createSnapshotAndValidate(util2.getAdmin(), TABLE_NAME,
+        Bytes.toString(FAMILY), peerSnapshotName, peerRootDir, peerFs, true);
 
     String peerFSAddress = peerFs.getUri().toString();
     String temPath1 = new Path(fs.getUri().toString(), "/tmp1").toString();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
index 9f0cf6b..24c9d78 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
@@ -61,11 +61,11 @@ public class TestSerialization {
 
   @Test public void testKeyValue() throws Exception {
     final String name = "testKeyValue2";
-    byte[] row = name.getBytes();
-    byte[] fam = "fam".getBytes();
-    byte[] qf = "qf".getBytes();
+    byte[] row = Bytes.toBytes(name);
+    byte[] fam = Bytes.toBytes("fam");
+    byte[] qf = Bytes.toBytes("qf");
     long ts = System.currentTimeMillis();
-    byte[] val = "val".getBytes();
+    byte[] val = Bytes.toBytes("val");
     KeyValue kv = new KeyValue(row, fam, qf, ts, val);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
@@ -290,9 +290,9 @@ public class TestSerialization {
   */
 
   @Test public void testGet() throws Exception{
-    byte[] row = "row".getBytes();
-    byte[] fam = "fam".getBytes();
-    byte[] qf1 = "qf1".getBytes();
+    byte[] row = Bytes.toBytes("row");
+    byte[] fam = Bytes.toBytes("fam");
+    byte[] qf1 = Bytes.toBytes("qf1");
 
     long ts = System.currentTimeMillis();
     int maxVersions = 2;
@@ -329,10 +329,10 @@ public class TestSerialization {
 
   @Test public void testScan() throws Exception {
 
-    byte[] startRow = "startRow".getBytes();
-    byte[] stopRow  = "stopRow".getBytes();
-    byte[] fam = "fam".getBytes();
-    byte[] qf1 = "qf1".getBytes();
+    byte[] startRow = Bytes.toBytes("startRow");
+    byte[] stopRow  = Bytes.toBytes("stopRow");
+    byte[] fam = Bytes.toBytes("fam");
+    byte[] qf1 = Bytes.toBytes("qf1");
 
     long ts = System.currentTimeMillis();
     int maxVersions = 2;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index dfc3a2c..376aa92 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -686,7 +686,7 @@ public class TestAdmin1 {
     TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3");
     desc = new HTableDescriptor(TABLE_3);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3);
+    admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 3);
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
       regions = l.getAllRegionLocations();
       assertEquals("Table should have only 3 region", 3, regions.size());
@@ -696,7 +696,7 @@ public class TestAdmin1 {
     desc = new HTableDescriptor(TABLE_4);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     try {
-      admin.createTable(desc, "a".getBytes(), "z".getBytes(), 2);
+      admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 2);
       fail("Should not be able to create a table with only 2 regions using this API.");
     } catch (IllegalArgumentException eae) {
     // Expected
@@ -921,9 +921,9 @@ public class TestAdmin1 {
   public void testCreateTableWithEmptyRowInTheSplitKeys() throws IOException{
     final byte[] tableName = Bytes.toBytes(name.getMethodName());
     byte[][] splitKeys = new byte[3][];
-    splitKeys[0] = "region1".getBytes();
+    splitKeys[0] = Bytes.toBytes("region1");
     splitKeys[1] = HConstants.EMPTY_BYTE_ARRAY;
-    splitKeys[2] = "region2".getBytes();
+    splitKeys[2] = Bytes.toBytes("region2");
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("col"));
     try {
@@ -1181,7 +1181,7 @@ public class TestAdmin1 {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.setRegionReplication(3);
-    byte[] cf = "f".getBytes();
+    byte[] cf = Bytes.toBytes("f");
     HColumnDescriptor hcd = new HColumnDescriptor(cf);
     desc.addFamily(hcd);
     byte[][] splitRows = new byte[2][];
@@ -1196,15 +1196,15 @@ public class TestAdmin1 {
     // write some data to the table
     Table ht = TEST_UTIL.getConnection().getTable(tableName);
     List<Put> puts = new ArrayList<>();
-    byte[] qualifier = "c".getBytes();
+    byte[] qualifier = Bytes.toBytes("c");
     Put put = new Put(new byte[]{(byte)'1'});
-    put.addColumn(cf, qualifier, "100".getBytes());
+    put.addColumn(cf, qualifier, Bytes.toBytes("100"));
     puts.add(put);
     put = new Put(new byte[]{(byte)'6'});
-    put.addColumn(cf, qualifier, "100".getBytes());
+    put.addColumn(cf, qualifier, Bytes.toBytes("100"));
     puts.add(put);
     put = new Put(new byte[]{(byte)'8'});
-    put.addColumn(cf, qualifier, "100".getBytes());
+    put.addColumn(cf, qualifier, Bytes.toBytes("100"));
     puts.add(put);
     ht.put(puts);
     ht.close();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index 1750926..990eed4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -100,7 +100,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "_3");
     TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName3);
     builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
-    admin.createTable(builder.build(), "a".getBytes(), "z".getBytes(), 3).join();
+    admin.createTable(builder.build(), Bytes.toBytes("a"), Bytes.toBytes("z"), 3).join();
     regionLocations =
       AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
     assertEquals("Table should have only 3 region", 3, regionLocations.size());
@@ -109,7 +109,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     builder = TableDescriptorBuilder.newBuilder(tableName4);
     builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
     try {
-      admin.createTable(builder.build(), "a".getBytes(), "z".getBytes(), 2).join();
+      admin.createTable(builder.build(), Bytes.toBytes("a"), Bytes.toBytes("z"), 2).join();
       fail("Should not be able to create a table with only 2 regions using this API.");
     } catch (CompletionException e) {
       assertTrue(e.getCause() instanceof IllegalArgumentException);
@@ -309,9 +309,9 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
   @Test
   public void testCreateTableWithEmptyRowInTheSplitKeys() throws Exception {
     byte[][] splitKeys = new byte[3][];
-    splitKeys[0] = "region1".getBytes();
+    splitKeys[0] = Bytes.toBytes("region1");
     splitKeys[1] = HConstants.EMPTY_BYTE_ARRAY;
-    splitKeys[2] = "region2".getBytes();
+    splitKeys[2] = Bytes.toBytes("region2");
     try {
       createTableWithDefaultConf(tableName, splitKeys);
       fail("Test case should fail as empty split key is passed.");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
index 4d9f39b..9523a61 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
@@ -218,8 +218,8 @@ public class TestConnectionImplementation {
   // dead servers is broke"
   public void testClusterStatus() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    byte[] cf = "cf".getBytes();
-    byte[] rk = "rk1".getBytes();
+    byte[] cf = Bytes.toBytes("cf");
+    byte[] rk = Bytes.toBytes("rk1");
 
     JVMClusterUtil.RegionServerThread rs = TEST_UTIL.getHBaseCluster().startRegionServer();
     rs.waitForServerOnline();
@@ -242,7 +242,7 @@ public class TestConnectionImplementation {
     }
 
     Put p1 = new Put(rk);
-    p1.addColumn(cf, "qual".getBytes(), "val".getBytes());
+    p1.addColumn(cf, Bytes.toBytes("qual"), Bytes.toBytes("val"));
     t.put(p1);
 
     rs.getRegionServer().abort("I'm dead");
@@ -606,7 +606,7 @@ public class TestConnectionImplementation {
     LOG.info("Move starting region="+toMove.getRegionInfo().getRegionNameAsString());
     TEST_UTIL.getAdmin().move(
       toMove.getRegionInfo().getEncodedNameAsBytes(),
-      destServerName.getServerName().getBytes()
+      Bytes.toBytes(destServerName.getServerName())
     );
 
     while (destServer.getOnlineRegion(regionName) == null ||
@@ -672,7 +672,7 @@ public class TestConnectionImplementation {
     LOG.info("Move starting region=" + toMove.getRegionInfo().getRegionNameAsString());
     TEST_UTIL.getAdmin().move(
       toMove.getRegionInfo().getEncodedNameAsBytes(),
-      curServer.getServerName().getServerName().getBytes()
+      Bytes.toBytes(curServer.getServerName().getServerName())
     );
 
     while (curServer.getOnlineRegion(regionName) == null ||
@@ -930,7 +930,7 @@ public class TestConnectionImplementation {
       LOG.info("Move starting region=" + toMove.getRegionInfo().getRegionNameAsString());
       TEST_UTIL.getAdmin().move(
           toMove.getRegionInfo().getEncodedNameAsBytes(),
-          destServerName.getServerName().getBytes()
+          Bytes.toBytes(destServerName.getServerName())
       );
 
       while (destServer.getOnlineRegion(regionName) == null ||
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
index 5d088b3..9e65c57 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
@@ -367,7 +367,7 @@ public class TestFastFail {
   }
 
   private byte[] longToByteArrayKey(long rowKey) {
-    return LoadTestKVGenerator.md5PrefixedKey(rowKey).getBytes();
+    return Bytes.toBytes(LoadTestKVGenerator.md5PrefixedKey(rowKey));
   }
 
   public static class CallQueueTooBigPffeInterceptor extends
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetScanPartialResult.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetScanPartialResult.java
index e9295c4..63976e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetScanPartialResult.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetScanPartialResult.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -77,7 +78,7 @@ public class TestGetScanPartialResult {
       byte[] val = makeLargeValue(VALUE_SIZE);
       Put p = new Put(ROW);
       for (int i = 0; i < NUM_COLUMNS; i++) {
-        p.addColumn(CF, Integer.toString(i).getBytes(), val);
+        p.addColumn(CF, Bytes.toBytes(Integer.toString(i)), val);
       }
       t.put(p);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 3b14b7f..6c633a2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -258,10 +258,10 @@ public class TestMetaWithReplicas {
       LOG.info("Running GETs");
       Get get = null;
       Result r = null;
-      byte[] row = "test".getBytes();
+      byte[] row = Bytes.toBytes("test");
       try (Table htable = c.getTable(TABLE)) {
         Put put = new Put(row);
-        put.addColumn("foo".getBytes(), row, row);
+        put.addColumn(Bytes.toBytes("foo"), row, row);
         BufferedMutator m = c.getBufferedMutator(TABLE);
         m.mutate(put);
         m.flush();
@@ -296,7 +296,7 @@ public class TestMetaWithReplicas {
       TEST_UTIL.getAdmin().deleteTable(tableName);
     }
     try (Table htable = TEST_UTIL.createTable(tableName, FAMILIES)) {
-      byte[] row = "test".getBytes();
+      byte[] row = Bytes.toBytes("test");
       ConnectionImplementation c = ((ConnectionImplementation) TEST_UTIL.getConnection());
       // check that metalookup pool would get created
       c.relocateRegion(tableName, row);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 59920d6..a42b26d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -74,7 +74,7 @@ public class TestReplicaWithCluster {
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicaWithCluster.class);
 
   private static final int NB_SERVERS = 3;
-  private static final byte[] row = TestReplicaWithCluster.class.getName().getBytes();
+  private static final byte[] row = Bytes.toBytes(TestReplicaWithCluster.class.getName());
   private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
 
   // second minicluster used in testing of replication
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index 311f651..4a72410 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -87,7 +87,7 @@ public class TestReplicasClient {
 
   private static final int NB_SERVERS = 1;
   private static Table table = null;
-  private static final byte[] row = TestReplicasClient.class.getName().getBytes();
+  private static final byte[] row = Bytes.toBytes(TestReplicasClient.class.getName());
 
   private static HRegionInfo hriPrimary;
   private static HRegionInfo hriSecondary;
@@ -313,7 +313,7 @@ public class TestReplicasClient {
 
   @Test
   public void testUseRegionWithoutReplica() throws Exception {
-    byte[] b1 = "testUseRegionWithoutReplica".getBytes();
+    byte[] b1 = Bytes.toBytes("testUseRegionWithoutReplica");
     openRegion(hriSecondary);
     SlowMeCopro.getPrimaryCdl().set(new CountDownLatch(0));
     try {
@@ -327,7 +327,7 @@ public class TestReplicasClient {
 
   @Test
   public void testLocations() throws Exception {
-    byte[] b1 = "testLocations".getBytes();
+    byte[] b1 = Bytes.toBytes("testLocations");
     openRegion(hriSecondary);
     ClusterConnection hc = (ClusterConnection) HTU.getAdmin().getConnection();
 
@@ -352,7 +352,7 @@ public class TestReplicasClient {
 
   @Test
   public void testGetNoResultNoStaleRegionWithReplica() throws Exception {
-    byte[] b1 = "testGetNoResultNoStaleRegionWithReplica".getBytes();
+    byte[] b1 = Bytes.toBytes("testGetNoResultNoStaleRegionWithReplica");
     openRegion(hriSecondary);
 
     try {
@@ -368,7 +368,7 @@ public class TestReplicasClient {
 
   @Test
   public void testGetNoResultStaleRegionWithReplica() throws Exception {
-    byte[] b1 = "testGetNoResultStaleRegionWithReplica".getBytes();
+    byte[] b1 = Bytes.toBytes("testGetNoResultStaleRegionWithReplica");
     openRegion(hriSecondary);
 
     SlowMeCopro.getPrimaryCdl().set(new CountDownLatch(1));
@@ -385,7 +385,7 @@ public class TestReplicasClient {
 
   @Test
   public void testGetNoResultNotStaleSleepRegionWithReplica() throws Exception {
-    byte[] b1 = "testGetNoResultNotStaleSleepRegionWithReplica".getBytes();
+    byte[] b1 = Bytes.toBytes("testGetNoResultNotStaleSleepRegionWithReplica");
     openRegion(hriSecondary);
 
     try {
@@ -461,7 +461,7 @@ public class TestReplicasClient {
 
   @Test
   public void testUseRegionWithReplica() throws Exception {
-    byte[] b1 = "testUseRegionWithReplica".getBytes();
+    byte[] b1 = Bytes.toBytes("testUseRegionWithReplica");
     openRegion(hriSecondary);
 
     try {
@@ -554,7 +554,7 @@ public class TestReplicasClient {
 
   @Test
   public void testHedgedRead() throws Exception {
-    byte[] b1 = "testHedgedRead".getBytes();
+    byte[] b1 = Bytes.toBytes("testHedgedRead");
     openRegion(hriSecondary);
 
     try {
@@ -769,7 +769,7 @@ public class TestReplicasClient {
         for (int col = 0; col < NUMCOLS; col++) {
           Put p = new Put(b1);
           String qualifier = "qualifer" + col;
-          KeyValue kv = new KeyValue(b1, f, qualifier.getBytes());
+          KeyValue kv = new KeyValue(b1, f, Bytes.toBytes(qualifier));
           p.add(kv);
           table.put(p);
         }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
index ef3fa10..b222cc9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
@@ -589,7 +589,7 @@ public class TestRegionObserverInterface {
       ServerName sn2 = rs1.getRegionServer().getServerName();
       String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
 
-      util.getAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
+      util.getAdmin().move(Bytes.toBytes(regEN), Bytes.toBytes(sn2.getServerName()));
       while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
         Thread.sleep(100);
       }
@@ -639,7 +639,7 @@ public class TestRegionObserverInterface {
       ServerName sn2 = rs1.getRegionServer().getServerName();
       String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
 
-      util.getAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
+      util.getAdmin().move(Bytes.toBytes(regEN), Bytes.toBytes(sn2.getServerName()));
       while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
         Thread.sleep(100);
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
index 85e02cb..05df8b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
@@ -99,7 +99,7 @@ public class TestFilterWrapper {
         for (Cell kv : result.listCells()) {
           LOG.debug(kv_number + ". kv: " + kv);
           kv_number++;
-          assertEquals("Returned row is not correct", new String(CellUtil.cloneRow(kv)),
+          assertEquals("Returned row is not correct", Bytes.toString(CellUtil.cloneRow(kv)),
               "row" + ( row_number + 1 ));
         }
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java
index a612290..dcd9ceb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java
@@ -125,7 +125,7 @@ public class TestFuzzyRowAndColumnRangeFilter {
 
           Put p = new Put(rk);
           p.setDurability(Durability.SKIP_WAL);
-          p.addColumn(cf.getBytes(), cq, Bytes.toBytes(c));
+          p.addColumn(Bytes.toBytes(cf), cq, Bytes.toBytes(c));
           ht.put(p);
           LOG.info("Inserting: rk: " + Bytes.toStringBinary(rk) + " cq: "
                   + Bytes.toStringBinary(cq));
@@ -167,7 +167,7 @@ public class TestFuzzyRowAndColumnRangeFilter {
   private void runScanner(Table hTable, int expectedSize, Filter... filters) throws IOException {
     String cf = "f";
     Scan scan = new Scan();
-    scan.addFamily(cf.getBytes());
+    scan.addFamily(Bytes.toBytes(cf));
     FilterList filterList = new FilterList(filters);
     scan.setFilter(filterList);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java
index cf32631..ca538aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java
@@ -136,7 +136,7 @@ public class TestFuzzyRowFilterEndToEnd {
 
     for (int i = 0; i < rows.length; i++) {
       Put p = new Put(Bytes.toBytesBinary(rows[i]));
-      p.addColumn(cf.getBytes(), cq.getBytes(), "value".getBytes());
+      p.addColumn(Bytes.toBytes(cf), Bytes.toBytes(cq), Bytes.toBytes("value"));
       ht.put(p);
     }
 
@@ -191,12 +191,12 @@ public class TestFuzzyRowFilterEndToEnd {
 
     for(int i=0; i < rows.length; i++){
       Put p = new Put(Bytes.toBytesBinary(rows[i]));
-      p.addColumn(cf.getBytes(), cq.getBytes(), "value".getBytes());
+      p.addColumn(Bytes.toBytes(cf), Bytes.toBytes(cq), Bytes.toBytes("value"));
       ht.put(p);
     }
 
     Put p = new Put(Bytes.toBytesBinary(badRow));
-    p.addColumn(cf.getBytes(), cq.getBytes(), "value".getBytes());
+    p.addColumn(Bytes.toBytes(cf), Bytes.toBytes(cq), Bytes.toBytes("value"));
     ht.put(p);
 
     TEST_UTIL.flush();
@@ -248,7 +248,7 @@ public class TestFuzzyRowFilterEndToEnd {
 
             Put p = new Put(rk);
             p.setDurability(Durability.SKIP_WAL);
-            p.addColumn(cf.getBytes(), cq, Bytes.toBytes(c));
+            p.addColumn(Bytes.toBytes(cf), cq, Bytes.toBytes(c));
             ht.put(p);
           }
         }
@@ -333,9 +333,9 @@ public class TestFuzzyRowFilterEndToEnd {
 
     String cf = "f";
     Scan scan = new Scan();
-    scan.addFamily(cf.getBytes());
+    scan.addFamily(Bytes.toBytes(cf));
     scan.setFilter(filter);
-    List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(table.getBytes());
+    List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(TableName.valueOf(table));
     HRegion first = regions.get(0);
     first.getScanner(scan);
     RegionScanner scanner = first.getScanner(scan);
@@ -385,7 +385,7 @@ public class TestFuzzyRowFilterEndToEnd {
 
           Put p = new Put(rk);
           p.setDurability(Durability.SKIP_WAL);
-          p.addColumn(cf.getBytes(), cq, Bytes.toBytes(c));
+          p.addColumn(Bytes.toBytes(cf), cq, Bytes.toBytes(c));
           ht.put(p);
           LOG.info("Inserting: rk: " + Bytes.toStringBinary(rk) + " cq: "
               + Bytes.toStringBinary(cq));
@@ -435,7 +435,7 @@ public class TestFuzzyRowFilterEndToEnd {
       throws IOException {
     String cf = "f";
     Scan scan = new Scan();
-    scan.addFamily(cf.getBytes());
+    scan.addFamily(Bytes.toBytes(cf));
     FilterList filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
     scan.setFilter(filterList);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java
index e3b78ed..f531954 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -84,7 +85,7 @@ public class TestScanRowPrefix extends FilterTestingCluster {
     for (byte[] rowId: rowIds) {
       Put p = new Put(rowId);
       // Use the rowId as the column qualifier
-      p.addColumn("F".getBytes(), rowId, "Dummy value".getBytes());
+      p.addColumn(Bytes.toBytes("F"), rowId, Bytes.toBytes("Dummy value"));
       table.put(p);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
index 350a316..12f2121 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
@@ -149,7 +149,7 @@ public class TestEncodedSeekers {
   private void doPuts(HRegion region) throws IOException{
     LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(MIN_VALUE_SIZE, MAX_VALUE_SIZE);
      for (int i = 0; i < NUM_ROWS; ++i) {
-      byte[] key = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
+      byte[] key = Bytes.toBytes(LoadTestKVGenerator.md5PrefixedKey(i));
       for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
         Put put = new Put(key);
         put.setDurability(Durability.ASYNC_WAL);
@@ -177,7 +177,7 @@ public class TestEncodedSeekers {
 
   private void doGets(Region region) throws IOException{
     for (int i = 0; i < NUM_ROWS; ++i) {
-      final byte[] rowKey = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
+      final byte[] rowKey = Bytes.toBytes(LoadTestKVGenerator.md5PrefixedKey(i));
       for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
         final String qualStr = String.valueOf(j);
         if (VERBOSE) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomKeyValueUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomKeyValueUtil.java
index b6ff37e..337a32f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomKeyValueUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomKeyValueUtil.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.io.hfile;
 
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
 
 import java.util.Random;
 
@@ -49,7 +50,7 @@ public class RandomKeyValueUtil {
 
   public static KeyValue randomKeyValue(Random rand) {
     return new KeyValue(randomRowOrQualifier(rand),
-        COLUMN_FAMILY_NAME.getBytes(), randomRowOrQualifier(rand),
+        Bytes.toBytes(COLUMN_FAMILY_NAME), randomRowOrQualifier(rand),
         randomValue(rand));
   }
 
@@ -60,7 +61,7 @@ public class RandomKeyValueUtil {
         - MIN_ROW_OR_QUALIFIER_LENGTH + 1);
     for (int i = 0; i < fieldLen; ++i)
       field.append(randomReadableChar(rand));
-    return field.toString().getBytes();
+    return Bytes.toBytes(field.toString());
   }
 
   public static byte[] randomValue(Random rand) {
@@ -69,7 +70,7 @@ public class RandomKeyValueUtil {
       v.append((char) (32 + rand.nextInt(95)));
     }
 
-    byte[] valueBytes = v.toString().getBytes();
+    byte[] valueBytes = Bytes.toBytes(v.toString());
     return valueBytes;
   }
 
@@ -98,7 +99,7 @@ public class RandomKeyValueUtil {
     for (int j = 0; j < rand.nextInt(50); ++j)
       k.append(randomReadableChar(rand));
 
-    byte[] keyBytes = k.toString().getBytes();
+    byte[] keyBytes = Bytes.toBytes(k.toString());
     return keyBytes;
   }
 
@@ -117,7 +118,7 @@ public class RandomKeyValueUtil {
     for (int j = 0; j < suffixLength; ++j)
       k.append(randomReadableChar(rand));
 
-    byte[] keyBytes = k.toString().getBytes();
+    byte[] keyBytes = Bytes.toBytes(k.toString());
     return keyBytes;
   }
 
@@ -127,7 +128,7 @@ public class RandomKeyValueUtil {
       v.append((char) (32 + rand.nextInt(95)));
     }
 
-    byte[] valueBytes = v.toString().getBytes();
+    byte[] valueBytes = Bytes.toBytes(v.toString());
     return valueBytes;
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 2a613de..bc608be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -289,7 +289,7 @@ public class TestHFile  {
   }
 
   private byte[] getSomeKey(int rowId) {
-    KeyValue kv = new KeyValue(String.format(localFormatter, Integer.valueOf(rowId)).getBytes(),
+    KeyValue kv = new KeyValue(Bytes.toBytes(String.format(localFormatter, Integer.valueOf(rowId))),
         Bytes.toBytes("family"), Bytes.toBytes("qual"), HConstants.LATEST_TIMESTAMP, Type.Put);
     return kv.getKey();
   }
@@ -377,7 +377,7 @@ public class TestHFile  {
 
         @Override
         public void write(DataOutput out) throws IOException {
-          out.write(("something to test" + val).getBytes());
+          out.write(Bytes.toBytes("something to test" + val));
         }
 
         @Override
@@ -394,7 +394,7 @@ public class TestHFile  {
     for (int i = 0; i < n; i++) {
       ByteBuff actual = reader.getMetaBlock("HFileMeta" + i, false).getBufferWithoutHeader();
       ByteBuffer expected =
-        ByteBuffer.wrap(("something to test" + i).getBytes());
+        ByteBuffer.wrap(Bytes.toBytes("something to test" + i));
       assertEquals(
           "failed to match metadata",
           Bytes.toStringBinary(expected),
@@ -451,7 +451,8 @@ public class TestHFile  {
           .withOutputStream(fout)
           .withFileContext(meta)
           .create();
-      KeyValue kv = new KeyValue("foo".getBytes(), "f1".getBytes(), null, "value".getBytes());
+      KeyValue kv = new KeyValue(Bytes.toBytes("foo"), Bytes.toBytes("f1"), null,
+          Bytes.toBytes("value"));
       writer.append(kv);
       writer.close();
       fout.close();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index 78f8584..937b641 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -276,7 +276,7 @@ public class TestHFileBlockIndex {
         new HFileBlockIndex.BlockIndexWriter(hbw, null, null);
 
     for (int i = 0; i < NUM_DATA_BLOCKS; ++i) {
-      hbw.startWriting(BlockType.DATA).write(String.valueOf(rand.nextInt(1000)).getBytes());
+      hbw.startWriting(BlockType.DATA).write(Bytes.toBytes(String.valueOf(rand.nextInt(1000))));
       long blockOffset = outputStream.getPos();
       hbw.writeHeaderAndData(outputStream);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
index 5eeecef..d77af6d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
@@ -179,7 +179,8 @@ public class TestHFileEncryption {
         .withFileContext(fileContext)
         .create();
     try {
-      KeyValue kv = new KeyValue("foo".getBytes(), "f1".getBytes(), null, "value".getBytes());
+      KeyValue kv = new KeyValue(Bytes.toBytes("foo"), Bytes.toBytes("f1"), null,
+          Bytes.toBytes("value"));
       writer.append(kv);
     } finally {
       writer.close();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
index 4dc9c68..d43c0da 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.BytesWritable;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
@@ -66,8 +67,8 @@ public class TestHFileSeek extends TestCase {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestHFileSeek.class);
 
-  private static final byte[] CF = "f1".getBytes();
-  private static final byte[] QUAL = "q1".getBytes();
+  private static final byte[] CF = Bytes.toBytes("f1");
+  private static final byte[] QUAL = Bytes.toBytes("q1");
   private static final boolean USE_PREAD = true;
   private MyOptions options;
   private Configuration conf;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
index 8a0365f..0411679 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.MoveRegionAction;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -259,8 +260,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
     Mockito.when(services.getServerManager()).thenReturn(sm);
     balancer.setMasterServices(services);
     RegionInfo hri1 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-        .setStartKey("key1".getBytes())
-        .setEndKey("key2".getBytes())
+        .setStartKey(Bytes.toBytes("key1"))
+        .setEndKey(Bytes.toBytes("key2"))
         .setSplit(false)
         .setRegionId(100)
         .build();
@@ -284,8 +285,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
     List<RegionInfo> list2 = new ArrayList<>();
     // create a region (region1)
     RegionInfo hri1 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-        .setStartKey("key1".getBytes())
-        .setEndKey("key2".getBytes())
+        .setStartKey(Bytes.toBytes("key1"))
+        .setEndKey(Bytes.toBytes("key2"))
         .setSplit(false)
         .setRegionId(100)
         .build();
@@ -293,8 +294,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
     RegionInfo hri2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1);
     // create a second region (region2)
     RegionInfo hri3 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-        .setStartKey("key2".getBytes())
-        .setEndKey("key3".getBytes())
+        .setStartKey(Bytes.toBytes("key2"))
+        .setEndKey(Bytes.toBytes("key3"))
         .setSplit(false)
         .setRegionId(101)
         .build();
@@ -358,8 +359,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
     List<RegionInfo> list2 = new ArrayList<>();
     // create a region (region1)
     RegionInfo hri1 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-        .setStartKey("key1".getBytes())
-        .setEndKey("key2".getBytes())
+        .setStartKey(Bytes.toBytes("key1"))
+        .setEndKey(Bytes.toBytes("key2"))
         .setSplit(false)
         .setRegionId(100)
         .build();
@@ -367,8 +368,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
     RegionInfo hri2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1);
     // create a second region (region2)
     RegionInfo hri3 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-        .setStartKey("key2".getBytes())
-        .setEndKey("key3".getBytes())
+        .setStartKey(Bytes.toBytes("key2"))
+        .setEndKey(Bytes.toBytes("key3"))
         .setSplit(false)
         .setRegionId(101)
         .build();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
index 9f964c6..d04ef79 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -80,7 +81,8 @@ public class TestLockManager {
     UTIL.startMiniCluster(1);
     masterServices = UTIL.getMiniHBaseCluster().getMaster();
     UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(namespace).build());
-    UTIL.createTable(tableName, new byte[][]{"fam".getBytes()}, new byte[][] {"1".getBytes()});
+    UTIL.createTable(tableName, new byte[][]{Bytes.toBytes("fam")},
+        new byte[][] {Bytes.toBytes("1")});
     List<HRegionInfo> regions = UTIL.getAdmin().getTableRegions(tableName);
     assert regions.size() > 0;
     tableRegions = new HRegionInfo[regions.size()];
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index 98c3978..b4d16c6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -289,7 +289,7 @@ public class MasterProcedureTestingUtility {
     TableDescriptor htd = master.getTableDescriptors().get(tableName);
     assertTrue(htd != null);
 
-    assertTrue(htd.hasColumnFamily(family.getBytes()));
+    assertTrue(htd.hasColumnFamily(Bytes.toBytes(family)));
   }
 
   public static void validateColumnFamilyDeletion(final HMaster master, final TableName tableName,
@@ -297,7 +297,7 @@ public class MasterProcedureTestingUtility {
     // verify htd
     TableDescriptor htd = master.getTableDescriptors().get(tableName);
     assertTrue(htd != null);
-    assertFalse(htd.hasColumnFamily(family.getBytes()));
+    assertFalse(htd.hasColumnFamily(Bytes.toBytes(family)));
 
     // verify fs
     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
@@ -314,7 +314,7 @@ public class MasterProcedureTestingUtility {
     TableDescriptor htd = master.getTableDescriptors().get(tableName);
     assertTrue(htd != null);
 
-    ColumnFamilyDescriptor hcfd = htd.getColumnFamily(family.getBytes());
+    ColumnFamilyDescriptor hcfd = htd.getColumnFamily(Bytes.toBytes(family));
     assertEquals(0, ColumnFamilyDescriptor.COMPARATOR.compare(hcfd, columnDescriptor));
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
index f5a858a..80f931c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.DaemonThreadFactory;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
@@ -249,7 +250,7 @@ public class SimpleRSProcedureManager extends RegionServerProcedureManager {
     @Override
     public byte[] insideBarrier() throws ForeignException {
       execute();
-      return SimpleMasterProcedureManager.SIMPLE_DATA.getBytes();
+      return Bytes.toBytes(SimpleMasterProcedureManager.SIMPLE_DATA);
     }
 
     /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
index 63273de..1b86b50 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -68,6 +69,6 @@ public class TestProcedureManager {
     byte[] result = admin.execProcedureWithRet(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
         "mytest", new HashMap<>());
     assertArrayEquals("Incorrect return data from execProcedure",
-      SimpleMasterProcedureManager.SIMPLE_DATA.getBytes(), result);
+        Bytes.toBytes(SimpleMasterProcedureManager.SIMPLE_DATA), result);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
index 143f30d..6b3de05 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -67,7 +68,7 @@ public class TestZKProcedureControllers {
   private static final String CONTROLLER_NODE_NAME = "controller";
   private static final VerificationMode once = Mockito.times(1);
 
-  private final byte[] memberData = new String("data from member").getBytes();
+  private final byte[] memberData = Bytes.toBytes("data from member");
 
   @BeforeClass
   public static void setupTest() throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
index 3d22b73..6da7abc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
@@ -357,8 +357,8 @@ public class TestCompoundBloomFilter {
 
   @Test
   public void testCreateKey() {
-    byte[] row = "myRow".getBytes();
-    byte[] qualifier = "myQualifier".getBytes();
+    byte[] row = Bytes.toBytes("myRow");
+    byte[] qualifier = Bytes.toBytes("myQualifier");
     // Mimic what Storefile.createBloomKeyValue() does
     byte[] rowKey = KeyValueUtil.createFirstOnRow(row, 0, row.length, new byte[0], 0, 0, row, 0, 0).getKey();
     byte[] rowColKey = KeyValueUtil.createFirstOnRow(row, 0, row.length,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
index 6434ff1..5ce32de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
@@ -106,7 +106,7 @@ public class TestMultiLogThreshold {
       RegionAction.Builder rab = RegionAction.newBuilder();
       rab.setRegion(RequestConverter.buildRegionSpecifier(
         HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME,
-        new String("someStuff" + i).getBytes()));
+        Bytes.toBytes("someStuff" + i)));
       for (int j = 0; j < numAs; j++) {
         Action.Builder ab = Action.newBuilder();
         rab.addAction(ab.build());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java
index 2a604b4..c2552ea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java
@@ -49,7 +49,6 @@ public class TestRegionReplicasWithModifyTable {
 
   private static final int NB_SERVERS = 3;
   private static Table table;
-  private static final byte[] row = "TestRegionReplicasWithModifyTable".getBytes();
 
   private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
   private static final byte[] f = HConstants.CATALOG_FAMILY;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
index 5ad8e21..116f2de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
@@ -82,17 +82,17 @@ public class TestRegionServerReadRequestMetrics {
       LoggerFactory.getLogger(TestRegionServerReadRequestMetrics.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final TableName TABLE_NAME = TableName.valueOf("test");
-  private static final byte[] CF1 = "c1".getBytes();
-  private static final byte[] CF2 = "c2".getBytes();
-
-  private static final byte[] ROW1 = "a".getBytes();
-  private static final byte[] ROW2 = "b".getBytes();
-  private static final byte[] ROW3 = "c".getBytes();
-  private static final byte[] COL1 = "q1".getBytes();
-  private static final byte[] COL2 = "q2".getBytes();
-  private static final byte[] COL3 = "q3".getBytes();
-  private static final byte[] VAL1 = "v1".getBytes();
-  private static final byte[] VAL2 = "v2".getBytes();
+  private static final byte[] CF1 = Bytes.toBytes("c1");
+  private static final byte[] CF2 = Bytes.toBytes("c2");
+
+  private static final byte[] ROW1 = Bytes.toBytes("a");
+  private static final byte[] ROW2 = Bytes.toBytes("b");
+  private static final byte[] ROW3 = Bytes.toBytes("c");
+  private static final byte[] COL1 = Bytes.toBytes("q1");
+  private static final byte[] COL2 = Bytes.toBytes("q2");
+  private static final byte[] COL3 = Bytes.toBytes("q3");
+  private static final byte[] VAL1 = Bytes.toBytes("v1");
+  private static final byte[] VAL2 = Bytes.toBytes("v2");
   private static final byte[] VAL3 = Bytes.toBytes(0L);
 
   private static final int MAX_TRY = 20;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index f24515d..b91bf11 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -455,12 +455,12 @@ public class TestSplitTransactionOnCluster {
     try {
       for (int i = 0; i <= 5; i++) {
         String row = "row" + i;
-        Put p = new Put(row.getBytes());
+        Put p = new Put(Bytes.toBytes(row));
         String val = "Val" + i;
-        p.addColumn("col".getBytes(), "ql".getBytes(), val.getBytes());
+        p.addColumn(Bytes.toBytes("col"), Bytes.toBytes("ql"), Bytes.toBytes(val));
         table.put(p);
         admin.flush(userTableName);
-        Delete d = new Delete(row.getBytes());
+        Delete d = new Delete(Bytes.toBytes(row));
         // Do a normal delete
         table.delete(d);
         admin.flush(userTableName);
@@ -471,17 +471,17 @@ public class TestSplitTransactionOnCluster {
           .getRegionsOfTable(userTableName);
       assertEquals(1, regionsOfTable.size());
       RegionInfo hRegionInfo = regionsOfTable.get(0);
-      Put p = new Put("row6".getBytes());
-      p.addColumn("col".getBytes(), "ql".getBytes(), "val".getBytes());
+      Put p = new Put(Bytes.toBytes("row6"));
+      p.addColumn(Bytes.toBytes("col"), Bytes.toBytes("ql"), Bytes.toBytes("val"));
       table.put(p);
-      p = new Put("row7".getBytes());
-      p.addColumn("col".getBytes(), "ql".getBytes(), "val".getBytes());
+      p = new Put(Bytes.toBytes("row7"));
+      p.addColumn(Bytes.toBytes("col"), Bytes.toBytes("ql"), Bytes.toBytes("val"));
       table.put(p);
-      p = new Put("row8".getBytes());
-      p.addColumn("col".getBytes(), "ql".getBytes(), "val".getBytes());
+      p = new Put(Bytes.toBytes("row8"));
+      p.addColumn(Bytes.toBytes("col"), Bytes.toBytes("ql"), Bytes.toBytes("val"));
       table.put(p);
       admin.flush(userTableName);
-      admin.splitRegionAsync(hRegionInfo.getRegionName(), "row7".getBytes());
+      admin.splitRegionAsync(hRegionInfo.getRegionName(), Bytes.toBytes("row7"));
       regionsOfTable = cluster.getMaster()
           .getAssignmentManager().getRegionStates()
           .getRegionsOfTable(userTableName);
@@ -630,7 +630,7 @@ public class TestSplitTransactionOnCluster {
           tableName);
       assertEquals("The specified table should be present.", true, tableExists);
       // exists works on stale and we see the put after the flush
-      byte[] b1 = "row1".getBytes();
+      byte[] b1 = Bytes.toBytes("row1");
       Get g = new Get(b1);
       g.setConsistency(Consistency.STRONG);
       // The following GET will make a trip to the meta to get the new location of the 1st daughter
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index 599260b..2333b27 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -143,7 +143,7 @@ public class TestWALMonotonicallyIncreasingSeqId {
         for (int i = 0; i < 100; i++) {
           byte[] row = Bytes.toBytes("putRow" + i);
           Put put = new Put(row);
-          put.addColumn("cf".getBytes(), Bytes.toBytes(0), Bytes.toBytes(""));
+          put.addColumn(Bytes.toBytes("cf"), Bytes.toBytes(0), new byte[0]);
           latch.await();
           region.batchMutate(new Mutation[] { put });
           Thread.sleep(10);
@@ -168,7 +168,7 @@ public class TestWALMonotonicallyIncreasingSeqId {
         for (int i = 0; i < 100; i++) {
           byte[] row = Bytes.toBytes("incrementRow" + i);
           Increment inc = new Increment(row);
-          inc.addColumn("cf".getBytes(), Bytes.toBytes(0), 1);
+          inc.addColumn(Bytes.toBytes("cf"), Bytes.toBytes(0), 1);
           // inc.setDurability(Durability.ASYNC_WAL);
           region.increment(inc);
           latch.countDown();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestStoreHotnessProtector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestStoreHotnessProtector.java
index 6d41934..67ac1c3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestStoreHotnessProtector.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestStoreHotnessProtector.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -66,7 +67,7 @@ public class TestStoreHotnessProtector {
 
     Store mockStore1 = mock(Store.class);
     RegionInfo mockRegionInfo = mock(RegionInfo.class);
-    byte[] family = "testF1".getBytes();
+    byte[] family = Bytes.toBytes("testF1");
 
     when(mockRegion.getStore(family)).thenReturn(mockStore1);
     when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index ab67d94..1e4a49e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.replication.ReplicationEndpoint.ReplicateContext;
 import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint.RegionReplicaReplayCallable;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -90,7 +91,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
   private static TableName tableName = TableName.valueOf(
     TestRegionReplicaReplicationEndpointNoMaster.class.getSimpleName());
   private static Table table;
-  private static final byte[] row = "TestRegionReplicaReplicator".getBytes();
+  private static final byte[] row = Bytes.toBytes("TestRegionReplicaReplicator");
 
   private static HRegionServer rs0;
   private static HRegionServer rs1;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
index 110afcd..67c43ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
@@ -876,7 +876,7 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCheckAndPut(ObserverContextImpl.createAndPrepare(RCP_ENV),
           TEST_ROW, TEST_FAMILY, TEST_Q1, CompareOperator.EQUAL,
-          new BinaryComparator("foo".getBytes()), new Put(TEST_ROW), true);
+          new BinaryComparator(Bytes.toBytes("foo")), new Put(TEST_ROW), true);
         return null;
       }
     }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
@@ -887,7 +887,7 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCheckAndDelete(ObserverContextImpl.createAndPrepare(RCP_ENV),
           TEST_ROW, TEST_FAMILY, TEST_Q1, CompareOperator.EQUAL,
-          new BinaryComparator("foo".getBytes()), new Delete(TEST_ROW), true);
+          new BinaryComparator(Bytes.toBytes("foo")), new Delete(TEST_ROW), true);
         return null;
       }
     }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java
index 129823e..7c04edc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java
@@ -648,45 +648,45 @@ public class TestLoadIncrementalHFiles {
 
     first = "a";
     last = "e";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     first = "r";
     last = "s";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     first = "o";
     last = "p";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     first = "g";
     last = "k";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     first = "v";
     last = "x";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     first = "c";
     last = "i";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     first = "m";
     last = "q";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     first = "s";
     last = "t";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     first = "u";
     last = "w";
-    addStartEndKeysForTest(map, first.getBytes(), last.getBytes());
+    addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
     byte[][] keysArray = LoadIncrementalHFiles.inferBoundaries(map);
     byte[][] compare = new byte[3][];
-    compare[0] = "m".getBytes();
-    compare[1] = "r".getBytes();
-    compare[2] = "u".getBytes();
+    compare[0] = Bytes.toBytes("m");
+    compare[1] = Bytes.toBytes("r");
+    compare[2] = Bytes.toBytes("u");
 
     assertEquals(3, keysArray.length);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java
index a59d347..57484d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java
@@ -58,7 +58,7 @@ public class TestBloomFilterChunk extends TestCase {
         (int) bf2.byteSize, bf2.hash, bf2.hashCount));
 
     byte [] bkey = {1,2,3,4};
-    byte [] bval = "this is a much larger byte array".getBytes();
+    byte [] bval = Bytes.toBytes("this is a much larger byte array");
 
     bf1.add(bkey, 0, bkey.length);
     bf1.add(bval, 1, bval.length-1);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover.java
index 670248d..0d768e4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover.java
@@ -85,7 +85,7 @@ public class TestRegionMover {
       .setColumnFamily(ColumnFamilyDescriptorBuilder.of("fam1")).build();
     String startKey = "a";
     String endKey = "z";
-    admin.createTable(tableDesc, startKey.getBytes(), endKey.getBytes(), 9);
+    admin.createTable(tableDesc, Bytes.toBytes(startKey), Bytes.toBytes(endKey), 9);
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java
index ace2238..8fdcc53 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java
@@ -82,218 +82,220 @@ public class TestRegionSplitter {
     /**
      * Test creating a pre-split table using the HexStringSplit algorithm.
      */
-    @Test
-    public void testCreatePresplitTableHex() throws Exception {
-      final List<byte[]> expectedBounds = new ArrayList<>(17);
-      expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
-      expectedBounds.add("10000000".getBytes());
-      expectedBounds.add("20000000".getBytes());
-      expectedBounds.add("30000000".getBytes());
-      expectedBounds.add("40000000".getBytes());
-      expectedBounds.add("50000000".getBytes());
-      expectedBounds.add("60000000".getBytes());
-      expectedBounds.add("70000000".getBytes());
-      expectedBounds.add("80000000".getBytes());
-      expectedBounds.add("90000000".getBytes());
-      expectedBounds.add("a0000000".getBytes());
-      expectedBounds.add("b0000000".getBytes());
-      expectedBounds.add("c0000000".getBytes());
-      expectedBounds.add("d0000000".getBytes());
-      expectedBounds.add("e0000000".getBytes());
-      expectedBounds.add("f0000000".getBytes());
-          expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
+  @Test
+  public void testCreatePresplitTableHex() throws Exception {
+    final List<byte[]> expectedBounds = new ArrayList<>(17);
+    expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
+    expectedBounds.add(Bytes.toBytes("10000000"));
+    expectedBounds.add(Bytes.toBytes("20000000"));
+    expectedBounds.add(Bytes.toBytes("30000000"));
+    expectedBounds.add(Bytes.toBytes("40000000"));
+    expectedBounds.add(Bytes.toBytes("50000000"));
+    expectedBounds.add(Bytes.toBytes("60000000"));
+    expectedBounds.add(Bytes.toBytes("70000000"));
+    expectedBounds.add(Bytes.toBytes("80000000"));
+    expectedBounds.add(Bytes.toBytes("90000000"));
+    expectedBounds.add(Bytes.toBytes("a0000000"));
+    expectedBounds.add(Bytes.toBytes("b0000000"));
+    expectedBounds.add(Bytes.toBytes("c0000000"));
+    expectedBounds.add(Bytes.toBytes("d0000000"));
+    expectedBounds.add(Bytes.toBytes("e0000000"));
+    expectedBounds.add(Bytes.toBytes("f0000000"));
+    expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
 
           // Do table creation/pre-splitting and verification of region boundaries
     preSplitTableAndVerify(expectedBounds,
         HexStringSplit.class.getSimpleName(),
         TableName.valueOf(name.getMethodName()));
-    }
+  }
 
-    /**
-     * Test creating a pre-split table using the UniformSplit algorithm.
-     */
-    @Test
-    public void testCreatePresplitTableUniform() throws Exception {
-      List<byte[]> expectedBounds = new ArrayList<>(17);
-      expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
-      expectedBounds.add(new byte[] {      0x10, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {      0x20, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {      0x30, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {      0x40, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] { 0x50, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(new byte[] { 0x60, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(new byte[] { 0x70, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(new byte[] { (byte) 0x90, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(new byte[] {(byte)0xa0, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] { (byte) 0xb0, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(new byte[] { (byte) 0xc0, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(new byte[] { (byte) 0xd0, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(new byte[] {(byte)0xe0, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] { (byte) 0xf0, 0, 0, 0, 0, 0, 0, 0 });
-      expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
-
-      // Do table creation/pre-splitting and verification of region boundaries
-      preSplitTableAndVerify(expectedBounds, UniformSplit.class.getSimpleName(),
-          TableName.valueOf(name.getMethodName()));
-    }
+  /**
+   * Test creating a pre-split table using the UniformSplit algorithm.
+   */
+  @Test
+  public void testCreatePresplitTableUniform() throws Exception {
+    List<byte[]> expectedBounds = new ArrayList<>(17);
+    expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
+    expectedBounds.add(new byte[] {      0x10, 0, 0, 0, 0, 0, 0, 0});
+    expectedBounds.add(new byte[] {      0x20, 0, 0, 0, 0, 0, 0, 0});
+    expectedBounds.add(new byte[] {      0x30, 0, 0, 0, 0, 0, 0, 0});
+    expectedBounds.add(new byte[] {      0x40, 0, 0, 0, 0, 0, 0, 0});
+    expectedBounds.add(new byte[] { 0x50, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(new byte[] { 0x60, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(new byte[] { 0x70, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(new byte[] { (byte) 0x90, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(new byte[] {(byte)0xa0, 0, 0, 0, 0, 0, 0, 0});
+    expectedBounds.add(new byte[] { (byte) 0xb0, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(new byte[] { (byte) 0xc0, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(new byte[] { (byte) 0xd0, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(new byte[] {(byte)0xe0, 0, 0, 0, 0, 0, 0, 0});
+    expectedBounds.add(new byte[] { (byte) 0xf0, 0, 0, 0, 0, 0, 0, 0 });
+    expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
 
-    /**
-     * Unit tests for the HexStringSplit algorithm. Makes sure it divides up the
-     * space of keys in the way that we expect.
-     */
-    @Test
-    public void unitTestHexStringSplit() {
-        HexStringSplit splitter = new HexStringSplit();
-        // Check splitting while starting from scratch
-
-        byte[][] twoRegionsSplits = splitter.split(2);
-        assertEquals(1, twoRegionsSplits.length);
-        assertArrayEquals("80000000".getBytes(), twoRegionsSplits[0]);
-
-        byte[][] threeRegionsSplits = splitter.split(3);
-        assertEquals(2, threeRegionsSplits.length);
-        byte[] expectedSplit0 = "55555555".getBytes();
-        assertArrayEquals(expectedSplit0, threeRegionsSplits[0]);
-        byte[] expectedSplit1 = "aaaaaaaa".getBytes();
-        assertArrayEquals(expectedSplit1, threeRegionsSplits[1]);
-
-        // Check splitting existing regions that have start and end points
-        byte[] splitPoint = splitter.split("10000000".getBytes(), "30000000".getBytes());
-        assertArrayEquals("20000000".getBytes(), splitPoint);
-
-        byte[] lastRow = "ffffffff".getBytes();
-        assertArrayEquals(lastRow, splitter.lastRow());
-        byte[] firstRow = "00000000".getBytes();
-        assertArrayEquals(firstRow, splitter.firstRow());
-
-        // Halfway between 00... and 20... should be 10...
-        splitPoint = splitter.split(firstRow, "20000000".getBytes());
-        assertArrayEquals("10000000".getBytes(), splitPoint);
-
-        // Halfway between df... and ff... should be ef....
-        splitPoint = splitter.split("dfffffff".getBytes(), lastRow);
-        assertArrayEquals("efffffff".getBytes(), splitPoint);
-
-        // Check splitting region with multiple mappers per region
-        byte[][] splits = splitter.split("00000000".getBytes(), "30000000".getBytes(), 3, false);
-        assertEquals(2, splits.length);
-        assertArrayEquals("10000000".getBytes(), splits[0]);
-        assertArrayEquals("20000000".getBytes(), splits[1]);
-
-        splits = splitter.split("00000000".getBytes(), "20000000".getBytes(), 2, true);
-        assertEquals(3, splits.length);
-        assertArrayEquals("10000000".getBytes(), splits[1]);
-    }
+    // Do table creation/pre-splitting and verification of region boundaries
+    preSplitTableAndVerify(expectedBounds, UniformSplit.class.getSimpleName(),
+        TableName.valueOf(name.getMethodName()));
+  }
 
-    /**
-     * Unit tests for the DecimalStringSplit algorithm. Makes sure it divides up the
-     * space of keys in the way that we expect.
-     */
-    @Test
-    public void unitTestDecimalStringSplit() {
-        DecimalStringSplit splitter = new DecimalStringSplit();
-        // Check splitting while starting from scratch
-
-        byte[][] twoRegionsSplits = splitter.split(2);
-        assertEquals(1, twoRegionsSplits.length);
-        assertArrayEquals("50000000".getBytes(), twoRegionsSplits[0]);
-
-        byte[][] threeRegionsSplits = splitter.split(3);
-        assertEquals(2, threeRegionsSplits.length);
-        byte[] expectedSplit0 = "33333333".getBytes();
-        assertArrayEquals(expectedSplit0, threeRegionsSplits[0]);
-        byte[] expectedSplit1 = "66666666".getBytes();
-        assertArrayEquals(expectedSplit1, threeRegionsSplits[1]);
-
-        // Check splitting existing regions that have start and end points
-        byte[] splitPoint = splitter.split("10000000".getBytes(), "30000000".getBytes());
-        assertArrayEquals("20000000".getBytes(), splitPoint);
-
-        byte[] lastRow = "99999999".getBytes();
-        assertArrayEquals(lastRow, splitter.lastRow());
-        byte[] firstRow = "00000000".getBytes();
-        assertArrayEquals(firstRow, splitter.firstRow());
-
-        // Halfway between 00... and 20... should be 10...
-        splitPoint = splitter.split(firstRow, "20000000".getBytes());
-        assertArrayEquals("10000000".getBytes(), splitPoint);
-
-        // Halfway between 00... and 19... should be 09...
-        splitPoint = splitter.split(firstRow, "19999999".getBytes());
-        assertArrayEquals("09999999".getBytes(), splitPoint);
-
-        // Halfway between 79... and 99... should be 89....
-        splitPoint = splitter.split("79999999".getBytes(), lastRow);
-        assertArrayEquals("89999999".getBytes(), splitPoint);
-
-        // Check splitting region with multiple mappers per region
-        byte[][] splits = splitter.split("00000000".getBytes(), "30000000".getBytes(), 3, false);
-        assertEquals(2, splits.length);
-        assertArrayEquals("10000000".getBytes(), splits[0]);
-        assertArrayEquals("20000000".getBytes(), splits[1]);
-
-        splits = splitter.split("00000000".getBytes(), "20000000".getBytes(), 2, true);
-        assertEquals(3, splits.length);
-        assertArrayEquals("10000000".getBytes(), splits[1]);
-    }
+  /**
+   * Unit tests for the HexStringSplit algorithm. Makes sure it divides up the
+   * space of keys in the way that we expect.
+   */
+  @Test
+  public void unitTestHexStringSplit() {
+    HexStringSplit splitter = new HexStringSplit();
+    // Check splitting while starting from scratch
+
+    byte[][] twoRegionsSplits = splitter.split(2);
+    assertEquals(1, twoRegionsSplits.length);
+    assertArrayEquals(Bytes.toBytes("80000000"), twoRegionsSplits[0]);
+
+    byte[][] threeRegionsSplits = splitter.split(3);
+    assertEquals(2, threeRegionsSplits.length);
+    byte[] expectedSplit0 = Bytes.toBytes("55555555");
+    assertArrayEquals(expectedSplit0, threeRegionsSplits[0]);
+    byte[] expectedSplit1 = Bytes.toBytes("aaaaaaaa");
+    assertArrayEquals(expectedSplit1, threeRegionsSplits[1]);
+
+    // Check splitting existing regions that have start and end points
+    byte[] splitPoint = splitter.split(Bytes.toBytes("10000000"), Bytes.toBytes("30000000"));
+    assertArrayEquals(Bytes.toBytes("20000000"), splitPoint);
+
+    byte[] lastRow = Bytes.toBytes("ffffffff");
+    assertArrayEquals(lastRow, splitter.lastRow());
+    byte[] firstRow = Bytes.toBytes("00000000");
+    assertArrayEquals(firstRow, splitter.firstRow());
+
+    // Halfway between 00... and 20... should be 10...
+    splitPoint = splitter.split(firstRow, Bytes.toBytes("20000000"));
+    assertArrayEquals(Bytes.toBytes("10000000"), splitPoint);
+
+    // Halfway between df... and ff... should be ef....
+    splitPoint = splitter.split(Bytes.toBytes("dfffffff"), lastRow);
+    assertArrayEquals(Bytes.toBytes("efffffff"), splitPoint);
+
+    // Check splitting region with multiple mappers per region
+    byte[][] splits = splitter.split(Bytes.toBytes("00000000"), Bytes.toBytes("30000000"),
+        3, false);
+    assertEquals(2, splits.length);
+    assertArrayEquals(Bytes.toBytes("10000000"), splits[0]);
+    assertArrayEquals(Bytes.toBytes("20000000"), splits[1]);
+
+    splits = splitter.split(Bytes.toBytes("00000000"), Bytes.toBytes("20000000"), 2, true);
+    assertEquals(3, splits.length);
+    assertArrayEquals(Bytes.toBytes("10000000"), splits[1]);
+  }
 
-    /**
-     * Unit tests for the UniformSplit algorithm. Makes sure it divides up the space of
-     * keys in the way that we expect.
-     */
-    @Test
-    public void unitTestUniformSplit() {
-        UniformSplit splitter = new UniformSplit();
-
-        // Check splitting while starting from scratch
-        try {
-            splitter.split(1);
-            throw new AssertionError("Splitting into <2 regions should have thrown exception");
-        } catch (IllegalArgumentException e) { }
-
-        byte[][] twoRegionsSplits = splitter.split(2);
-        assertEquals(1, twoRegionsSplits.length);
-        assertArrayEquals(twoRegionsSplits[0], new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
-
-        byte[][] threeRegionsSplits = splitter.split(3);
-        assertEquals(2, threeRegionsSplits.length);
-        byte[] expectedSplit0 = new byte[] {0x55, 0x55, 0x55, 0x55, 0x55, 0x55, 0x55, 0x55};
-        assertArrayEquals(expectedSplit0, threeRegionsSplits[0]);
-        byte[] expectedSplit1 = new byte[] {(byte)0xAA, (byte)0xAA, (byte)0xAA, (byte)0xAA,
-                (byte)0xAA, (byte)0xAA, (byte)0xAA, (byte)0xAA};
-        assertArrayEquals(expectedSplit1, threeRegionsSplits[1]);
-
-        // Check splitting existing regions that have start and end points
-        byte[] splitPoint = splitter.split(new byte[] {0x10}, new byte[] {0x30});
-        assertArrayEquals(new byte[] { 0x20 }, splitPoint);
-
-        byte[] lastRow = new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF};
-        assertArrayEquals(lastRow, splitter.lastRow());
-        byte[] firstRow = ArrayUtils.EMPTY_BYTE_ARRAY;
-        assertArrayEquals(firstRow, splitter.firstRow());
-
-        splitPoint = splitter.split(firstRow, new byte[] {0x20});
-        assertArrayEquals(splitPoint, new byte[] { 0x10 });
-
-        splitPoint = splitter.split(new byte[] {(byte)0xdf, xFF, xFF, xFF, xFF,
-                xFF, xFF, xFF}, lastRow);
-        assertArrayEquals(splitPoint, new byte[] { (byte) 0xef, xFF, xFF, xFF, xFF, xFF, xFF, xFF
-        });
-
-        splitPoint = splitter.split(new byte[] {'a', 'a', 'a'}, new byte[] {'a', 'a', 'b'});
-        assertArrayEquals(splitPoint, new byte[] { 'a', 'a', 'a', (byte) 0x80 });
-
-        // Check splitting region with multiple mappers per region
-        byte[][] splits = splitter.split(new byte[] {'a', 'a', 'a'}, new byte[] {'a', 'a', 'd'}, 3, false);
-        assertEquals(2, splits.length);
-        assertArrayEquals(splits[0], new byte[]{'a', 'a', 'b'});
-        assertArrayEquals(splits[1], new byte[]{'a', 'a', 'c'});
-
-        splits = splitter.split(new byte[] {'a', 'a', 'a'}, new byte[] {'a', 'a', 'e'}, 2, true);
-        assertEquals(3, splits.length);
-        assertArrayEquals(splits[1], new byte[] { 'a', 'a', 'c'});
-    }
+  /**
+   * Unit tests for the DecimalStringSplit algorithm. Makes sure it divides up the
+   * space of keys in the way that we expect.
+   */
+  @Test
+  public void unitTestDecimalStringSplit() {
+    DecimalStringSplit splitter = new DecimalStringSplit();
+    // Check splitting while starting from scratch
+
+    byte[][] twoRegionsSplits = splitter.split(2);
+    assertEquals(1, twoRegionsSplits.length);
+    assertArrayEquals(Bytes.toBytes("50000000"), twoRegionsSplits[0]);
+
+    byte[][] threeRegionsSplits = splitter.split(3);
+    assertEquals(2, threeRegionsSplits.length);
+    byte[] expectedSplit0 = Bytes.toBytes("33333333");
+    assertArrayEquals(expectedSplit0, threeRegionsSplits[0]);
+    byte[] expectedSplit1 = Bytes.toBytes("66666666");
+    assertArrayEquals(expectedSplit1, threeRegionsSplits[1]);
+
+    // Check splitting existing regions that have start and end points
+    byte[] splitPoint = splitter.split(Bytes.toBytes("10000000"), Bytes.toBytes("30000000"));
+    assertArrayEquals(Bytes.toBytes("20000000"), splitPoint);
+
+    byte[] lastRow = Bytes.toBytes("99999999");
+    assertArrayEquals(lastRow, splitter.lastRow());
+    byte[] firstRow = Bytes.toBytes("00000000");
+    assertArrayEquals(firstRow, splitter.firstRow());
+
+    // Halfway between 00... and 20... should be 10...
+    splitPoint = splitter.split(firstRow, Bytes.toBytes("20000000"));
+    assertArrayEquals(Bytes.toBytes("10000000"), splitPoint);
+
+    // Halfway between 00... and 19... should be 09...
+    splitPoint = splitter.split(firstRow, Bytes.toBytes("19999999"));
+    assertArrayEquals(Bytes.toBytes("09999999"), splitPoint);
+
+    // Halfway between 79... and 99... should be 89....
+    splitPoint = splitter.split(Bytes.toBytes("79999999"), lastRow);
+    assertArrayEquals(Bytes.toBytes("89999999"), splitPoint);
+
+    // Check splitting region with multiple mappers per region
+    byte[][] splits = splitter.split(Bytes.toBytes("00000000"), Bytes.toBytes("30000000"),
+        3, false);
+    assertEquals(2, splits.length);
+    assertArrayEquals(Bytes.toBytes("10000000"), splits[0]);
+    assertArrayEquals(Bytes.toBytes("20000000"), splits[1]);
+
+    splits = splitter.split(Bytes.toBytes("00000000"), Bytes.toBytes("20000000"), 2, true);
+    assertEquals(3, splits.length);
+    assertArrayEquals(Bytes.toBytes("10000000"), splits[1]);
+  }
+
+  /**
+   * Unit tests for the UniformSplit algorithm. Makes sure it divides up the space of
+   * keys in the way that we expect.
+   */
+  @Test
+  public void unitTestUniformSplit() {
+    UniformSplit splitter = new UniformSplit();
+
+    // Check splitting while starting from scratch
+    try {
+      splitter.split(1);
+      throw new AssertionError("Splitting into <2 regions should have thrown exception");
+    } catch (IllegalArgumentException e) { }
+
+    byte[][] twoRegionsSplits = splitter.split(2);
+    assertEquals(1, twoRegionsSplits.length);
+    assertArrayEquals(twoRegionsSplits[0], new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
+
+    byte[][] threeRegionsSplits = splitter.split(3);
+    assertEquals(2, threeRegionsSplits.length);
+    byte[] expectedSplit0 = new byte[] {0x55, 0x55, 0x55, 0x55, 0x55, 0x55, 0x55, 0x55};
+    assertArrayEquals(expectedSplit0, threeRegionsSplits[0]);
+    byte[] expectedSplit1 = new byte[] {(byte)0xAA, (byte)0xAA, (byte)0xAA, (byte)0xAA,
+      (byte)0xAA, (byte)0xAA, (byte)0xAA, (byte)0xAA};
+    assertArrayEquals(expectedSplit1, threeRegionsSplits[1]);
+
+    // Check splitting existing regions that have start and end points
+    byte[] splitPoint = splitter.split(new byte[] {0x10}, new byte[] {0x30});
+    assertArrayEquals(new byte[] { 0x20 }, splitPoint);
+
+    byte[] lastRow = new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF};
+    assertArrayEquals(lastRow, splitter.lastRow());
+    byte[] firstRow = ArrayUtils.EMPTY_BYTE_ARRAY;
+    assertArrayEquals(firstRow, splitter.firstRow());
+
+    splitPoint = splitter.split(firstRow, new byte[] {0x20});
+    assertArrayEquals(splitPoint, new byte[] { 0x10 });
+
+    splitPoint = splitter.split(new byte[] {(byte)0xdf, xFF, xFF, xFF, xFF,
+      xFF, xFF, xFF}, lastRow);
+    assertArrayEquals(splitPoint, new byte[] { (byte) 0xef, xFF, xFF, xFF, xFF, xFF, xFF, xFF});
+
+    splitPoint = splitter.split(new byte[] {'a', 'a', 'a'}, new byte[] {'a', 'a', 'b'});
+    assertArrayEquals(splitPoint, new byte[] { 'a', 'a', 'a', (byte) 0x80 });
+
+    // Check splitting region with multiple mappers per region
+    byte[][] splits = splitter.split(new byte[] {'a', 'a', 'a'}, new byte[] {'a', 'a', 'd'},
+        3, false);
+    assertEquals(2, splits.length);
+    assertArrayEquals(splits[0], new byte[]{'a', 'a', 'b'});
+    assertArrayEquals(splits[1], new byte[]{'a', 'a', 'c'});
+
+    splits = splitter.split(new byte[] {'a', 'a', 'a'}, new byte[] {'a', 'a', 'e'}, 2, true);
+    assertEquals(3, splits.length);
+    assertArrayEquals(splits[1], new byte[] { 'a', 'a', 'c'});
+  }
 
   @Test
   public void testUserInput() {
@@ -362,21 +364,21 @@ public class TestRegionSplitter {
     }
   }
 
-    /**
-     * Creates a pre-split table with expectedBounds.size()+1 regions, then
-     * verifies that the region boundaries are the same as the expected
-     * region boundaries in expectedBounds.
-     * @throws Various junit assertions
-     */
-    private void preSplitTableAndVerify(List<byte[]> expectedBounds,
-            String splitClass, TableName tableName) throws Exception {
-        final int numRegions = expectedBounds.size()-1;
-        final Configuration conf = UTIL.getConfiguration();
-        conf.setInt("split.count", numRegions);
-        SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass);
-        RegionSplitter.createPresplitTable(tableName, splitAlgo, new String[] { CF_NAME }, conf);
-        verifyBounds(expectedBounds, tableName);
-    }
+  /**
+   * Creates a pre-split table with expectedBounds.size()+1 regions, then
+   * verifies that the region boundaries are the same as the expected
+   * region boundaries in expectedBounds.
+   * @throws Various junit assertions
+   */
+  private void preSplitTableAndVerify(List<byte[]> expectedBounds,
+      String splitClass, TableName tableName) throws Exception {
+    final int numRegions = expectedBounds.size()-1;
+    final Configuration conf = UTIL.getConfiguration();
+    conf.setInt("split.count", numRegions);
+    SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass);
+    RegionSplitter.createPresplitTable(tableName, splitAlgo, new String[] { CF_NAME }, conf);
+    verifyBounds(expectedBounds, tableName);
+  }
 
   @Test
   public void noopRollingSplit() throws Exception {
@@ -386,60 +388,58 @@ public class TestRegionSplitter {
         "UniformSplit", expectedBounds);
   }
 
-    private void rollingSplitAndVerify(TableName tableName, String splitClass,
-            List<byte[]> expectedBounds)  throws Exception {
-        final Configuration conf = UTIL.getConfiguration();
+  private void rollingSplitAndVerify(TableName tableName, String splitClass,
+      List<byte[]> expectedBounds)  throws Exception {
+    final Configuration conf = UTIL.getConfiguration();
 
-        // Set this larger than the number of splits so RegionSplitter won't block
-        conf.setInt("split.outstanding", 5);
-        SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass);
-        RegionSplitter.rollingSplit(tableName, splitAlgo, conf);
-        verifyBounds(expectedBounds, tableName);
-    }
+    // Set this larger than the number of splits so RegionSplitter won't block
+    conf.setInt("split.outstanding", 5);
+    SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass);
+    RegionSplitter.rollingSplit(tableName, splitAlgo, conf);
+    verifyBounds(expectedBounds, tableName);
+  }
 
-    private void verifyBounds(List<byte[]> expectedBounds, TableName tableName)
-            throws Exception {
-      // Get region boundaries from the cluster and verify their endpoints
-      final int numRegions = expectedBounds.size()-1;
-      try (Table table = UTIL.getConnection().getTable(tableName);
-          RegionLocator locator = UTIL.getConnection().getRegionLocator(tableName)) {
-        final List<HRegionLocation> regionInfoMap = locator.getAllRegionLocations();
-        assertEquals(numRegions, regionInfoMap.size());
-        for (HRegionLocation entry : regionInfoMap) {
-          final HRegionInfo regionInfo = entry.getRegionInfo();
-          byte[] regionStart = regionInfo.getStartKey();
-          byte[] regionEnd = regionInfo.getEndKey();
-
-          // This region's start key should be one of the region boundaries
-          int startBoundaryIndex = indexOfBytes(expectedBounds, regionStart);
-          assertNotSame(-1, startBoundaryIndex);
-
-          // This region's end key should be the region boundary that comes
-          // after the starting boundary.
-          byte[] expectedRegionEnd = expectedBounds.get(startBoundaryIndex + 1);
-          assertEquals(0, Bytes.compareTo(regionEnd, expectedRegionEnd));
-        }
+  private void verifyBounds(List<byte[]> expectedBounds, TableName tableName)
+          throws Exception {
+    // Get region boundaries from the cluster and verify their endpoints
+    final int numRegions = expectedBounds.size()-1;
+    try (Table table = UTIL.getConnection().getTable(tableName);
+        RegionLocator locator = UTIL.getConnection().getRegionLocator(tableName)) {
+      final List<HRegionLocation> regionInfoMap = locator.getAllRegionLocations();
+      assertEquals(numRegions, regionInfoMap.size());
+      for (HRegionLocation entry : regionInfoMap) {
+        final HRegionInfo regionInfo = entry.getRegionInfo();
+        byte[] regionStart = regionInfo.getStartKey();
+        byte[] regionEnd = regionInfo.getEndKey();
+
+        // This region's start key should be one of the region boundaries
+        int startBoundaryIndex = indexOfBytes(expectedBounds, regionStart);
+        assertNotSame(-1, startBoundaryIndex);
+
+        // This region's end key should be the region boundary that comes
+        // after the starting boundary.
+        byte[] expectedRegionEnd = expectedBounds.get(startBoundaryIndex + 1);
+        assertEquals(0, Bytes.compareTo(regionEnd, expectedRegionEnd));
       }
     }
+  }
 
-    /**
-     * List.indexOf() doesn't really work for a List&lt;byte[]>, because byte[]
-     * doesn't override equals(). This method checks whether a list contains
-     * a given element by checking each element using the byte array
-     * comparator.
-     * @return the index of the first element that equals compareTo, or -1
-     * if no elements are equal.
-     */
-    static private int indexOfBytes(List<byte[]> list,  byte[] compareTo) {
-        int listIndex = 0;
-        for(byte[] elem: list) {
-            if(Bytes.BYTES_COMPARATOR.compare(elem, compareTo) == 0) {
-                return listIndex;
-            }
-            listIndex++;
-        }
-        return -1;
+  /**
+   * List.indexOf() doesn't really work for a List&lt;byte[]>, because byte[]
+   * doesn't override equals(). This method checks whether a list contains
+   * a given element by checking each element using the byte array comparator.
+   * @return the index of the first element that equals compareTo, or -1 if no elements are equal.
+   */
+  static private int indexOfBytes(List<byte[]> list,  byte[] compareTo) {
+    int listIndex = 0;
+    for(byte[] elem: list) {
+      if(Bytes.BYTES_COMPARATOR.compare(elem, compareTo) == 0) {
+        return listIndex;
+      }
+      listIndex++;
     }
+    return -1;
+  }
 
 }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
index 993cf7f..c9cca85 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
@@ -22,8 +22,10 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
-import java.io.FileWriter;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
 import javax.security.auth.login.AppConfigurationEntry;
@@ -65,19 +67,19 @@ public class TestZooKeeperACL {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     File saslConfFile = File.createTempFile("tmp", "jaas.conf");
-    FileWriter fwriter = new FileWriter(saslConfFile);
-
-    fwriter.write("" +
-      "Server {\n" +
-        "org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
-        "user_hbase=\"secret\";\n" +
-      "};\n" +
-      "Client {\n" +
-        "org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
-        "username=\"hbase\"\n" +
-        "password=\"secret\";\n" +
-      "};" + "\n");
-    fwriter.close();
+    try (OutputStreamWriter fwriter = new OutputStreamWriter(
+          new FileOutputStream(saslConfFile), StandardCharsets.UTF_8)) {
+      fwriter.write(
+        "Server {\n" +
+          "org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+          "user_hbase=\"secret\";\n" +
+        "};\n" +
+        "Client {\n" +
+          "org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+          "username=\"hbase\"\n" +
+          "password=\"secret\";\n" +
+        "};" + "\n");
+    }
     System.setProperty("java.security.auth.login.config",
         saslConfFile.getAbsolutePath());
     System.setProperty("zookeeper.authProvider.1",
@@ -279,10 +281,11 @@ public class TestZooKeeperACL {
     assertEquals(testJaasConfig, secureZKAvailable);
     // Define Jaas configuration without ZooKeeper Jaas config
     File saslConfFile = File.createTempFile("tmp", "fakeJaas.conf");
-    FileWriter fwriter = new FileWriter(saslConfFile);
+    try (OutputStreamWriter fwriter = new OutputStreamWriter(
+          new FileOutputStream(saslConfFile), StandardCharsets.UTF_8)) {
+      fwriter.write("");
+    }
 
-    fwriter.write("");
-    fwriter.close();
     System.setProperty("java.security.auth.login.config",
         saslConfFile.getAbsolutePath());
 
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
index 730d0dd..b158bb7 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
@@ -34,6 +34,7 @@ import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.server.NIOServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
@@ -54,6 +55,7 @@ public class MiniZooKeeperCluster {
 
   private static final int TICK_TIME = 2000;
   private static final int DEFAULT_CONNECTION_TIMEOUT = 30000;
+  private static final byte[] STATIC_BYTES = Bytes.toBytes("stat");
   private int connectionTimeout;
 
   private boolean started;
@@ -406,7 +408,7 @@ public class MiniZooKeeperCluster {
         Socket sock = new Socket("localhost", port);
         try {
           OutputStream outstream = sock.getOutputStream();
-          outstream.write("stat".getBytes());
+          outstream.write(STATIC_BYTES);
           outstream.flush();
         } finally {
           sock.close();
@@ -436,7 +438,7 @@ public class MiniZooKeeperCluster {
         BufferedReader reader = null;
         try {
           OutputStream outstream = sock.getOutputStream();
-          outstream.write("stat".getBytes());
+          outstream.write(STATIC_BYTES);
           outstream.flush();
 
           Reader isr = new InputStreamReader(sock.getInputStream());
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
index 1e7db05..47e0d11 100644
--- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
@@ -81,14 +81,14 @@ public class TestRecoverableZooKeeper {
     String ensemble = ZKConfig.getZKQuorumServersString(conf);
     RecoverableZooKeeper rzk = ZKUtil.connect(conf, ensemble, zkw);
     rzk.create(znode, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-    rzk.setData(znode, "OPENING".getBytes(), 0);
+    rzk.setData(znode, Bytes.toBytes("OPENING"), 0);
     Field zkField = RecoverableZooKeeper.class.getDeclaredField("zk");
     zkField.setAccessible(true);
     int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
     ZookeeperStub zkStub = new ZookeeperStub(ensemble, timeout, zkw);
     zkStub.setThrowExceptionInNumOperations(1);
     zkField.set(rzk, zkStub);
-    byte[] opened = "OPENED".getBytes();
+    byte[] opened = Bytes.toBytes("OPENED");
     rzk.setData(znode, opened, 1);
     byte[] data = rzk.getData(znode, false, new Stat());
     assertTrue(Bytes.equals(opened, data));
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
index 1508441..f103a18 100644
--- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
@@ -157,7 +157,7 @@ public class TestZKUtil {
     String quorumServers = ZKConfig.getZKQuorumServersString(c);
     int sessionTimeout = 5 * 1000; // 5 seconds
     ZooKeeper zk = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
-    zk.addAuthInfo("digest", "hbase:rox".getBytes());
+    zk.addAuthInfo("digest", Bytes.toBytes("hbase:rox"));
 
     // Save the previous ACL
     Stat s = null;
@@ -223,7 +223,7 @@ public class TestZKUtil {
 
     // Restore the ACL
     ZooKeeper zk3 = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
-    zk3.addAuthInfo("digest", "hbase:rox".getBytes());
+    zk3.addAuthInfo("digest", Bytes.toBytes("hbase:rox"));
     try {
       zk3.setACL("/", oldACL, -1);
     } finally {