You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ai...@apache.org on 2017/11/07 16:53:41 UTC

[2/2] hive git commit: HIVE-15016: Run tests with Hadoop 3.0.0-beta1

HIVE-15016: Run tests with Hadoop 3.0.0-beta1


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

Branch: refs/heads/master
Commit: a3e872822ece99eb14a5a12ff8bb191d9e014788
Parents: 7ea12e7
Author: Aihua Xu <ai...@apache.org>
Authored: Sat Oct 28 16:51:06 2017 -0700
Committer: Aihua Xu <ai...@apache.org>
Committed: Tue Nov 7 08:39:35 2017 -0800

----------------------------------------------------------------------
 common/pom.xml                                  |   6 +
 hbase-handler/pom.xml                           |  45 +++-
 .../apache/hadoop/hive/hbase/HBaseMetaHook.java | 224 +++++++++++++++++++
 .../hadoop/hive/hbase/HBaseRowSerializer.java   |   4 +-
 .../hadoop/hive/hbase/HBaseStorageHandler.java  | 209 ++---------------
 .../hive/hbase/HiveHBaseInputFormatUtil.java    |  11 -
 .../hive/hbase/HiveHBaseTableInputFormat.java   |   1 -
 .../hive/hbase/HiveHBaseTableOutputFormat.java  |  21 +-
 .../hive/hbase/HiveHFileOutputFormat.java       |  12 +-
 .../hadoop/hive/hbase/ResultWritable.java       |   5 +-
 .../hadoop/hive/hbase/TestHBaseSerDe.java       | 120 +++++-----
 .../hadoop/hive/hbase/TestLazyHBaseObject.java  |  79 +++----
 .../hive/hbase/TestPutResultWritable.java       |   7 +-
 .../src/test/queries/positive/hbase_bulk.q      |   2 +-
 .../test/queries/positive/hbase_handler_bulk.q  |   2 +-
 .../results/positive/hbase_handler_bulk.q.out   |   4 +-
 .../apache/hive/hcatalog/common/HCatUtil.java   |   2 +-
 .../rcfile/TestRCFileMapReduceInputFormat.java  |  14 +-
 .../hive/hcatalog/templeton/TestWebHCatE2e.java |   4 +-
 .../hcatalog/templeton/mock/MockUriInfo.java    |  11 +
 itests/hcatalog-unit/pom.xml                    |   6 +
 .../hive/hcatalog/hbase/ManyMiniCluster.java    |  27 ++-
 .../hive/hcatalog/hbase/SkeletonHBaseTest.java  |  24 +-
 .../hbase/TestPigHBaseStorageHandler.java       | 206 +++++++++++------
 itests/hive-minikdc/pom.xml                     |   8 +-
 itests/hive-unit-hadoop2/pom.xml                |   6 +
 itests/hive-unit/pom.xml                        |  31 ++-
 .../apache/hadoop/hive/ql/TestAcidOnTez.java    |   3 +
 .../hive/ql/txn/compactor/TestCompactor.java    |   6 +-
 .../jdbc/TestJdbcWithLocalClusterSpark.java     |   5 +
 ...stMultiSessionsHS2WithLocalClusterSpark.java |   5 +
 itests/qtest-accumulo/pom.xml                   |  37 ++-
 itests/qtest-spark/pom.xml                      |  21 +-
 itests/qtest/pom.xml                            |   6 +
 itests/util/pom.xml                             |  19 +-
 .../hadoop/hive/hbase/HBaseQTestUtil.java       |  37 +--
 .../hadoop/hive/hbase/HBaseTestSetup.java       |  46 ++--
 llap-server/pom.xml                             |  61 ++++-
 .../llap/shufflehandler/ShuffleHandler.java     |  26 +--
 metastore/pom.xml                               |   2 +-
 pom.xml                                         |  77 ++++++-
 .../apache/hadoop/hive/ql/io/TestRCFile.java    |  17 +-
 .../exim_00_unsupported_schema.q.out            |   2 +-
 .../test/results/clientnegative/external1.q.out |   2 +-
 .../test/results/clientnegative/external2.q.out |   2 +-
 serde/pom.xml                                   |   6 +
 shims/0.23/pom.xml                              |  12 +
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |  43 ++--
 .../org/apache/hadoop/fs/ProxyFileSystem.java   |  12 +
 shims/scheduler/pom.xml                         |   8 +-
 spark-client/pom.xml                            |   4 +
 standalone-metastore/pom.xml                    |  18 +-
 .../hadoop/hive/metastore/utils/HdfsUtils.java  |  10 +-
 .../ptest2/src/main/resources/batch-exec.vm     |   2 +-
 54 files changed, 1027 insertions(+), 553 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 868e14d..aaeecc0 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -220,6 +220,12 @@
       <groupId>io.dropwizard.metrics</groupId>
       <artifactId>metrics-json</artifactId>
       <version>${dropwizard.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-databind</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-handler/pom.xml b/hbase-handler/pom.xml
index 7f57b77..59f9dd8 100644
--- a/hbase-handler/pom.xml
+++ b/hbase-handler/pom.xml
@@ -81,8 +81,23 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
       <version>${hbase.version}</version>
-          <exclusions>
-             <exclusion>
+      <exclusions>
+         <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-mapreduce</artifactId>
+      <version>${hbase.version}</version>
+      <exclusions>
+          <exclusion>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-log4j12</artifactId>
           </exclusion>
@@ -92,6 +107,7 @@
           </exclusion>
       </exclusions>
     </dependency>
+
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
@@ -140,7 +156,24 @@
             <artifactId>commons-logging</artifactId>
           </exclusion>
       </exclusions>
-   </dependency>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-mapreduce</artifactId>
+      <version>${hbase.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <exclusions>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+      </exclusions>
+    </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
@@ -149,6 +182,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-runner</artifactId>
+      <version>${jetty.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>com.sun.jersey</groupId>
       <artifactId>jersey-servlet</artifactId>
       <version>${jersey.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseMetaHook.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseMetaHook.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseMetaHook.java
new file mode 100644
index 0000000..9fe07af
--- /dev/null
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseMetaHook.java
@@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.hbase;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.util.StringUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * MetaHook for HBase. Updates the table data in HBase too. Not thread safe, and cleanup should
+ * be used after usage.
+ */
+public class HBaseMetaHook implements HiveMetaHook, Closeable {
+  private Configuration hbaseConf;
+  private Admin admin;
+
+  public HBaseMetaHook(Configuration hbaseConf) {
+    this.hbaseConf = hbaseConf;
+  }
+
+  private Admin getHBaseAdmin() throws MetaException {
+    try {
+      if (admin == null) {
+        Connection conn = ConnectionFactory.createConnection(hbaseConf);
+        admin = conn.getAdmin();
+      }
+      return admin;
+    } catch (IOException ioe) {
+      throw new MetaException(StringUtils.stringifyException(ioe));
+    }
+  }
+
+  private String getHBaseTableName(Table tbl) {
+    // Give preference to TBLPROPERTIES over SERDEPROPERTIES
+    // (really we should only use TBLPROPERTIES, so this is just
+    // for backwards compatibility with the original specs).
+    String tableName = tbl.getParameters().get(HBaseSerDe.HBASE_TABLE_NAME);
+    if (tableName == null) {
+      //convert to lower case in case we are getting from serde
+      tableName = tbl.getSd().getSerdeInfo().getParameters().get(HBaseSerDe.HBASE_TABLE_NAME);
+      //standardize to lower case
+      if (tableName != null) {
+        tableName = tableName.toLowerCase();
+      }
+    }
+    if (tableName == null) {
+      tableName = (tbl.getDbName() + "." + tbl.getTableName()).toLowerCase();
+      if (tableName.startsWith(HBaseStorageHandler.DEFAULT_PREFIX)) {
+        tableName = tableName.substring(HBaseStorageHandler.DEFAULT_PREFIX.length());
+      }
+    }
+    return tableName;
+  }
+
+  @Override
+  public void preDropTable(Table table) throws MetaException {
+    // nothing to do
+  }
+
+  @Override
+  public void rollbackDropTable(Table table) throws MetaException {
+    // nothing to do
+  }
+
+  @Override
+  public void commitDropTable(Table tbl, boolean deleteData) throws MetaException {
+    try {
+      String tableName = getHBaseTableName(tbl);
+      boolean isExternal = MetaStoreUtils.isExternalTable(tbl);
+      if (deleteData && !isExternal) {
+        if (getHBaseAdmin().isTableEnabled(TableName.valueOf(tableName))) {
+          getHBaseAdmin().disableTable(TableName.valueOf(tableName));
+        }
+        getHBaseAdmin().deleteTable(TableName.valueOf(tableName));
+      }
+    } catch (IOException ie) {
+      throw new MetaException(StringUtils.stringifyException(ie));
+    }
+  }
+
+  @Override
+  public void preCreateTable(Table tbl) throws MetaException {
+    boolean isExternal = MetaStoreUtils.isExternalTable(tbl);
+
+    // We'd like to move this to HiveMetaStore for any non-native table, but
+    // first we need to support storing NULL for location on a table
+    if (tbl.getSd().getLocation() != null) {
+      throw new MetaException("LOCATION may not be specified for HBase.");
+    }
+
+    org.apache.hadoop.hbase.client.Table htable = null;
+
+    try {
+      String tableName = getHBaseTableName(tbl);
+      Map<String, String> serdeParam = tbl.getSd().getSerdeInfo().getParameters();
+      String hbaseColumnsMapping = serdeParam.get(HBaseSerDe.HBASE_COLUMNS_MAPPING);
+
+      ColumnMappings columnMappings = HBaseSerDe.parseColumnsMapping(hbaseColumnsMapping);
+
+      HTableDescriptor tableDesc;
+
+      if (!getHBaseAdmin().tableExists(TableName.valueOf(tableName))) {
+        // if it is not an external table then create one
+        if (!isExternal) {
+          // Create the column descriptors
+          tableDesc = new HTableDescriptor(TableName.valueOf(tableName));
+          Set<String> uniqueColumnFamilies = new HashSet<String>();
+
+          for (ColumnMappings.ColumnMapping colMap : columnMappings) {
+            if (!colMap.hbaseRowKey && !colMap.hbaseTimestamp) {
+              uniqueColumnFamilies.add(colMap.familyName);
+            }
+          }
+
+          for (String columnFamily : uniqueColumnFamilies) {
+            tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes(columnFamily)));
+          }
+
+          getHBaseAdmin().createTable(tableDesc);
+        } else {
+          // an external table
+          throw new MetaException("HBase table " + tableName +
+              " doesn't exist while the table is declared as an external table.");
+        }
+
+      } else {
+        if (!isExternal) {
+          throw new MetaException("Table " + tableName + " already exists within HBase; "
+              + "use CREATE EXTERNAL TABLE instead to register it in Hive.");
+        }
+        // make sure the schema mapping is right
+        tableDesc = getHBaseAdmin().getTableDescriptor(TableName.valueOf(tableName));
+
+        for (ColumnMappings.ColumnMapping colMap : columnMappings) {
+
+          if (colMap.hbaseRowKey || colMap.hbaseTimestamp) {
+            continue;
+          }
+
+          if (!tableDesc.hasFamily(colMap.familyNameBytes)) {
+            throw new MetaException("Column Family " + colMap.familyName
+                + " is not defined in hbase table " + tableName);
+          }
+        }
+      }
+
+      // ensure the table is online
+      htable = getHBaseAdmin().getConnection().getTable(tableDesc.getTableName());
+    } catch (Exception se) {
+      throw new MetaException(StringUtils.stringifyException(se));
+    } finally {
+      if (htable != null) {
+        IOUtils.closeQuietly(htable);
+      }
+    }
+  }
+
+  @Override
+  public void rollbackCreateTable(Table table) throws MetaException {
+    boolean isExternal = MetaStoreUtils.isExternalTable(table);
+    String tableName = getHBaseTableName(table);
+    try {
+      if (!isExternal && getHBaseAdmin().tableExists(TableName.valueOf(tableName))) {
+        // we have created an HBase table, so we delete it to roll back;
+        if (getHBaseAdmin().isTableEnabled(TableName.valueOf(tableName))) {
+          getHBaseAdmin().disableTable(TableName.valueOf(tableName));
+        }
+        getHBaseAdmin().deleteTable(TableName.valueOf(tableName));
+      }
+    } catch (IOException ie) {
+      throw new MetaException(StringUtils.stringifyException(ie));
+    }
+  }
+
+  @Override
+  public void commitCreateTable(Table table) throws MetaException {
+    // nothing to do
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (admin != null) {
+      Connection connection = admin.getConnection();
+      admin.close();
+      admin = null;
+      if (connection != null) {
+        connection.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java
index c6f3b0f..ce7071e 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java
@@ -174,7 +174,7 @@ public class HBaseRowSerializer {
           continue;
         }
 
-        put.add(colMap.familyNameBytes, columnQualifierBytes, bytes);
+        put.addColumn(colMap.familyNameBytes, columnQualifierBytes, bytes);
       }
     } else {
       byte[] bytes;
@@ -198,7 +198,7 @@ public class HBaseRowSerializer {
         return;
       }
 
-      put.add(colMap.familyNameBytes, colMap.qualifierNameBytes, bytes);
+      put.addColumn(colMap.familyNameBytes, colMap.qualifierNameBytes, bytes);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
index 9cad97a..0a3788f 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hive.hbase;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -28,28 +27,20 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.mapred.TableOutputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableInputFormatBase;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.token.TokenUtil;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
 import org.apache.hadoop.hive.metastore.HiveMetaHook;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer;
@@ -81,14 +72,14 @@ import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.yammer.metrics.core.MetricsRegistry;
+import com.codahale.metrics.MetricRegistry;
 
 /**
  * HBaseStorageHandler provides a HiveStorageHandler implementation for
  * HBase.
  */
 public class HBaseStorageHandler extends DefaultStorageHandler
-  implements HiveMetaHook, HiveStoragePredicateHandler {
+  implements HiveStoragePredicateHandler {
 
   private static final Logger LOG = LoggerFactory.getLogger(HBaseStorageHandler.class);
 
@@ -117,169 +108,6 @@ public class HBaseStorageHandler extends DefaultStorageHandler
 
   private Configuration jobConf;
   private Configuration hbaseConf;
-  private HBaseAdmin admin;
-
-  private HBaseAdmin getHBaseAdmin() throws MetaException {
-    try {
-      if (admin == null) {
-        admin = new HBaseAdmin(hbaseConf);
-      }
-      return admin;
-    } catch (IOException ioe) {
-      throw new MetaException(StringUtils.stringifyException(ioe));
-    }
-  }
-
-  private String getHBaseTableName(Table tbl) {
-    // Give preference to TBLPROPERTIES over SERDEPROPERTIES
-    // (really we should only use TBLPROPERTIES, so this is just
-    // for backwards compatibility with the original specs).
-    String tableName = tbl.getParameters().get(HBaseSerDe.HBASE_TABLE_NAME);
-    if (tableName == null) {
-      //convert to lower case in case we are getting from serde
-      tableName = tbl.getSd().getSerdeInfo().getParameters().get(
-        HBaseSerDe.HBASE_TABLE_NAME);
-      //standardize to lower case
-      if (tableName != null) {
-        tableName = tableName.toLowerCase();
-      }
-    }
-    if (tableName == null) {
-      tableName = (tbl.getDbName() + "." + tbl.getTableName()).toLowerCase();
-      if (tableName.startsWith(DEFAULT_PREFIX)) {
-        tableName = tableName.substring(DEFAULT_PREFIX.length());
-      }
-    }
-    return tableName;
-  }
-
-  @Override
-  public void preDropTable(Table table) throws MetaException {
-    // nothing to do
-  }
-
-  @Override
-  public void rollbackDropTable(Table table) throws MetaException {
-    // nothing to do
-  }
-
-  @Override
-  public void commitDropTable(
-    Table tbl, boolean deleteData) throws MetaException {
-
-    try {
-      String tableName = getHBaseTableName(tbl);
-      boolean isExternal = MetaStoreUtils.isExternalTable(tbl);
-      if (deleteData && !isExternal) {
-        if (getHBaseAdmin().isTableEnabled(tableName)) {
-          getHBaseAdmin().disableTable(tableName);
-        }
-        getHBaseAdmin().deleteTable(tableName);
-      }
-    } catch (IOException ie) {
-      throw new MetaException(StringUtils.stringifyException(ie));
-    }
-  }
-
-  @Override
-  public void preCreateTable(Table tbl) throws MetaException {
-    boolean isExternal = MetaStoreUtils.isExternalTable(tbl);
-
-    // We'd like to move this to HiveMetaStore for any non-native table, but
-    // first we need to support storing NULL for location on a table
-    if (tbl.getSd().getLocation() != null) {
-      throw new MetaException("LOCATION may not be specified for HBase.");
-    }
-
-    HTable htable = null;
-
-    try {
-      String tableName = getHBaseTableName(tbl);
-      Map<String, String> serdeParam = tbl.getSd().getSerdeInfo().getParameters();
-      String hbaseColumnsMapping = serdeParam.get(HBaseSerDe.HBASE_COLUMNS_MAPPING);
-
-      ColumnMappings columnMappings = HBaseSerDe.parseColumnsMapping(hbaseColumnsMapping);
-
-      HTableDescriptor tableDesc;
-
-      if (!getHBaseAdmin().tableExists(tableName)) {
-        // if it is not an external table then create one
-        if (!isExternal) {
-          // Create the column descriptors
-          tableDesc = new HTableDescriptor(tableName);
-          Set<String> uniqueColumnFamilies = new HashSet<String>();
-
-          for (ColumnMapping colMap : columnMappings) {
-            if (!colMap.hbaseRowKey && !colMap.hbaseTimestamp) {
-              uniqueColumnFamilies.add(colMap.familyName);
-            }
-          }
-
-          for (String columnFamily : uniqueColumnFamilies) {
-            tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes(columnFamily)));
-          }
-
-          getHBaseAdmin().createTable(tableDesc);
-        } else {
-          // an external table
-          throw new MetaException("HBase table " + tableName +
-              " doesn't exist while the table is declared as an external table.");
-        }
-
-      } else {
-        if (!isExternal) {
-          throw new MetaException("Table " + tableName + " already exists"
-            + " within HBase; use CREATE EXTERNAL TABLE instead to"
-            + " register it in Hive.");
-        }
-        // make sure the schema mapping is right
-        tableDesc = getHBaseAdmin().getTableDescriptor(Bytes.toBytes(tableName));
-
-        for (ColumnMapping colMap : columnMappings) {
-
-          if (colMap.hbaseRowKey || colMap.hbaseTimestamp) {
-            continue;
-          }
-
-          if (!tableDesc.hasFamily(colMap.familyNameBytes)) {
-            throw new MetaException("Column Family " + colMap.familyName
-                + " is not defined in hbase table " + tableName);
-          }
-        }
-      }
-
-      // ensure the table is online
-      htable = new HTable(hbaseConf, tableDesc.getName());
-    } catch (Exception se) {
-      throw new MetaException(StringUtils.stringifyException(se));
-    } finally {
-      if (htable != null) {
-        IOUtils.closeQuietly(htable);
-      }
-    }
-  }
-
-  @Override
-  public void rollbackCreateTable(Table table) throws MetaException {
-    boolean isExternal = MetaStoreUtils.isExternalTable(table);
-    String tableName = getHBaseTableName(table);
-    try {
-      if (!isExternal && getHBaseAdmin().tableExists(tableName)) {
-        // we have created an HBase table, so we delete it to roll back;
-        if (getHBaseAdmin().isTableEnabled(tableName)) {
-          getHBaseAdmin().disableTable(tableName);
-        }
-        getHBaseAdmin().deleteTable(tableName);
-      }
-    } catch (IOException ie) {
-      throw new MetaException(StringUtils.stringifyException(ie));
-    }
-  }
-
-  @Override
-  public void commitCreateTable(Table table) throws MetaException {
-    // nothing to do
-  }
 
   @Override
   public Configuration getConf() {
@@ -321,7 +149,7 @@ public class HBaseStorageHandler extends DefaultStorageHandler
 
   @Override
   public HiveMetaHook getMetaHook() {
-    return this;
+    return new HBaseMetaHook(hbaseConf);
   }
 
   @Override
@@ -371,12 +199,10 @@ public class HBaseStorageHandler extends DefaultStorageHandler
       jobProperties.put(HBaseSerDe.HBASE_SCAN_BATCH, scanBatch);
     }
 
-    String tableName =
-      tableProperties.getProperty(HBaseSerDe.HBASE_TABLE_NAME);
+    String tableName = tableProperties.getProperty(HBaseSerDe.HBASE_TABLE_NAME);
     if (tableName == null) {
-      tableName =
-        tableProperties.getProperty(hive_metastoreConstants.META_TABLE_NAME);
-        tableName = tableName.toLowerCase();
+      tableName = tableProperties.getProperty(hive_metastoreConstants.META_TABLE_NAME);
+      tableName = tableName.toLowerCase();
       if (tableName.startsWith(DEFAULT_PREFIX)) {
         tableName = tableName.substring(DEFAULT_PREFIX.length());
       }
@@ -432,8 +258,7 @@ public class HBaseStorageHandler extends DefaultStorageHandler
       }
       try {
         addHBaseDelegationToken(jobConf);
-      }//try
-      catch (IOException e) {
+      } catch (IOException | MetaException e) {
         throw new IllegalStateException("Error while configuring input job properties", e);
       } //input job properties
     }
@@ -480,18 +305,19 @@ public class HBaseStorageHandler extends DefaultStorageHandler
     }
   }
 
-  private void addHBaseDelegationToken(Configuration conf) throws IOException {
+  private void addHBaseDelegationToken(Configuration conf) throws IOException, MetaException {
     if (User.isHBaseSecurityEnabled(conf)) {
-      HConnection conn = HConnectionManager.createConnection(conf);
+      Connection connection = ConnectionFactory.createConnection(hbaseConf);
       try {
         User curUser = User.getCurrent();
         Job job = new Job(conf);
-        TokenUtil.addTokenForJob(conn, curUser, job);
+        TokenUtil.addTokenForJob(connection, curUser, job);
       } catch (InterruptedException e) {
         throw new IOException("Error while obtaining hbase delegation token", e);
-      }
-      finally {
-        conn.close();
+      } finally {
+        if (connection != null) {
+          connection.close();
+        }
       }
     }
   }
@@ -523,8 +349,9 @@ public class HBaseStorageHandler extends DefaultStorageHandler
       }
       if (HiveConf.getVar(jobConf, HiveConf.ConfVars.HIVE_HBASE_SNAPSHOT_NAME) != null) {
         // There is an extra dependency on MetricsRegistry for snapshot IF.
-        TableMapReduceUtil.addDependencyJars(jobConf, MetricsRegistry.class);
+        TableMapReduceUtil.addDependencyJars(jobConf, MetricRegistry.class);
       }
+
       Set<String> merged = new LinkedHashSet<String>(jobConf.getStringCollection("tmpjars"));
 
       Job copy = new Job(jobConf);

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java
index 6054d53..d42b7ec 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java
@@ -24,13 +24,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
 import org.apache.hadoop.hive.ql.index.IndexSearchCondition;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
@@ -43,14 +40,6 @@ import org.apache.hadoop.mapred.JobConf;
 class HiveHBaseInputFormatUtil {
 
   /**
-   * Parse {@code jobConf} to create the target {@link HTable} instance.
-   */
-  public static HTable getTable(JobConf jobConf) throws IOException {
-    String hbaseTableName = jobConf.get(HBaseSerDe.HBASE_TABLE_NAME);
-    return new HTable(HBaseConfiguration.create(jobConf), Bytes.toBytes(hbaseTableName));
-  }
-
-  /**
    * Parse {@code jobConf} to create a {@link Scan} instance.
    */
   public static Scan getScan(JobConf jobConf) throws IOException {

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java
index 1ef4545..8b89817 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java
index 4b8f62c..7c78d7b 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java
@@ -20,19 +20,21 @@ package org.apache.hadoop.hive.hbase;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapred.TableMapReduceUtil;
 import org.apache.hadoop.hbase.mapreduce.TableOutputCommitter;
 import org.apache.hadoop.hbase.mapreduce.TableOutputFormat;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.hbase.PutWritable;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputFormat;
@@ -102,9 +104,9 @@ public class HiveHBaseTableOutputFormat extends
     jobConf.set(TableOutputFormat.OUTPUT_TABLE, hbaseTableName);
     final boolean walEnabled = HiveConf.getBoolVar(
         jobConf, HiveConf.ConfVars.HIVE_HBASE_WAL_ENABLED);
-    final HTable table = new HTable(HBaseConfiguration.create(jobConf), hbaseTableName);
-    table.setAutoFlush(false);
-    return new MyRecordWriter(table,walEnabled);
+    final Connection conn = ConnectionFactory.createConnection(HBaseConfiguration.create(jobConf));
+    final BufferedMutator table = conn.getBufferedMutator(TableName.valueOf(hbaseTableName));
+    return new MyRecordWriter(table, conn, walEnabled);
   }
 
   @Override
@@ -115,12 +117,14 @@ public class HiveHBaseTableOutputFormat extends
 
 
   static private class MyRecordWriter implements org.apache.hadoop.mapred.RecordWriter<ImmutableBytesWritable, Object> {
-    private final HTable m_table;
+    private final BufferedMutator m_table;
     private final boolean m_walEnabled;
+    private final Connection m_connection;
 
-    public MyRecordWriter(HTable table, boolean walEnabled) {
+    public MyRecordWriter(BufferedMutator table, Connection connection, boolean walEnabled) {
       m_table = table;
       m_walEnabled = walEnabled;
+      m_connection = connection;
     }
 
     public void close(Reporter reporter)
@@ -143,13 +147,14 @@ public class HiveHBaseTableOutputFormat extends
       } else {
         put.setDurability(Durability.SKIP_WAL);
       }
-      m_table.put(put);
+      m_table.mutate(put);
     }
 
     @Override
     protected void finalize() throws Throwable {
       try {
         m_table.close();
+        m_connection.close();
       } finally {
         super.finalize();
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java
index a25a96f..223dbe1 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
@@ -60,15 +60,15 @@ import org.apache.hadoop.util.Progressable;
  * for loading a table with a single column family.
  */
 public class HiveHFileOutputFormat extends
-    HFileOutputFormat implements
-    HiveOutputFormat<ImmutableBytesWritable, KeyValue> {
+    HFileOutputFormat2 implements
+    HiveOutputFormat<ImmutableBytesWritable, Cell> {
 
   public static final String HFILE_FAMILY_PATH = "hfile.family.path";
 
   static final Logger LOG = LoggerFactory.getLogger(HiveHFileOutputFormat.class.getName());
 
   private
-  org.apache.hadoop.mapreduce.RecordWriter<ImmutableBytesWritable, KeyValue>
+  org.apache.hadoop.mapreduce.RecordWriter<ImmutableBytesWritable, Cell>
   getFileWriter(org.apache.hadoop.mapreduce.TaskAttemptContext tac)
   throws IOException {
     try {
@@ -118,7 +118,7 @@ public class HiveHFileOutputFormat extends
     final Path outputdir = FileOutputFormat.getOutputPath(tac);
     final Path taskAttemptOutputdir = new FileOutputCommitter(outputdir, tac).getWorkPath();
     final org.apache.hadoop.mapreduce.RecordWriter<
-      ImmutableBytesWritable, KeyValue> fileWriter = getFileWriter(tac);
+      ImmutableBytesWritable, Cell> fileWriter = getFileWriter(tac);
 
     // Individual columns are going to be pivoted to HBase cells,
     // and for each row, they need to be written out in order
@@ -262,7 +262,7 @@ public class HiveHFileOutputFormat extends
   }
 
   @Override
-  public org.apache.hadoop.mapred.RecordWriter<ImmutableBytesWritable, KeyValue> getRecordWriter(
+  public org.apache.hadoop.mapred.RecordWriter<ImmutableBytesWritable, Cell> getRecordWriter(
       FileSystem ignored, JobConf job, String name, Progressable progress) throws IOException {
     throw new NotImplementedException("This will not be invoked");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java
index b35aea9..93c2f96 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -65,9 +66,9 @@ public class ResultWritable implements Writable {
   throws IOException {
     ProtobufUtil.toResultNoData(result).writeDelimitedTo(DataOutputOutputStream.from(out));
     out.writeInt(result.size());
-    for(KeyValue kv : result.list()) {
+    for(Cell cell : result.listCells()) {
+      KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
       KeyValue.write(kv, out);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
index f244ed6..14557d3 100644
--- a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
+++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
@@ -42,6 +42,8 @@ import org.apache.avro.specific.SpecificDatumWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -179,7 +181,7 @@ public class TestHBaseSerDe extends TestCase {
     byte [] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     kvs.add(new KeyValue(rowKey, cfa, qualByte, Bytes.toBytes("123")));
     kvs.add(new KeyValue(rowKey, cfb, qualShort, Bytes.toBytes("456")));
@@ -191,18 +193,18 @@ public class TestHBaseSerDe extends TestCase {
     kvs.add(new KeyValue(rowKey, cfb, qualBool, Bytes.toBytes("true")));
     Collections.sort(kvs, KeyValue.COMPARATOR);
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
-    p.add(cfa, qualByte, Bytes.toBytes("123"));
-    p.add(cfb, qualShort, Bytes.toBytes("456"));
-    p.add(cfc, qualInt, Bytes.toBytes("789"));
-    p.add(cfa, qualLong, Bytes.toBytes("1000"));
-    p.add(cfb, qualFloat, Bytes.toBytes("-0.01"));
-    p.add(cfc, qualDouble, Bytes.toBytes("5.3"));
-    p.add(cfa, qualString, Bytes.toBytes("Hadoop, HBase, and Hive"));
-    p.add(cfb, qualBool, Bytes.toBytes("true"));
+    p.addColumn(cfa, qualByte, Bytes.toBytes("123"));
+    p.addColumn(cfb, qualShort, Bytes.toBytes("456"));
+    p.addColumn(cfc, qualInt, Bytes.toBytes("789"));
+    p.addColumn(cfa, qualLong, Bytes.toBytes("1000"));
+    p.addColumn(cfb, qualFloat, Bytes.toBytes("-0.01"));
+    p.addColumn(cfc, qualDouble, Bytes.toBytes("5.3"));
+    p.addColumn(cfa, qualString, Bytes.toBytes("Hadoop, HBase, and Hive"));
+    p.addColumn(cfb, qualBool, Bytes.toBytes("true"));
 
     Object[] expectedFieldsData = {
       new Text("test-row1"),
@@ -273,7 +275,7 @@ public class TestHBaseSerDe extends TestCase {
     byte [] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     kvs.add(new KeyValue(rowKey, cfa, qualByte, Bytes.toBytes("123")));
     kvs.add(new KeyValue(rowKey, cfb, qualShort, Bytes.toBytes("456")));
@@ -285,18 +287,18 @@ public class TestHBaseSerDe extends TestCase {
     kvs.add(new KeyValue(rowKey, cfb, qualBool, Bytes.toBytes("true")));
     Collections.sort(kvs, KeyValue.COMPARATOR);
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey,putTimestamp);
 
-    p.add(cfa, qualByte, Bytes.toBytes("123"));
-    p.add(cfb, qualShort, Bytes.toBytes("456"));
-    p.add(cfc, qualInt, Bytes.toBytes("789"));
-    p.add(cfa, qualLong, Bytes.toBytes("1000"));
-    p.add(cfb, qualFloat, Bytes.toBytes("-0.01"));
-    p.add(cfc, qualDouble, Bytes.toBytes("5.3"));
-    p.add(cfa, qualString, Bytes.toBytes("Hadoop, HBase, and Hive"));
-    p.add(cfb, qualBool, Bytes.toBytes("true"));
+    p.addColumn(cfa, qualByte, Bytes.toBytes("123"));
+    p.addColumn(cfb, qualShort, Bytes.toBytes("456"));
+    p.addColumn(cfc, qualInt, Bytes.toBytes("789"));
+    p.addColumn(cfa, qualLong, Bytes.toBytes("1000"));
+    p.addColumn(cfb, qualFloat, Bytes.toBytes("-0.01"));
+    p.addColumn(cfc, qualDouble, Bytes.toBytes("5.3"));
+    p.addColumn(cfa, qualString, Bytes.toBytes("Hadoop, HBase, and Hive"));
+    p.addColumn(cfb, qualBool, Bytes.toBytes("true"));
 
     Object[] expectedFieldsData = {
       new Text("test-row1"),
@@ -419,7 +421,7 @@ public class TestHBaseSerDe extends TestCase {
     byte [] rowKey = Bytes.toBytes("test-row-2");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     kvs.add(new KeyValue(rowKey, cfa, qualByte, new byte [] { Byte.MIN_VALUE }));
     kvs.add(new KeyValue(rowKey, cfb, qualShort, Bytes.toBytes(Short.MIN_VALUE)));
@@ -431,19 +433,21 @@ public class TestHBaseSerDe extends TestCase {
       "Hadoop, HBase, and Hive Again!")));
     kvs.add(new KeyValue(rowKey, cfb, qualBool, Bytes.toBytes(false)));
 
+//    When using only HBase2, then we could change to this
+//    Collections.sort(kvs, CellComparator.COMPARATOR);
     Collections.sort(kvs, KeyValue.COMPARATOR);
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
-    p.add(cfa, qualByte, new byte [] { Byte.MIN_VALUE });
-    p.add(cfb, qualShort, Bytes.toBytes(Short.MIN_VALUE));
-    p.add(cfc, qualInt, Bytes.toBytes(Integer.MIN_VALUE));
-    p.add(cfa, qualLong, Bytes.toBytes(Long.MIN_VALUE));
-    p.add(cfb, qualFloat, Bytes.toBytes(Float.MIN_VALUE));
-    p.add(cfc, qualDouble, Bytes.toBytes(Double.MAX_VALUE));
-    p.add(cfa, qualString, Bytes.toBytes("Hadoop, HBase, and Hive Again!"));
-    p.add(cfb, qualBool, Bytes.toBytes(false));
+    p.addColumn(cfa, qualByte, new byte [] { Byte.MIN_VALUE });
+    p.addColumn(cfb, qualShort, Bytes.toBytes(Short.MIN_VALUE));
+    p.addColumn(cfc, qualInt, Bytes.toBytes(Integer.MIN_VALUE));
+    p.addColumn(cfa, qualLong, Bytes.toBytes(Long.MIN_VALUE));
+    p.addColumn(cfb, qualFloat, Bytes.toBytes(Float.MIN_VALUE));
+    p.addColumn(cfc, qualDouble, Bytes.toBytes(Double.MAX_VALUE));
+    p.addColumn(cfa, qualString, Bytes.toBytes("Hadoop, HBase, and Hive Again!"));
+    p.addColumn(cfb, qualBool, Bytes.toBytes(false));
 
     Object[] expectedFieldsData = {
       new Text("test-row-2"),
@@ -557,7 +561,7 @@ public class TestHBaseSerDe extends TestCase {
          Bytes.toBytes(true)}
     };
 
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
     Result [] r = new Result [] {null, null, null};
     Put [] p = new Put [] {null, null, null};
 
@@ -568,11 +572,11 @@ public class TestHBaseSerDe extends TestCase {
       for (int j = 0; j < columnQualifiersAndValues[i].length; j++) {
         kvs.add(new KeyValue(rowKeys[i], columnFamilies[j], columnQualifiersAndValues[i][j],
             columnQualifiersAndValues[i][j]));
-        p[i].add(columnFamilies[j], columnQualifiersAndValues[i][j],
+        p[i].addColumn(columnFamilies[j], columnQualifiersAndValues[i][j],
             columnQualifiersAndValues[i][j]);
       }
 
-      r[i] = new Result(kvs);
+      r[i] = Result.create(kvs);
     }
 
     Object [][] expectedData = {
@@ -701,15 +705,15 @@ public class TestHBaseSerDe extends TestCase {
     };
 
     Put p = new Put(rowKey);
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     for (int j = 0; j < columnQualifiersAndValues.length; j++) {
       kvs.add(new KeyValue(rowKey,
           columnFamilies[j], columnQualifiersAndValues[j], columnQualifiersAndValues[j]));
-      p.add(columnFamilies[j], columnQualifiersAndValues[j], columnQualifiersAndValues[j]);
+      p.addColumn(columnFamilies[j], columnQualifiersAndValues[j], columnQualifiersAndValues[j]);
     }
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Object [] expectedData = {
         new Text("row-key"), new ByteWritable((byte) 123), new ShortWritable((short) 456),
@@ -821,7 +825,7 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] dataA = "This is first test data".getBytes();
     byte[] dataB = "This is second test data".getBytes();
@@ -833,7 +837,7 @@ public class TestHBaseSerDe extends TestCase {
     kvs.add(new KeyValue(rowKey, cfa, qualC, dataC));
     kvs.add(new KeyValue(rowKey, cfa, qualD, dataD));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -928,13 +932,13 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = testStruct.getBytes();
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] testData = "This is a test data".getBytes();
 
     kvs.add(new KeyValue(rowKey, cfa, qualStruct, testData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -976,13 +980,13 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = testStruct.getBytes();
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] testData = "This is a test data".getBytes();
 
     kvs.add(new KeyValue(rowKey, cfa, qualStruct, testData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     byte[] putRowKey = testStruct.getBytesWithDelimiters();
 
@@ -1047,13 +1051,13 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] avroData = getTestAvroBytesFromSchema(RECORD_SCHEMA);
 
     kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -1092,13 +1096,13 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] avroData = getTestAvroBytesFromSchema(RECORD_SCHEMA);
 
     kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -1138,13 +1142,13 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] avroData = getTestAvroBytesFromSchema(RECORD_SCHEMA_EVOLVED);
 
     kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -1183,13 +1187,13 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] avroData = getTestAvroBytesFromClass1(1);
 
     kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -1234,13 +1238,13 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] avroData = getTestAvroBytesFromSchema(RECORD_SCHEMA);
 
     kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -1298,13 +1302,13 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] avroData = getTestAvroBytesFromClass2(1);
 
     kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -1362,7 +1366,7 @@ public class TestHBaseSerDe extends TestCase {
     byte[] rowKey = Bytes.toBytes("test-row1");
 
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] avroDataA = getTestAvroBytesFromSchema(RECORD_SCHEMA);
     byte[] avroDataB = getTestAvroBytesFromClass1(1);
@@ -1372,7 +1376,7 @@ public class TestHBaseSerDe extends TestCase {
     kvs.add(new KeyValue(rowKey, cfa, qualAvroB, avroDataB));
     kvs.add(new KeyValue(rowKey, cfa, qualAvroC, avroDataC));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     Put p = new Put(rowKey);
 
@@ -1426,12 +1430,12 @@ public class TestHBaseSerDe extends TestCase {
     TestStruct testStruct = new TestStruct("A", "B", "C", false, (byte) 0);
     byte[] key = testStruct.getBytes();
     // Data
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     byte[] testData = testStruct.getBytes();
     kvs.add(new KeyValue(key, cfa, qualStruct, testData));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
     byte[] putKey = testStruct.getBytesWithDelimiters();
 
     Put p = new Put(putKey);

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java
index b2bdd19..216d7ae 100644
--- a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java
+++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java
@@ -25,6 +25,7 @@ import java.util.List;
 
 import junit.framework.TestCase;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -69,7 +70,7 @@ public class TestLazyHBaseObject extends TestCase {
     LazyHBaseCellMap b = new LazyHBaseCellMap((LazyMapObjectInspector) oi);
 
     // Initialize a result
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     kvs.add(new KeyValue(Bytes.toBytes("test-row"), Bytes.toBytes("cfa"),
         Bytes.toBytes("col1"), Bytes.toBytes("cfacol1")));
@@ -86,7 +87,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"), Bytes.toBytes("cfc"),
         Bytes.toBytes("col3"), Bytes.toBytes("cfccol3")));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
 
     List<Boolean> mapBinaryStorage = new ArrayList<Boolean>();
     mapBinaryStorage.add(false);
@@ -131,7 +132,7 @@ public class TestLazyHBaseObject extends TestCase {
     LazyHBaseCellMap b = new LazyHBaseCellMap((LazyMapObjectInspector) oi);
 
     // Initialize a result
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfa"), Bytes.toBytes("col1"), Bytes.toBytes("cfacol1")));
@@ -148,7 +149,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfc"), Bytes.toBytes("col3"), Bytes.toBytes("cfccol3")));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
     List<Boolean> mapBinaryStorage = new ArrayList<Boolean>();
     mapBinaryStorage.add(false);
     mapBinaryStorage.add(false);
@@ -192,11 +193,11 @@ public class TestLazyHBaseObject extends TestCase {
         mapBinaryIntKeyValue, new byte [] {(byte)1, (byte) 2}, 0, nullSequence, false, (byte) 0);
     LazyHBaseCellMap hbaseCellMap = new LazyHBaseCellMap((LazyMapObjectInspector) oi);
 
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
     byte [] rowKey = "row-key".getBytes();
     byte [] cfInt = "cf-int".getBytes();
     kvs.add(new KeyValue(rowKey, cfInt, Bytes.toBytes(1), Bytes.toBytes(1)));
-    Result result = new Result(kvs);
+    Result result = Result.create(kvs);
     List<Boolean> mapBinaryStorage = new ArrayList<Boolean>();
     mapBinaryStorage.add(true);
     mapBinaryStorage.add(true);
@@ -210,7 +211,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(
         rowKey, cfInt, Bytes.toBytes(Integer.MIN_VALUE), Bytes.toBytes(Integer.MIN_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfInt, mapBinaryStorage);
     expectedIntValue = new IntWritable(Integer.MIN_VALUE);
     lazyPrimitive =
@@ -221,7 +222,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(
         rowKey, cfInt, Bytes.toBytes(Integer.MAX_VALUE), Bytes.toBytes(Integer.MAX_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfInt, mapBinaryStorage);
     expectedIntValue = new IntWritable(Integer.MAX_VALUE);
     lazyPrimitive =
@@ -237,7 +238,7 @@ public class TestLazyHBaseObject extends TestCase {
     byte [] cfByte = "cf-byte".getBytes();
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfByte, new byte [] {(byte) 1}, new byte [] {(byte) 1}));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfByte, mapBinaryStorage);
     ByteWritable expectedByteValue = new ByteWritable((byte) 1);
     lazyPrimitive =
@@ -248,7 +249,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfByte, new byte [] {Byte.MIN_VALUE},
       new byte [] {Byte.MIN_VALUE}));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfByte, mapBinaryStorage);
     expectedByteValue = new ByteWritable(Byte.MIN_VALUE);
     lazyPrimitive =
@@ -259,7 +260,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfByte, new byte [] {Byte.MAX_VALUE},
       new byte [] {Byte.MAX_VALUE}));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfByte, mapBinaryStorage);
     expectedByteValue = new ByteWritable(Byte.MAX_VALUE);
     lazyPrimitive =
@@ -275,7 +276,7 @@ public class TestLazyHBaseObject extends TestCase {
     byte [] cfShort = "cf-short".getBytes();
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfShort, Bytes.toBytes((short) 1), Bytes.toBytes((short) 1)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfShort, mapBinaryStorage);
     ShortWritable expectedShortValue = new ShortWritable((short) 1);
     lazyPrimitive =
@@ -286,7 +287,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfShort, Bytes.toBytes(Short.MIN_VALUE),
       Bytes.toBytes(Short.MIN_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfShort, mapBinaryStorage);
     expectedShortValue = new ShortWritable(Short.MIN_VALUE);
     lazyPrimitive =
@@ -297,7 +298,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfShort, Bytes.toBytes(Short.MAX_VALUE),
       Bytes.toBytes(Short.MAX_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfShort, mapBinaryStorage);
     expectedShortValue = new ShortWritable(Short.MAX_VALUE);
     lazyPrimitive =
@@ -313,7 +314,7 @@ public class TestLazyHBaseObject extends TestCase {
     byte [] cfLong = "cf-long".getBytes();
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfLong, Bytes.toBytes((long) 1), Bytes.toBytes((long) 1)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfLong, mapBinaryStorage);
     LongWritable expectedLongValue = new LongWritable(1);
     lazyPrimitive =
@@ -324,7 +325,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfLong, Bytes.toBytes(Long.MIN_VALUE),
       Bytes.toBytes(Long.MIN_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfLong, mapBinaryStorage);
     expectedLongValue = new LongWritable(Long.MIN_VALUE);
     lazyPrimitive =
@@ -335,7 +336,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfLong, Bytes.toBytes(Long.MAX_VALUE),
       Bytes.toBytes(Long.MAX_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfLong, mapBinaryStorage);
     expectedLongValue = new LongWritable(Long.MAX_VALUE);
     lazyPrimitive =
@@ -353,7 +354,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfFloat, Bytes.toBytes((float) 1.0F),
       Bytes.toBytes((float) 1.0F)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfFloat, mapBinaryStorage);
     FloatWritable expectedFloatValue = new FloatWritable(1.0F);
     lazyPrimitive =
@@ -364,7 +365,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfFloat, Bytes.toBytes((float) Float.MIN_VALUE),
       Bytes.toBytes((float) Float.MIN_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfFloat, mapBinaryStorage);
     expectedFloatValue = new FloatWritable(Float.MIN_VALUE);
     lazyPrimitive =
@@ -375,7 +376,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfFloat, Bytes.toBytes((float) Float.MAX_VALUE),
       Bytes.toBytes((float) Float.MAX_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfFloat, mapBinaryStorage);
     expectedFloatValue = new FloatWritable(Float.MAX_VALUE);
     lazyPrimitive =
@@ -392,7 +393,7 @@ public class TestLazyHBaseObject extends TestCase {
     byte [] cfDouble = "cf-double".getBytes();
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfDouble, Bytes.toBytes(1.0), Bytes.toBytes(1.0)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfDouble, mapBinaryStorage);
     DoubleWritable expectedDoubleValue = new DoubleWritable(1.0);
     lazyPrimitive =
@@ -403,7 +404,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfDouble, Bytes.toBytes(Double.MIN_VALUE),
       Bytes.toBytes(Double.MIN_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfDouble, mapBinaryStorage);
     expectedDoubleValue = new DoubleWritable(Double.MIN_VALUE);
     lazyPrimitive =
@@ -414,7 +415,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfDouble, Bytes.toBytes(Double.MAX_VALUE),
       Bytes.toBytes(Double.MAX_VALUE)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfDouble, mapBinaryStorage);
     expectedDoubleValue = new DoubleWritable(Double.MAX_VALUE);
     lazyPrimitive =
@@ -431,7 +432,7 @@ public class TestLazyHBaseObject extends TestCase {
     byte [] cfBoolean = "cf-boolean".getBytes();
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfBoolean, Bytes.toBytes(false), Bytes.toBytes(false)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfBoolean, mapBinaryStorage);
     BooleanWritable expectedBooleanValue = new BooleanWritable(false);
     lazyPrimitive =
@@ -441,7 +442,7 @@ public class TestLazyHBaseObject extends TestCase {
 
     kvs.clear();
     kvs.add(new KeyValue(rowKey, cfBoolean, Bytes.toBytes(true), Bytes.toBytes(true)));
-    result = new Result(kvs);
+    result = Result.create(kvs);
     hbaseCellMap.init(result, cfBoolean, mapBinaryStorage);
     expectedBooleanValue = new BooleanWritable(true);
     lazyPrimitive =
@@ -485,7 +486,7 @@ public class TestLazyHBaseObject extends TestCase {
       nullSequence, false, false, (byte)0);
     LazyHBaseRow o = new LazyHBaseRow((LazySimpleStructObjectInspector) oi, columnMappings);
 
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
 
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfa"), Bytes.toBytes("a"), Bytes.toBytes("123")));
@@ -496,7 +497,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfb"), Bytes.toBytes("d"), Bytes.toBytes("hi")));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -510,7 +511,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfb"), Bytes.toBytes("c"), Bytes.toBytes("d=e:f=g")));
 
-    r = new Result(kvs);
+    r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -526,7 +527,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfb"), Bytes.toBytes("d"), Bytes.toBytes("no")));
 
-    r = new Result(kvs);
+    r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -540,7 +541,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfb"), Bytes.toBytes("d"), Bytes.toBytes("no")));
 
-    r = new Result(kvs);
+    r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -564,7 +565,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfb"), Bytes.toBytes("d"), Bytes.toBytes("")));
 
-    r = new Result(kvs);
+    r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -611,7 +612,7 @@ public class TestLazyHBaseObject extends TestCase {
       nullSequence, false, false, (byte) 0);
     LazyHBaseRow o = new LazyHBaseRow((LazySimpleStructObjectInspector) oi, columnMappings);
 
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfa"), Bytes.toBytes("a"), Bytes.toBytes("123")));
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
@@ -623,7 +624,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfc"), Bytes.toBytes("d"), Bytes.toBytes("hi")));
 
-    Result r = new Result(kvs);
+    Result r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -639,7 +640,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfb"), Bytes.toBytes("f"), Bytes.toBytes("g")));
 
-    r = new Result(kvs);
+    r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -655,7 +656,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfc"), Bytes.toBytes("d"), Bytes.toBytes("no")));
 
-    r = new Result(kvs);
+    r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -669,7 +670,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfc"), Bytes.toBytes("d"), Bytes.toBytes("no")));
 
-    r = new Result(kvs);
+    r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -685,7 +686,7 @@ public class TestLazyHBaseObject extends TestCase {
     kvs.add(new KeyValue(Bytes.toBytes("test-row"),
         Bytes.toBytes("cfc"), Bytes.toBytes("d"), Bytes.toBytes("")));
 
-    r = new Result(kvs);
+    r = Result.create(kvs);
     o.init(r);
 
     assertEquals(
@@ -736,7 +737,7 @@ public class TestLazyHBaseObject extends TestCase {
     LazyHBaseRow o = new LazyHBaseRow((LazySimpleStructObjectInspector) oi, columnMappings);
 
     byte [] rowKey = "row-key".getBytes();
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    List<Cell> kvs = new ArrayList<Cell>();
     byte [] value;
 
     for (int i = 1; i < columnsMapping.length; i++) {
@@ -784,7 +785,7 @@ public class TestLazyHBaseObject extends TestCase {
     }
 
     Collections.sort(kvs, KeyValue.COMPARATOR);
-    Result result = new Result(kvs);
+    Result result = Result.create(kvs);
     o.init(result);
     List<? extends StructField> fieldRefs = ((StructObjectInspector) oi).getAllStructFieldRefs();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java
index 561b0a8..cd9afed 100644
--- a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java
+++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java
@@ -43,9 +43,9 @@ public class TestPutResultWritable {
       new KeyValue(Bytes.toBytes("test-row"), Bytes.toBytes("cfa"),
         Bytes.toBytes("col2"), Bytes.toBytes("cfacol2"))
     };
-    Result expected = new Result(kvs);
+    Result expected = Result.create(kvs);
     ResultWritable actual = copy(new ResultWritable(expected), new ResultWritable());
-    Assert.assertArrayEquals(expected.raw(), actual.getResult().raw());
+    Assert.assertArrayEquals(expected.rawCells(), actual.getResult().rawCells());
 
   }
 
@@ -65,7 +65,8 @@ public class TestPutResultWritable {
     }
     PutWritable actual = copy(new PutWritable(expected), new PutWritable());
     Assert.assertArrayEquals(expected.getRow(), actual.getPut().getRow());
-    Assert.assertEquals(expected.getFamilyMap(), actual.getPut().getFamilyMap());
+    Assert.assertEquals(expected.getFamilyCellMap().keySet(),
+        actual.getPut().getFamilyCellMap().keySet());
   }
 
   private <T extends Writable> T copy(T oldWritable, T newWritable) throws IOException {

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/test/queries/positive/hbase_bulk.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_bulk.q b/hbase-handler/src/test/queries/positive/hbase_bulk.q
index 475aafc..5e0c14e 100644
--- a/hbase-handler/src/test/queries/positive/hbase_bulk.q
+++ b/hbase-handler/src/test/queries/positive/hbase_bulk.q
@@ -9,7 +9,7 @@ create table hbsort(key string, val string, val2 string)
 stored as
 INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
 OUTPUTFORMAT 'org.apache.hadoop.hive.hbase.HiveHFileOutputFormat'
-TBLPROPERTIES ('hfile.family.path' = '/tmp/hbsort/cf');
+TBLPROPERTIES ('hfile.family.path' = '/tmp/hbsort/cf','hbase.mapreduce.hfileoutputformat.table.name'='hbsort');
 
 -- this is a dummy table used for controlling how the input file
 -- for TotalOrderPartitioner is created

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q b/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q
index 85581ec..ac2fdfa 100644
--- a/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q
+++ b/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q
@@ -6,7 +6,7 @@ drop table if exists hb_target;
 create table hb_target(key int, val string)
 stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
 with serdeproperties ('hbase.columns.mapping' = ':key,cf:val')
-tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk');
+tblproperties ('hbase.mapreduce.hfileoutputformat.table.name' = 'positive_hbase_handler_bulk');
 
 set hive.hbase.generatehfiles=true;
 set hfile.family.path=/tmp/hb_target/cf;

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out b/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out
index 1f42567..10e1c0a 100644
--- a/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out
@@ -5,14 +5,14 @@ POSTHOOK: type: DROPTABLE
 PREHOOK: query: create table hb_target(key int, val string)
 stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
 with serdeproperties ('hbase.columns.mapping' = ':key,cf:val')
-tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk')
+tblproperties ('hbase.mapreduce.hfileoutputformat.table.name' = 'positive_hbase_handler_bulk')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@hb_target
 POSTHOOK: query: create table hb_target(key int, val string)
 stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
 with serdeproperties ('hbase.columns.mapping' = ':key,cf:val')
-tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk')
+tblproperties ('hbase.mapreduce.hfileoutputformat.table.name' = 'positive_hbase_handler_bulk')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@hb_target

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java
index 4ace16c..65bfb9c 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java
@@ -739,7 +739,7 @@ public class HCatUtil {
 
   public static boolean isHadoop23() {
     String version = org.apache.hadoop.util.VersionInfo.getVersion();
-    if (version.matches("\\b0\\.23\\..+\\b")||version.matches("\\b2\\..*"))
+    if (version.matches("\\b0\\.23\\..+\\b")||version.matches("\\b2\\..*")||version.matches("\\b3\\..*"))
       return true;
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
index bb6c582..b146b24 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable;
 import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
@@ -114,7 +115,6 @@ public class TestRCFileMapReduceInputFormat extends TestCase {
       patialS.set(5, new BytesRefWritable("NULL".getBytes("UTF-8")));
       patialS.set(6, new BytesRefWritable("NULL".getBytes("UTF-8")));
       patialS.set(7, new BytesRefWritable("NULL".getBytes("UTF-8")));
-
     } catch (UnsupportedEncodingException e) {
     }
   }
@@ -182,24 +182,24 @@ public class TestRCFileMapReduceInputFormat extends TestCase {
   }
 
   private void splitBeforeSync() throws IOException, InterruptedException {
-    writeThenReadByRecordReader(600, 1000, 2, 17684, null);
+    writeThenReadByRecordReader(600, 10000, 2, 176840, null);
   }
 
   private void splitRightBeforeSync() throws IOException, InterruptedException {
-    writeThenReadByRecordReader(500, 1000, 2, 17750, null);
+    writeThenReadByRecordReader(500, 10000, 2, 177500, null);
   }
 
   private void splitInMiddleOfSync() throws IOException, InterruptedException {
-    writeThenReadByRecordReader(500, 1000, 2, 17760, null);
+    writeThenReadByRecordReader(500, 10000, 2, 177600, null);
 
   }
 
   private void splitRightAfterSync() throws IOException, InterruptedException {
-    writeThenReadByRecordReader(500, 1000, 2, 17770, null);
+    writeThenReadByRecordReader(500, 10000, 2, 177700, null);
   }
 
   private void splitAfterSync() throws IOException, InterruptedException {
-    writeThenReadByRecordReader(500, 1000, 2, 19950, null);
+    writeThenReadByRecordReader(500, 10000, 2, 199500, null);
   }
 
   private void writeThenReadByRecordReader(int intervalRecordCount,
@@ -233,7 +233,7 @@ public class TestRCFileMapReduceInputFormat extends TestCase {
     HiveConf.setLongVar(context.getConfiguration(),
         HiveConf.ConfVars.MAPREDMAXSPLITSIZE, maxSplitSize);
     List<InputSplit> splits = inputFormat.getSplits(context);
-    assertEquals("splits length should be " + splitNumber, splits.size(), splitNumber);
+    assertEquals("splits length should be " + splitNumber, splitNumber, splits.size());
     int readCount = 0;
     for (int i = 0; i < splits.size(); i++) {
       TaskAttemptContext tac = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(jonconf,

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestWebHCatE2e.java
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestWebHCatE2e.java b/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestWebHCatE2e.java
index 22d2cc6..341314b 100644
--- a/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestWebHCatE2e.java
+++ b/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestWebHCatE2e.java
@@ -238,7 +238,7 @@ public class TestWebHCatE2e {
     Map<String, Object> props = JsonBuilder.jsonToMap(p.responseBody);
     Assert.assertEquals("hadoop", props.get("module"));
     Assert.assertTrue(p.getAssertMsg(),
-        ((String)props.get("version")).matches("[1-2].[0-9]+.[0-9]+.*"));
+        ((String)props.get("version")).matches("[1-3].[0-9]+.[0-9]+.*"));
   }
 
   @Test
@@ -356,4 +356,4 @@ public class TestWebHCatE2e {
     }
     return new MethodCallRetVal(-1, "Http " + type + " failed; see log file for details", actualUri, method.getName());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/mock/MockUriInfo.java
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/mock/MockUriInfo.java b/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/mock/MockUriInfo.java
index d69a733..30c4a05 100644
--- a/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/mock/MockUriInfo.java
+++ b/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/mock/MockUriInfo.java
@@ -136,4 +136,15 @@ public class MockUriInfo implements UriInfo {
     return null;
   }
 
+  @Override
+  public URI relativize(URI uri) {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public URI resolve(URI uri) {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/itests/hcatalog-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/pom.xml b/itests/hcatalog-unit/pom.xml
index bb6b105..2a2f845 100644
--- a/itests/hcatalog-unit/pom.xml
+++ b/itests/hcatalog-unit/pom.xml
@@ -283,6 +283,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-mapreduce</artifactId>
+      <version>${hbase.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
       <version>${hbase.version}</version>
       <classifier>tests</classifier>

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java
index 745aa99..ad44bc2 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java
@@ -25,8 +25,10 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -121,7 +123,6 @@ public class ManyMiniCluster {
 
   protected synchronized void stop() {
     if (hbaseCluster != null) {
-      HConnectionManager.deleteAllConnections(true);
       try {
         hbaseCluster.shutdown();
       } catch (Exception e) {
@@ -245,6 +246,8 @@ public class ManyMiniCluster {
 
   private void setupHBaseCluster() {
     final int numRegionServers = 1;
+    Connection connection = null;
+    Table table = null;
 
     try {
       hbaseDir = new File(workDir, "hbase").getCanonicalPath();
@@ -266,9 +269,25 @@ public class ManyMiniCluster {
       hbaseCluster = new MiniHBaseCluster(hbaseConf, numRegionServers);
       hbaseConf.set("hbase.master", hbaseCluster.getMaster().getServerName().getHostAndPort());
       //opening the META table ensures that cluster is running
-      new HTable(hbaseConf, HConstants.META_TABLE_NAME);
+      connection = ConnectionFactory.createConnection(hbaseConf);
+      table = connection.getTable(TableName.META_TABLE_NAME);
     } catch (Exception e) {
       throw new IllegalStateException("Failed to setup HBase Cluster", e);
+    } finally {
+      if (table != null) {
+        try {
+          table.close();
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+      if (connection != null) {
+        try {
+          connection.close();
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a3e87282/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/SkeletonHBaseTest.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/SkeletonHBaseTest.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/SkeletonHBaseTest.java
index 4e1384a..c8bb4f5 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/SkeletonHBaseTest.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/SkeletonHBaseTest.java
@@ -33,6 +33,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.junit.AfterClass;
@@ -56,10 +60,13 @@ public abstract class SkeletonHBaseTest {
    */
   protected static Configuration testConf = null;
 
-  protected void createTable(String tableName, String[] families) {
+  protected void createTable(String tableName, String[] families) throws IOException {
+    Connection connection = null;
+    Admin admin = null;
     try {
-      HBaseAdmin admin = new HBaseAdmin(getHbaseConf());
-      HTableDescriptor tableDesc = new HTableDescriptor(tableName);
+      connection = ConnectionFactory.createConnection(getHbaseConf());
+      admin = connection.getAdmin();
+      HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(tableName));
       for (String family : families) {
         HColumnDescriptor columnDescriptor = new HColumnDescriptor(family);
         tableDesc.addFamily(columnDescriptor);
@@ -68,8 +75,14 @@ public abstract class SkeletonHBaseTest {
     } catch (Exception e) {
       e.printStackTrace();
       throw new IllegalStateException(e);
+    } finally {
+      if (admin != null) {
+        admin.close();
+      }
+      if (connection != null) {
+        connection.close();
+      }
     }
-
   }
 
   protected String newTableName(String prefix) {
@@ -90,6 +103,9 @@ public abstract class SkeletonHBaseTest {
    */
   @BeforeClass
   public static void setup() {
+    // Fix needed due to dependency for hbase-mapreduce module
+    System.setProperty("org.apache.hadoop.hbase.shaded.io.netty.packagePrefix",
+        "org.apache.hadoop.hbase.shaded.");
     if (!contextMap.containsKey(getContextHandle()))
       contextMap.put(getContextHandle(), new Context(getContextHandle()));