You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/09/12 03:21:29 UTC

svn commit: r1522098 [10/30] - in /hive/branches/vectorization: ./ beeline/src/test/org/apache/hive/beeline/src/test/ bin/ bin/ext/ common/src/java/org/apache/hadoop/hive/common/ common/src/java/org/apache/hadoop/hive/conf/ conf/ contrib/src/java/org/a...

Modified: hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java?rev=1522098&r1=1522097&r2=1522098&view=diff
==============================================================================
--- hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java (original)
+++ hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java Thu Sep 12 01:21:10 2013
@@ -36,8 +36,8 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hcatalog.cli.HCatDriver;
-import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
+import org.apache.hive.hcatalog.cli.HCatDriver;
+import org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
 import org.apache.hcatalog.hbase.snapshot.RevisionManager;
 import org.apache.hcatalog.hbase.snapshot.RevisionManagerConfiguration;
 import org.apache.zookeeper.KeeperException.NoNodeException;
@@ -45,197 +45,197 @@ import org.junit.Test;
 
 public class TestHBaseHCatStorageHandler extends SkeletonHBaseTest {
 
-    private static HiveConf   hcatConf;
-    private static HCatDriver hcatDriver;
-    private static Warehouse  wh;
-
-    public void Initialize() throws Exception {
-
-        hcatConf = getHiveConf();
-        hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
-                HCatSemanticAnalyzer.class.getName());
-        URI fsuri = getFileSystem().getUri();
-        Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(),
-                getTestDir());
-        hcatConf.set(HiveConf.ConfVars.HADOOPFS.varname, fsuri.toString());
-        hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString());
-
-        //Add hbase properties
-        for (Map.Entry<String, String> el : getHbaseConf()) {
-            if (el.getKey().startsWith("hbase.")) {
-                hcatConf.set(el.getKey(), el.getValue());
-            }
-        }
-        HBaseConfiguration.merge(
-                hcatConf,
-                RevisionManagerConfiguration.create());
+  private static HiveConf   hcatConf;
+  private static HCatDriver hcatDriver;
+  private static Warehouse  wh;
+
+  public void Initialize() throws Exception {
+
+    hcatConf = getHiveConf();
+    hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
+        HCatSemanticAnalyzer.class.getName());
+    URI fsuri = getFileSystem().getUri();
+    Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(),
+        getTestDir());
+    hcatConf.set(HiveConf.ConfVars.HADOOPFS.varname, fsuri.toString());
+    hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString());
+
+    //Add hbase properties
+    for (Map.Entry<String, String> el : getHbaseConf()) {
+      if (el.getKey().startsWith("hbase.")) {
+        hcatConf.set(el.getKey(), el.getValue());
+      }
+    }
+    HBaseConfiguration.merge(
+        hcatConf,
+        RevisionManagerConfiguration.create());
+
+    SessionState.start(new CliSessionState(hcatConf));
+    hcatDriver = new HCatDriver();
+
+  }
+
+  @Test
+  public void testTableCreateDrop() throws Exception {
+    Initialize();
+
+    hcatDriver.run("drop table test_table");
+    CommandProcessorResponse response = hcatDriver
+        .run("create table test_table(key int, value string) STORED BY " +
+                     "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+          + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val')");
 
-        SessionState.start(new CliSessionState(hcatConf));
-        hcatDriver = new HCatDriver();
+    assertEquals(0, response.getResponseCode());
 
-    }
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists("test_table");
 
-    @Test
-    public void testTableCreateDrop() throws Exception {
-        Initialize();
-
-        hcatDriver.run("drop table test_table");
-        CommandProcessorResponse response = hcatDriver
-                .run("create table test_table(key int, value string) STORED BY " +
-                		     "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                    + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val')");
-
-        assertEquals(0, response.getResponseCode());
-
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists("test_table");
-
-        assertTrue(doesTableExist);
-
-        RevisionManager rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
-        rm.open();
-        //Should be able to successfully query revision manager
-        rm.getAbortedWriteTransactions("test_table", "cf1");
-
-        hcatDriver.run("drop table test_table");
-        doesTableExist = hAdmin.tableExists("test_table");
-        assertTrue(doesTableExist == false);
-
-        try {
-            rm.getAbortedWriteTransactions("test_table", "cf1");
-        } catch (Exception e) {
-            assertTrue(e.getCause() instanceof NoNodeException);
-        }
-        rm.close();
+    assertTrue(doesTableExist);
 
-    }
+    RevisionManager rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
+    rm.open();
+    //Should be able to successfully query revision manager
+    rm.getAbortedWriteTransactions("test_table", "cf1");
 
-    @Test
-    public void testTableCreateDropDifferentCase() throws Exception {
-        Initialize();
-
-        hcatDriver.run("drop table test_Table");
-        CommandProcessorResponse response = hcatDriver
-                .run("create table test_Table(key int, value string) STORED BY " +
-                             "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                    + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val')");
-
-        assertEquals(0, response.getResponseCode());
-
-        //HBase table gets created with lower case unless specified as a table property.
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists("test_table");
-
-        assertTrue(doesTableExist);
-
-        RevisionManager rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
-        rm.open();
-        //Should be able to successfully query revision manager
-        rm.getAbortedWriteTransactions("test_table", "cf1");
-
-        hcatDriver.run("drop table test_table");
-        doesTableExist = hAdmin.tableExists("test_table");
-        assertTrue(doesTableExist == false);
-
-        try {
-            rm.getAbortedWriteTransactions("test_table", "cf1");
-        } catch (Exception e) {
-            assertTrue(e.getCause() instanceof NoNodeException);
-        }
-        rm.close();
+    hcatDriver.run("drop table test_table");
+    doesTableExist = hAdmin.tableExists("test_table");
+    assertTrue(doesTableExist == false);
 
+    try {
+      rm.getAbortedWriteTransactions("test_table", "cf1");
+    } catch (Exception e) {
+      assertTrue(e.getCause() instanceof NoNodeException);
     }
+    rm.close();
+
+  }
+
+  @Test
+  public void testTableCreateDropDifferentCase() throws Exception {
+    Initialize();
+
+    hcatDriver.run("drop table test_Table");
+    CommandProcessorResponse response = hcatDriver
+        .run("create table test_Table(key int, value string) STORED BY " +
+               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+          + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val')");
+
+    assertEquals(0, response.getResponseCode());
 
-    @Test
-    public void testTableCreateDropCaseSensitive() throws Exception {
-        Initialize();
-
-        hcatDriver.run("drop table test_Table");
-        CommandProcessorResponse response = hcatDriver
-                .run("create table test_Table(key int, value string) STORED BY " +
-                             "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                    + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val'," +
-                    " 'hbase.table.name'='CaseSensitiveTable')");
-
-        assertEquals(0, response.getResponseCode());
-
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists("CaseSensitiveTable");
-
-        assertTrue(doesTableExist);
-
-        RevisionManager rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
-        rm.open();
-        //Should be able to successfully query revision manager
-        rm.getAbortedWriteTransactions("CaseSensitiveTable", "cf1");
-
-        hcatDriver.run("drop table test_table");
-        doesTableExist = hAdmin.tableExists("CaseSensitiveTable");
-        assertTrue(doesTableExist == false);
-
-        try {
-            rm.getAbortedWriteTransactions("CaseSensitiveTable", "cf1");
-        } catch (Exception e) {
-            assertTrue(e.getCause() instanceof NoNodeException);
-        }
-        rm.close();
+    //HBase table gets created with lower case unless specified as a table property.
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists("test_table");
 
+    assertTrue(doesTableExist);
+
+    RevisionManager rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
+    rm.open();
+    //Should be able to successfully query revision manager
+    rm.getAbortedWriteTransactions("test_table", "cf1");
+
+    hcatDriver.run("drop table test_table");
+    doesTableExist = hAdmin.tableExists("test_table");
+    assertTrue(doesTableExist == false);
+
+    try {
+      rm.getAbortedWriteTransactions("test_table", "cf1");
+    } catch (Exception e) {
+      assertTrue(e.getCause() instanceof NoNodeException);
     }
+    rm.close();
 
-    @Test
-    public void testTableDropNonExistent() throws Exception {
-        Initialize();
-
-        hcatDriver.run("drop table mytable");
-        CommandProcessorResponse response = hcatDriver
-                .run("create table mytable(key int, value string) STORED BY " +
-                     "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                    + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val')");
+  }
+
+  @Test
+  public void testTableCreateDropCaseSensitive() throws Exception {
+    Initialize();
+
+    hcatDriver.run("drop table test_Table");
+    CommandProcessorResponse response = hcatDriver
+        .run("create table test_Table(key int, value string) STORED BY " +
+               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+          + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val'," +
+          " 'hbase.table.name'='CaseSensitiveTable')");
 
-        assertEquals(0, response.getResponseCode());
+    assertEquals(0, response.getResponseCode());
 
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists("mytable");
-        assertTrue(doesTableExist);
-
-        //Now delete the table from hbase
-        if (hAdmin.isTableEnabled("mytable")) {
-            hAdmin.disableTable("mytable");
-        }
-        hAdmin.deleteTable("mytable");
-        doesTableExist = hAdmin.tableExists("mytable");
-        assertTrue(doesTableExist == false);
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists("CaseSensitiveTable");
 
-        CommandProcessorResponse responseTwo = hcatDriver.run("drop table mytable");
-        assertTrue(responseTwo.getResponseCode() == 0);
+    assertTrue(doesTableExist);
 
+    RevisionManager rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
+    rm.open();
+    //Should be able to successfully query revision manager
+    rm.getAbortedWriteTransactions("CaseSensitiveTable", "cf1");
+
+    hcatDriver.run("drop table test_table");
+    doesTableExist = hAdmin.tableExists("CaseSensitiveTable");
+    assertTrue(doesTableExist == false);
+
+    try {
+      rm.getAbortedWriteTransactions("CaseSensitiveTable", "cf1");
+    } catch (Exception e) {
+      assertTrue(e.getCause() instanceof NoNodeException);
     }
+    rm.close();
 
-    @Test
-    public void testTableCreateExternal() throws Exception {
+  }
 
-        String tableName = "testTable";
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+  @Test
+  public void testTableDropNonExistent() throws Exception {
+    Initialize();
 
-        HTableDescriptor tableDesc = new HTableDescriptor(tableName);
-        tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("key")));
-        tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("familyone")));
-        tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("familytwo")));
-
-        hAdmin.createTable(tableDesc);
-        boolean doesTableExist = hAdmin.tableExists(tableName);
-        assertTrue(doesTableExist);
-
-        hcatDriver.run("drop table mytabletwo");
-        CommandProcessorResponse response = hcatDriver
-                .run("create external table mytabletwo(key int, valueone string, valuetwo string) STORED BY " +
-                     "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                    + "TBLPROPERTIES ('hbase.columns.mapping'=':key,familyone:val,familytwo:val'," +
-                    "'hbase.table.name'='testTable')");
+    hcatDriver.run("drop table mytable");
+    CommandProcessorResponse response = hcatDriver
+        .run("create table mytable(key int, value string) STORED BY " +
+           "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+          + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val')");
+
+    assertEquals(0, response.getResponseCode());
 
-        assertEquals(0, response.getResponseCode());
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists("mytable");
+    assertTrue(doesTableExist);
 
+    //Now delete the table from hbase
+    if (hAdmin.isTableEnabled("mytable")) {
+      hAdmin.disableTable("mytable");
     }
+    hAdmin.deleteTable("mytable");
+    doesTableExist = hAdmin.tableExists("mytable");
+    assertTrue(doesTableExist == false);
+
+    CommandProcessorResponse responseTwo = hcatDriver.run("drop table mytable");
+    assertTrue(responseTwo.getResponseCode() == 0);
+
+  }
+
+  @Test
+  public void testTableCreateExternal() throws Exception {
+
+    String tableName = "testTable";
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+
+    HTableDescriptor tableDesc = new HTableDescriptor(tableName);
+    tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("key")));
+    tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("familyone")));
+    tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("familytwo")));
+
+    hAdmin.createTable(tableDesc);
+    boolean doesTableExist = hAdmin.tableExists(tableName);
+    assertTrue(doesTableExist);
+
+    hcatDriver.run("drop table mytabletwo");
+    CommandProcessorResponse response = hcatDriver
+        .run("create external table mytabletwo(key int, valueone string, valuetwo string) STORED BY " +
+           "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+          + "TBLPROPERTIES ('hbase.columns.mapping'=':key,familyone:val,familytwo:val'," +
+          "'hbase.table.name'='testTable')");
+
+    assertEquals(0, response.getResponseCode());
+
+  }
 
 
 }

Modified: hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputFormat.java?rev=1522098&r1=1522097&r2=1522098&view=diff
==============================================================================
--- hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputFormat.java (original)
+++ hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputFormat.java Thu Sep 12 01:21:10 2013
@@ -59,551 +59,551 @@ import org.apache.hadoop.mapred.RunningJ
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.hcatalog.cli.HCatDriver;
-import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
-import org.apache.hcatalog.common.HCatConstants;
-import org.apache.hcatalog.common.HCatException;
-import org.apache.hcatalog.common.HCatUtil;
-import org.apache.hcatalog.data.HCatRecord;
-import org.apache.hcatalog.data.schema.HCatFieldSchema;
-import org.apache.hcatalog.data.schema.HCatSchema;
+import org.apache.hive.hcatalog.cli.HCatDriver;
+import org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
+import org.apache.hive.hcatalog.common.HCatConstants;
+import org.apache.hive.hcatalog.common.HCatException;
+import org.apache.hive.hcatalog.common.HCatUtil;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
+import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hcatalog.hbase.snapshot.RevisionManager;
 import org.apache.hcatalog.hbase.snapshot.RevisionManagerConfiguration;
 import org.apache.hcatalog.hbase.snapshot.Transaction;
-import org.apache.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hcatalog.mapreduce.InputJobInfo;
-import org.apache.hcatalog.mapreduce.PartInfo;
+import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
+import org.apache.hive.hcatalog.mapreduce.PartInfo;
 import org.junit.Test;
 
 public class TestHBaseInputFormat extends SkeletonHBaseTest {
 
-    private static HiveConf hcatConf;
-    private static HCatDriver hcatDriver;
-    private final byte[] FAMILY = Bytes.toBytes("testFamily");
-    private final byte[] QUALIFIER1 = Bytes.toBytes("testQualifier1");
-    private final byte[] QUALIFIER2 = Bytes.toBytes("testQualifier2");
-
-    public TestHBaseInputFormat() throws Exception {
-        hcatConf = getHiveConf();
-        hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
-            HCatSemanticAnalyzer.class.getName());
-        URI fsuri = getFileSystem().getUri();
-        Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(),
-            getTestDir());
-        hcatConf.set(HiveConf.ConfVars.HADOOPFS.varname, fsuri.toString());
-        hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString());
-
-        //Add hbase properties
-
-        for (Map.Entry<String, String> el : getHbaseConf()) {
-            if (el.getKey().startsWith("hbase.")) {
-                hcatConf.set(el.getKey(), el.getValue());
-            }
-        }
-        HBaseConfiguration.merge(hcatConf,
-            RevisionManagerConfiguration.create());
-
-
-        SessionState.start(new CliSessionState(hcatConf));
-        hcatDriver = new HCatDriver();
-
-    }
-
-    private List<Put> generatePuts(int num, String tableName) throws IOException {
-
-        List<String> columnFamilies = Arrays.asList("testFamily");
-        RevisionManager rm = null;
-        List<Put> myPuts;
-        try {
-            rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
-            rm.open();
-            myPuts = new ArrayList<Put>();
-            for (int i = 1; i <= num; i++) {
-                Put put = new Put(Bytes.toBytes("testRow"));
-                put.add(FAMILY, QUALIFIER1, i, Bytes.toBytes("textValue-" + i));
-                put.add(FAMILY, QUALIFIER2, i, Bytes.toBytes("textValue-" + i));
-                myPuts.add(put);
-                Transaction tsx = rm.beginWriteTransaction(tableName,
-                    columnFamilies);
-                rm.commitWriteTransaction(tsx);
-            }
-        } finally {
-            if (rm != null)
-                rm.close();
-        }
-
-        return myPuts;
-    }
-
-    private void populateHBaseTable(String tName, int revisions) throws IOException {
-        List<Put> myPuts = generatePuts(revisions, tName);
-        HTable table = new HTable(getHbaseConf(), Bytes.toBytes(tName));
-        table.put(myPuts);
-    }
-
-    private long populateHBaseTableQualifier1(String tName, int value, Boolean commit)
-        throws IOException {
-        List<String> columnFamilies = Arrays.asList("testFamily");
-        RevisionManager rm = null;
-        List<Put> myPuts = new ArrayList<Put>();
-        long revision;
-        try {
-            rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
-            rm.open();
-            Transaction tsx = rm.beginWriteTransaction(tName, columnFamilies);
-
-            Put put = new Put(Bytes.toBytes("testRow"));
-            revision = tsx.getRevisionNumber();
-            put.add(FAMILY, QUALIFIER1, revision,
-                Bytes.toBytes("textValue-" + value));
-            myPuts.add(put);
-
-            // If commit is null it is left as a running transaction
-            if (commit != null) {
-                if (commit) {
-                    rm.commitWriteTransaction(tsx);
-                } else {
-                    rm.abortWriteTransaction(tsx);
-                }
-            }
-        } finally {
-            if (rm != null)
-                rm.close();
-        }
-        HTable table = new HTable(getHbaseConf(), Bytes.toBytes(tName));
-        table.put(myPuts);
-        return revision;
-    }
-
-    @Test
-    public void TestHBaseTableReadMR() throws Exception {
-        String tableName = newTableName("MyTable");
-        String databaseName = newTableName("MyDatabase");
-        //Table name will be lower case unless specified by hbase.table.name property
-        String hbaseTableName = (databaseName + "." + tableName).toLowerCase();
-        String db_dir = getTestDir() + "/hbasedb";
-
-        String dbquery = "CREATE DATABASE IF NOT EXISTS " + databaseName + " LOCATION '"
-            + db_dir + "'";
-        String tableQuery = "CREATE TABLE " + databaseName + "." + tableName
-            + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
-            "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-            + "TBLPROPERTIES ('hbase.columns.mapping'=':key,testFamily:testQualifier1,testFamily:testQualifier2')";
-
-        CommandProcessorResponse responseOne = hcatDriver.run(dbquery);
-        assertEquals(0, responseOne.getResponseCode());
-        CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
-        assertEquals(0, responseTwo.getResponseCode());
-
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists(hbaseTableName);
-        assertTrue(doesTableExist);
-
-        populateHBaseTable(hbaseTableName, 5);
-        Configuration conf = new Configuration(hcatConf);
-        conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
-            HCatUtil.serialize(getHiveConf().getAllProperties()));
-
-        // output settings
-        Path outputDir = new Path(getTestDir(), "mapred/testHbaseTableMRRead");
-        FileSystem fs = getFileSystem();
-        if (fs.exists(outputDir)) {
-            fs.delete(outputDir, true);
-        }
-        // create job
-        Job job = new Job(conf, "hbase-mr-read-test");
-        job.setJarByClass(this.getClass());
-        job.setMapperClass(MapReadHTable.class);
-        MapReadHTable.resetCounters();
-
-        job.setInputFormatClass(HCatInputFormat.class);
-        HCatInputFormat.setInput(job.getConfiguration(), databaseName, tableName);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        TextOutputFormat.setOutputPath(job, outputDir);
-        job.setMapOutputKeyClass(BytesWritable.class);
-        job.setMapOutputValueClass(Text.class);
-        job.setOutputKeyClass(BytesWritable.class);
-        job.setOutputValueClass(Text.class);
-        job.setNumReduceTasks(0);
-        assertTrue(job.waitForCompletion(true));
-        // Note: These asserts only works in case of LocalJobRunner as they run in same jvm.
-        // If using MiniMRCluster, the tests will have to be modified.
-        assertFalse(MapReadHTable.error);
-        assertEquals(MapReadHTable.count, 1);
-
-        String dropTableQuery = "DROP TABLE " + hbaseTableName;
-        CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
-        assertEquals(0, responseThree.getResponseCode());
-
-        boolean isHbaseTableThere = hAdmin.tableExists(hbaseTableName);
-        assertFalse(isHbaseTableThere);
-
-        String dropDB = "DROP DATABASE " + databaseName;
-        CommandProcessorResponse responseFour = hcatDriver.run(dropDB);
-        assertEquals(0, responseFour.getResponseCode());
-    }
-
-    @Test
-    public void TestHBaseTableProjectionReadMR() throws Exception {
-
-        String tableName = newTableName("MyTable");
-        //Table name as specified by hbase.table.name property
-        String hbaseTableName = "MyDB_" + tableName;
-        String tableQuery = "CREATE TABLE " + tableName
-            + "(key string, testqualifier1 string, testqualifier2 string) STORED BY "
-            + "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-            + "TBLPROPERTIES ('hbase.columns.mapping'="
-            + "':key,testFamily:testQualifier1,testFamily:testQualifier2',"
-            + "'hbase.table.name'='" + hbaseTableName + "')";
-
-        CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
-        assertEquals(0, responseTwo.getResponseCode());
-
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists(hbaseTableName);
-        assertTrue(doesTableExist);
-
-        populateHBaseTable(hbaseTableName, 5);
-
-        Configuration conf = new Configuration(hcatConf);
-        conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
-            HCatUtil.serialize(getHiveConf().getAllProperties()));
-
-        // output settings
-        Path outputDir = new Path(getTestDir(), "mapred/testHBaseTableProjectionReadMR");
-        FileSystem fs = getFileSystem();
-        if (fs.exists(outputDir)) {
-            fs.delete(outputDir, true);
-        }
-        // create job
-        Job job = new Job(conf, "hbase-column-projection");
-        job.setJarByClass(this.getClass());
-        job.setMapperClass(MapReadProjHTable.class);
-        job.setInputFormatClass(HCatInputFormat.class);
-        HCatInputFormat.setOutputSchema(job, getProjectionSchema());
-        HCatInputFormat.setInput(job, MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        TextOutputFormat.setOutputPath(job, outputDir);
-        job.setMapOutputKeyClass(BytesWritable.class);
-        job.setMapOutputValueClass(Text.class);
-        job.setOutputKeyClass(BytesWritable.class);
-        job.setOutputValueClass(Text.class);
-        job.setNumReduceTasks(0);
-        assertTrue(job.waitForCompletion(true));
-        assertFalse(MapReadProjHTable.error);
-        assertEquals(MapReadProjHTable.count, 1);
-
-        String dropTableQuery = "DROP TABLE " + tableName;
-        CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
-        assertEquals(0, responseThree.getResponseCode());
-
-        boolean isHbaseTableThere = hAdmin.tableExists(hbaseTableName);
-        assertFalse(isHbaseTableThere);
-    }
-
-    @Test
-    public void TestHBaseInputFormatProjectionReadMR() throws Exception {
-
-        String tableName = newTableName("mytable");
-        String tableQuery = "CREATE TABLE " + tableName
-            + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
-            "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-            + "TBLPROPERTIES ('hbase.columns.mapping'=':key," +
-            "testFamily:testQualifier1,testFamily:testQualifier2')";
-
-        CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
-        assertEquals(0, responseTwo.getResponseCode());
-
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists(tableName);
-        assertTrue(doesTableExist);
-
-        populateHBaseTable(tableName, 5);
-
-        Configuration conf = new Configuration(hcatConf);
-        conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
-            HCatUtil.serialize(getHiveConf().getAllProperties()));
-
-        // output settings
-        Path outputDir = new Path(getTestDir(), "mapred/testHBaseTableProjectionReadMR");
-        FileSystem fs = getFileSystem();
-        if (fs.exists(outputDir)) {
-            fs.delete(outputDir, true);
-        }
-        // create job
-        JobConf job = new JobConf(conf);
-        job.setJobName("hbase-scan-column");
-        job.setJarByClass(this.getClass());
-        job.setMapperClass(MapReadProjectionHTable.class);
-        job.setInputFormat(HBaseInputFormat.class);
-
-        //Configure projection schema
-        job.set(HCatConstants.HCAT_KEY_OUTPUT_SCHEMA, HCatUtil.serialize(getProjectionSchema()));
-        Job newJob = new Job(job);
-        HCatInputFormat.setInput(newJob, MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
-        String inputJobString = newJob.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO);
-        InputJobInfo info = (InputJobInfo) HCatUtil.deserialize(inputJobString);
-        job.set(HCatConstants.HCAT_KEY_JOB_INFO, inputJobString);
-        for (PartInfo partinfo : info.getPartitions()) {
-            for (Entry<String, String> entry : partinfo.getJobProperties().entrySet())
-                job.set(entry.getKey(), entry.getValue());
-        }
-        assertEquals("testFamily:testQualifier1", job.get(TableInputFormat.SCAN_COLUMNS));
-
-        job.setOutputFormat(org.apache.hadoop.mapred.TextOutputFormat.class);
-        org.apache.hadoop.mapred.TextOutputFormat.setOutputPath(job, outputDir);
-        job.setMapOutputKeyClass(BytesWritable.class);
-        job.setMapOutputValueClass(Text.class);
-        job.setOutputKeyClass(BytesWritable.class);
-        job.setOutputValueClass(Text.class);
-        job.setNumReduceTasks(0);
-
-        RunningJob runJob = JobClient.runJob(job);
-        runJob.waitForCompletion();
-        assertTrue(runJob.isSuccessful());
-        assertFalse(MapReadProjHTable.error);
-        assertEquals(MapReadProjHTable.count, 1);
-
-        String dropTableQuery = "DROP TABLE " + tableName;
-        CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
-        assertEquals(0, responseThree.getResponseCode());
-
-        boolean isHbaseTableThere = hAdmin.tableExists(tableName);
-        assertFalse(isHbaseTableThere);
-    }
-
-    @Test
-    public void TestHBaseTableIgnoreAbortedTransactions() throws Exception {
-        String tableName = newTableName("mytable");
-        String tableQuery = "CREATE TABLE " + tableName
-            + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
-            "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-            + "TBLPROPERTIES ('hbase.columns.mapping'=':key," +
-            "testFamily:testQualifier1,testFamily:testQualifier2')";
-
-        CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
-        assertEquals(0, responseTwo.getResponseCode());
-
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists(tableName);
-        assertTrue(doesTableExist);
-
-        populateHBaseTable(tableName, 5);
-        populateHBaseTableQualifier1(tableName, 6, false);
-        populateHBaseTableQualifier1(tableName, 7, false);
-
-        Configuration conf = new Configuration(hcatConf);
-        conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
-            HCatUtil.serialize(getHiveConf().getAllProperties()));
-
-        Path outputDir = new Path(getTestDir(), "mapred/testHBaseTableIgnoreAbortedTransactions");
-        FileSystem fs = getFileSystem();
-        if (fs.exists(outputDir)) {
-            fs.delete(outputDir, true);
-        }
-        Job job = new Job(conf, "hbase-aborted-transaction");
-        job.setJarByClass(this.getClass());
-        job.setMapperClass(MapReadHTable.class);
-        MapReadHTable.resetCounters();
-        job.setInputFormatClass(HCatInputFormat.class);
-        HCatInputFormat.setInput(job, MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        TextOutputFormat.setOutputPath(job, outputDir);
-        job.setMapOutputKeyClass(BytesWritable.class);
-        job.setMapOutputValueClass(Text.class);
-        job.setOutputKeyClass(BytesWritable.class);
-        job.setOutputValueClass(Text.class);
-        job.setNumReduceTasks(0);
-        assertTrue(job.waitForCompletion(true));
-        // Verify that the records do not contain aborted transaction
-        // revisions 6 and 7 for testFamily:testQualifier1 and
-        // fetches revision 5 for both testQualifier1 and testQualifier2
-        assertFalse(MapReadHTable.error);
-        assertEquals(1, MapReadHTable.count);
-
-        String dropTableQuery = "DROP TABLE " + tableName;
-        CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
-        assertEquals(0, responseThree.getResponseCode());
-
-        boolean isHbaseTableThere = hAdmin.tableExists(tableName);
-        assertFalse(isHbaseTableThere);
-    }
-
-    @Test
-    public void TestHBaseTableIgnoreAbortedAndRunningTransactions() throws Exception {
-        String tableName = newTableName("mytable");
-        String tableQuery = "CREATE TABLE " + tableName
-            + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
-            "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-            + "TBLPROPERTIES ('hbase.columns.mapping'=':key," +
-            "testFamily:testQualifier1,testFamily:testQualifier2')";
-
-        CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
-        assertEquals(0, responseTwo.getResponseCode());
-
-        HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
-        boolean doesTableExist = hAdmin.tableExists(tableName);
-        assertTrue(doesTableExist);
-
-        populateHBaseTable(tableName, 2);
-        populateHBaseTableQualifier1(tableName, 3, Boolean.TRUE); //Committed transaction
-        populateHBaseTableQualifier1(tableName, 4, null); //Running transaction
-        populateHBaseTableQualifier1(tableName, 5, Boolean.FALSE);  //Aborted transaction
-        populateHBaseTableQualifier1(tableName, 6, Boolean.TRUE); //Committed transaction
-        populateHBaseTableQualifier1(tableName, 7, null); //Running Transaction
-        populateHBaseTableQualifier1(tableName, 8, Boolean.FALSE); //Aborted Transaction
-
-        Configuration conf = new Configuration(hcatConf);
-        conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
-            HCatUtil.serialize(getHiveConf().getAllProperties()));
-
-        Path outputDir = new Path(getTestDir(), "mapred/testHBaseTableIgnoreAbortedTransactions");
-        FileSystem fs = getFileSystem();
-        if (fs.exists(outputDir)) {
-            fs.delete(outputDir, true);
-        }
-        Job job = new Job(conf, "hbase-running-aborted-transaction");
-        job.setJarByClass(this.getClass());
-        job.setMapperClass(MapReadHTableRunningAbort.class);
-        job.setInputFormatClass(HCatInputFormat.class);
-        HCatInputFormat.setInput(job, MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        TextOutputFormat.setOutputPath(job, outputDir);
-        job.setMapOutputKeyClass(BytesWritable.class);
-        job.setMapOutputValueClass(Text.class);
-        job.setOutputKeyClass(BytesWritable.class);
-        job.setOutputValueClass(Text.class);
-        job.setNumReduceTasks(0);
-        assertTrue(job.waitForCompletion(true));
-        // Verify that the records do not contain running and aborted transaction
-        // and it fetches revision 2 for testQualifier1 and testQualifier2
-        assertFalse(MapReadHTableRunningAbort.error);
-        assertEquals(1, MapReadHTableRunningAbort.count);
-
-        String dropTableQuery = "DROP TABLE " + tableName;
-        CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
-        assertEquals(0, responseThree.getResponseCode());
-
-        boolean isHbaseTableThere = hAdmin.tableExists(tableName);
-        assertFalse(isHbaseTableThere);
-    }
-
-
-    static class MapReadHTable
-        extends
-        Mapper<ImmutableBytesWritable, HCatRecord, WritableComparable<?>, Text> {
-
-        static boolean error = false;
-        static int count = 0;
-
-        @Override
-        public void map(ImmutableBytesWritable key, HCatRecord value,
-                        Context context) throws IOException, InterruptedException {
-            System.out.println("HCat record value" + value.toString());
-            boolean correctValues = (value.size() == 3)
-                && (value.get(0).toString()).equalsIgnoreCase("testRow")
-                && (value.get(1).toString()).equalsIgnoreCase("textValue-5")
-                && (value.get(2).toString()).equalsIgnoreCase("textValue-5");
-
-            if (correctValues == false) {
-                error = true;
-            }
-            count++;
-        }
-
-        public static void resetCounters() {
-            error = false;
-            count = 0;
-        }
-    }
-
-    static class MapReadProjHTable
-        extends
-        Mapper<ImmutableBytesWritable, HCatRecord, WritableComparable<?>, Text> {
-
-        static boolean error = false;
-        static int count = 0;
-
-        @Override
-        public void map(ImmutableBytesWritable key, HCatRecord value,
-                        Context context) throws IOException, InterruptedException {
-            System.out.println("HCat record value" + value.toString());
-            boolean correctValues = (value.size() == 2)
-                && (value.get(0).toString()).equalsIgnoreCase("testRow")
-                && (value.get(1).toString()).equalsIgnoreCase("textValue-5");
-
-            if (correctValues == false) {
-                error = true;
-            }
-            count++;
-        }
-    }
-
-    static class MapReadProjectionHTable
-        implements org.apache.hadoop.mapred.Mapper<ImmutableBytesWritable, Result, WritableComparable<?>, Text> {
-
-        static boolean error = false;
-        static int count = 0;
-
-        @Override
-        public void configure(JobConf job) {
-        }
-
-        @Override
-        public void close() throws IOException {
-        }
-
-        @Override
-        public void map(ImmutableBytesWritable key, Result result,
-                        OutputCollector<WritableComparable<?>, Text> output, Reporter reporter)
-            throws IOException {
-            System.out.println("Result " + result.toString());
-            List<KeyValue> list = result.list();
-            boolean correctValues = (list.size() == 1)
-                && (Bytes.toString(list.get(0).getRow())).equalsIgnoreCase("testRow")
-                && (Bytes.toString(list.get(0).getValue())).equalsIgnoreCase("textValue-5")
-                && (Bytes.toString(list.get(0).getFamily())).equalsIgnoreCase("testFamily")
-                && (Bytes.toString(list.get(0).getQualifier())).equalsIgnoreCase("testQualifier1");
-
-            if (correctValues == false) {
-                error = true;
-            }
-            count++;
-        }
-    }
-
-    static class MapReadHTableRunningAbort
-        extends
-        Mapper<ImmutableBytesWritable, HCatRecord, WritableComparable<?>, Text> {
-
-        static boolean error = false;
-        static int count = 0;
-
-        @Override
-        public void map(ImmutableBytesWritable key, HCatRecord value,
-                        Context context) throws IOException, InterruptedException {
-            System.out.println("HCat record value" + value.toString());
-            boolean correctValues = (value.size() == 3)
-                && (value.get(0).toString()).equalsIgnoreCase("testRow")
-                && (value.get(1).toString()).equalsIgnoreCase("textValue-3")
-                && (value.get(2).toString()).equalsIgnoreCase("textValue-2");
-
-            if (correctValues == false) {
-                error = true;
-            }
-            count++;
-        }
-    }
-
-    private HCatSchema getProjectionSchema() throws HCatException {
-
-        HCatSchema schema = new HCatSchema(new ArrayList<HCatFieldSchema>());
-        schema.append(new HCatFieldSchema("key", HCatFieldSchema.Type.STRING,
-            ""));
-        schema.append(new HCatFieldSchema("testqualifier1",
-            HCatFieldSchema.Type.STRING, ""));
-        return schema;
-    }
+  private static HiveConf hcatConf;
+  private static HCatDriver hcatDriver;
+  private final byte[] FAMILY = Bytes.toBytes("testFamily");
+  private final byte[] QUALIFIER1 = Bytes.toBytes("testQualifier1");
+  private final byte[] QUALIFIER2 = Bytes.toBytes("testQualifier2");
+
+  public TestHBaseInputFormat() throws Exception {
+    hcatConf = getHiveConf();
+    hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
+      HCatSemanticAnalyzer.class.getName());
+    URI fsuri = getFileSystem().getUri();
+    Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(),
+      getTestDir());
+    hcatConf.set(HiveConf.ConfVars.HADOOPFS.varname, fsuri.toString());
+    hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString());
+
+    //Add hbase properties
+
+    for (Map.Entry<String, String> el : getHbaseConf()) {
+      if (el.getKey().startsWith("hbase.")) {
+        hcatConf.set(el.getKey(), el.getValue());
+      }
+    }
+    HBaseConfiguration.merge(hcatConf,
+      RevisionManagerConfiguration.create());
+
+
+    SessionState.start(new CliSessionState(hcatConf));
+    hcatDriver = new HCatDriver();
+
+  }
+
+  private List<Put> generatePuts(int num, String tableName) throws IOException {
+
+    List<String> columnFamilies = Arrays.asList("testFamily");
+    RevisionManager rm = null;
+    List<Put> myPuts;
+    try {
+      rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
+      rm.open();
+      myPuts = new ArrayList<Put>();
+      for (int i = 1; i <= num; i++) {
+        Put put = new Put(Bytes.toBytes("testRow"));
+        put.add(FAMILY, QUALIFIER1, i, Bytes.toBytes("textValue-" + i));
+        put.add(FAMILY, QUALIFIER2, i, Bytes.toBytes("textValue-" + i));
+        myPuts.add(put);
+        Transaction tsx = rm.beginWriteTransaction(tableName,
+          columnFamilies);
+        rm.commitWriteTransaction(tsx);
+      }
+    } finally {
+      if (rm != null)
+        rm.close();
+    }
+
+    return myPuts;
+  }
+
+  private void populateHBaseTable(String tName, int revisions) throws IOException {
+    List<Put> myPuts = generatePuts(revisions, tName);
+    HTable table = new HTable(getHbaseConf(), Bytes.toBytes(tName));
+    table.put(myPuts);
+  }
+
+  private long populateHBaseTableQualifier1(String tName, int value, Boolean commit)
+    throws IOException {
+    List<String> columnFamilies = Arrays.asList("testFamily");
+    RevisionManager rm = null;
+    List<Put> myPuts = new ArrayList<Put>();
+    long revision;
+    try {
+      rm = HBaseRevisionManagerUtil.getOpenedRevisionManager(hcatConf);
+      rm.open();
+      Transaction tsx = rm.beginWriteTransaction(tName, columnFamilies);
+
+      Put put = new Put(Bytes.toBytes("testRow"));
+      revision = tsx.getRevisionNumber();
+      put.add(FAMILY, QUALIFIER1, revision,
+        Bytes.toBytes("textValue-" + value));
+      myPuts.add(put);
+
+      // If commit is null it is left as a running transaction
+      if (commit != null) {
+        if (commit) {
+          rm.commitWriteTransaction(tsx);
+        } else {
+          rm.abortWriteTransaction(tsx);
+        }
+      }
+    } finally {
+      if (rm != null)
+        rm.close();
+    }
+    HTable table = new HTable(getHbaseConf(), Bytes.toBytes(tName));
+    table.put(myPuts);
+    return revision;
+  }
+
+  @Test
+  public void TestHBaseTableReadMR() throws Exception {
+    String tableName = newTableName("MyTable");
+    String databaseName = newTableName("MyDatabase");
+    //Table name will be lower case unless specified by hbase.table.name property
+    String hbaseTableName = (databaseName + "." + tableName).toLowerCase();
+    String db_dir = new Path(getTestDir(), "hbasedb").toString();
+
+    String dbquery = "CREATE DATABASE IF NOT EXISTS " + databaseName + " LOCATION '"
+      + db_dir + "'";
+    String tableQuery = "CREATE TABLE " + databaseName + "." + tableName
+      + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
+      "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+      + "TBLPROPERTIES ('hbase.columns.mapping'=':key,testFamily:testQualifier1,testFamily:testQualifier2')";
+
+    CommandProcessorResponse responseOne = hcatDriver.run(dbquery);
+    assertEquals(0, responseOne.getResponseCode());
+    CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
+    assertEquals(0, responseTwo.getResponseCode());
+
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists(hbaseTableName);
+    assertTrue(doesTableExist);
+
+    populateHBaseTable(hbaseTableName, 5);
+    Configuration conf = new Configuration(hcatConf);
+    conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
+      HCatUtil.serialize(getHiveConf().getAllProperties()));
+
+    // output settings
+    Path outputDir = new Path(getTestDir(), "mapred/testHbaseTableMRRead");
+    FileSystem fs = getFileSystem();
+    if (fs.exists(outputDir)) {
+      fs.delete(outputDir, true);
+    }
+    // create job
+    Job job = new Job(conf, "hbase-mr-read-test");
+    job.setJarByClass(this.getClass());
+    job.setMapperClass(MapReadHTable.class);
+    MapReadHTable.resetCounters();
+
+    job.setInputFormatClass(HCatInputFormat.class);
+    HCatInputFormat.setInput(job.getConfiguration(), databaseName, tableName);
+    job.setOutputFormatClass(TextOutputFormat.class);
+    TextOutputFormat.setOutputPath(job, outputDir);
+    job.setMapOutputKeyClass(BytesWritable.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setOutputKeyClass(BytesWritable.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(0);
+    assertTrue(job.waitForCompletion(true));
+    // Note: These asserts only works in case of LocalJobRunner as they run in same jvm.
+    // If using MiniMRCluster, the tests will have to be modified.
+    assertFalse(MapReadHTable.error);
+    assertEquals(MapReadHTable.count, 1);
+
+    String dropTableQuery = "DROP TABLE " + hbaseTableName;
+    CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
+    assertEquals(0, responseThree.getResponseCode());
+
+    boolean isHbaseTableThere = hAdmin.tableExists(hbaseTableName);
+    assertFalse(isHbaseTableThere);
+
+    String dropDB = "DROP DATABASE " + databaseName;
+    CommandProcessorResponse responseFour = hcatDriver.run(dropDB);
+    assertEquals(0, responseFour.getResponseCode());
+  }
+
+  @Test
+  public void TestHBaseTableProjectionReadMR() throws Exception {
+
+    String tableName = newTableName("MyTable");
+    //Table name as specified by hbase.table.name property
+    String hbaseTableName = "MyDB_" + tableName;
+    String tableQuery = "CREATE TABLE " + tableName
+      + "(key string, testqualifier1 string, testqualifier2 string) STORED BY "
+      + "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+      + "TBLPROPERTIES ('hbase.columns.mapping'="
+      + "':key,testFamily:testQualifier1,testFamily:testQualifier2',"
+      + "'hbase.table.name'='" + hbaseTableName + "')";
+
+    CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
+    assertEquals(0, responseTwo.getResponseCode());
+
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists(hbaseTableName);
+    assertTrue(doesTableExist);
+
+    populateHBaseTable(hbaseTableName, 5);
+
+    Configuration conf = new Configuration(hcatConf);
+    conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
+      HCatUtil.serialize(getHiveConf().getAllProperties()));
+
+    // output settings
+    Path outputDir = new Path(getTestDir(), "mapred/testHBaseTableProjectionReadMR");
+    FileSystem fs = getFileSystem();
+    if (fs.exists(outputDir)) {
+      fs.delete(outputDir, true);
+    }
+    // create job
+    Job job = new Job(conf, "hbase-column-projection");
+    job.setJarByClass(this.getClass());
+    job.setMapperClass(MapReadProjHTable.class);
+    job.setInputFormatClass(HCatInputFormat.class);
+    HCatInputFormat.setOutputSchema(job, getProjectionSchema());
+    HCatInputFormat.setInput(job, MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+    job.setOutputFormatClass(TextOutputFormat.class);
+    TextOutputFormat.setOutputPath(job, outputDir);
+    job.setMapOutputKeyClass(BytesWritable.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setOutputKeyClass(BytesWritable.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(0);
+    assertTrue(job.waitForCompletion(true));
+    assertFalse(MapReadProjHTable.error);
+    assertEquals(MapReadProjHTable.count, 1);
+
+    String dropTableQuery = "DROP TABLE " + tableName;
+    CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
+    assertEquals(0, responseThree.getResponseCode());
+
+    boolean isHbaseTableThere = hAdmin.tableExists(hbaseTableName);
+    assertFalse(isHbaseTableThere);
+  }
+
+  @Test
+  public void TestHBaseInputFormatProjectionReadMR() throws Exception {
+
+    String tableName = newTableName("mytable");
+    String tableQuery = "CREATE TABLE " + tableName
+      + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
+      "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+      + "TBLPROPERTIES ('hbase.columns.mapping'=':key," +
+      "testFamily:testQualifier1,testFamily:testQualifier2')";
+
+    CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
+    assertEquals(0, responseTwo.getResponseCode());
+
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists(tableName);
+    assertTrue(doesTableExist);
+
+    populateHBaseTable(tableName, 5);
+
+    Configuration conf = new Configuration(hcatConf);
+    conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
+      HCatUtil.serialize(getHiveConf().getAllProperties()));
+
+    // output settings
+    Path outputDir = new Path(getTestDir(), "mapred/testHBaseTableProjectionReadMR");
+    FileSystem fs = getFileSystem();
+    if (fs.exists(outputDir)) {
+      fs.delete(outputDir, true);
+    }
+    // create job
+    JobConf job = new JobConf(conf);
+    job.setJobName("hbase-scan-column");
+    job.setJarByClass(this.getClass());
+    job.setMapperClass(MapReadProjectionHTable.class);
+    job.setInputFormat(HBaseInputFormat.class);
+
+    //Configure projection schema
+    job.set(HCatConstants.HCAT_KEY_OUTPUT_SCHEMA, HCatUtil.serialize(getProjectionSchema()));
+    Job newJob = new Job(job);
+    HCatInputFormat.setInput(newJob, MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+    String inputJobString = newJob.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO);
+    InputJobInfo info = (InputJobInfo) HCatUtil.deserialize(inputJobString);
+    job.set(HCatConstants.HCAT_KEY_JOB_INFO, inputJobString);
+    for (PartInfo partinfo : info.getPartitions()) {
+      for (Entry<String, String> entry : partinfo.getJobProperties().entrySet())
+        job.set(entry.getKey(), entry.getValue());
+    }
+    assertEquals("testFamily:testQualifier1", job.get(TableInputFormat.SCAN_COLUMNS));
+
+    job.setOutputFormat(org.apache.hadoop.mapred.TextOutputFormat.class);
+    org.apache.hadoop.mapred.TextOutputFormat.setOutputPath(job, outputDir);
+    job.setMapOutputKeyClass(BytesWritable.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setOutputKeyClass(BytesWritable.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(0);
+
+    RunningJob runJob = JobClient.runJob(job);
+    runJob.waitForCompletion();
+    assertTrue(runJob.isSuccessful());
+    assertFalse(MapReadProjHTable.error);
+    assertEquals(MapReadProjHTable.count, 1);
+
+    String dropTableQuery = "DROP TABLE " + tableName;
+    CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
+    assertEquals(0, responseThree.getResponseCode());
+
+    boolean isHbaseTableThere = hAdmin.tableExists(tableName);
+    assertFalse(isHbaseTableThere);
+  }
+
+  @Test
+  public void TestHBaseTableIgnoreAbortedTransactions() throws Exception {
+    String tableName = newTableName("mytable");
+    String tableQuery = "CREATE TABLE " + tableName
+      + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
+      "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+      + "TBLPROPERTIES ('hbase.columns.mapping'=':key," +
+      "testFamily:testQualifier1,testFamily:testQualifier2')";
+
+    CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
+    assertEquals(0, responseTwo.getResponseCode());
+
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists(tableName);
+    assertTrue(doesTableExist);
+
+    populateHBaseTable(tableName, 5);
+    populateHBaseTableQualifier1(tableName, 6, false);
+    populateHBaseTableQualifier1(tableName, 7, false);
+
+    Configuration conf = new Configuration(hcatConf);
+    conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
+      HCatUtil.serialize(getHiveConf().getAllProperties()));
+
+    Path outputDir = new Path(getTestDir(), "mapred/testHBaseTableIgnoreAbortedTransactions");
+    FileSystem fs = getFileSystem();
+    if (fs.exists(outputDir)) {
+      fs.delete(outputDir, true);
+    }
+    Job job = new Job(conf, "hbase-aborted-transaction");
+    job.setJarByClass(this.getClass());
+    job.setMapperClass(MapReadHTable.class);
+    MapReadHTable.resetCounters();
+    job.setInputFormatClass(HCatInputFormat.class);
+    HCatInputFormat.setInput(job, MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+    job.setOutputFormatClass(TextOutputFormat.class);
+    TextOutputFormat.setOutputPath(job, outputDir);
+    job.setMapOutputKeyClass(BytesWritable.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setOutputKeyClass(BytesWritable.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(0);
+    assertTrue(job.waitForCompletion(true));
+    // Verify that the records do not contain aborted transaction
+    // revisions 6 and 7 for testFamily:testQualifier1 and
+    // fetches revision 5 for both testQualifier1 and testQualifier2
+    assertFalse(MapReadHTable.error);
+    assertEquals(1, MapReadHTable.count);
+
+    String dropTableQuery = "DROP TABLE " + tableName;
+    CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
+    assertEquals(0, responseThree.getResponseCode());
+
+    boolean isHbaseTableThere = hAdmin.tableExists(tableName);
+    assertFalse(isHbaseTableThere);
+  }
+
+  @Test
+  public void TestHBaseTableIgnoreAbortedAndRunningTransactions() throws Exception {
+    String tableName = newTableName("mytable");
+    String tableQuery = "CREATE TABLE " + tableName
+      + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
+      "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+      + "TBLPROPERTIES ('hbase.columns.mapping'=':key," +
+      "testFamily:testQualifier1,testFamily:testQualifier2')";
+
+    CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
+    assertEquals(0, responseTwo.getResponseCode());
+
+    HBaseAdmin hAdmin = new HBaseAdmin(getHbaseConf());
+    boolean doesTableExist = hAdmin.tableExists(tableName);
+    assertTrue(doesTableExist);
+
+    populateHBaseTable(tableName, 2);
+    populateHBaseTableQualifier1(tableName, 3, Boolean.TRUE); //Committed transaction
+    populateHBaseTableQualifier1(tableName, 4, null); //Running transaction
+    populateHBaseTableQualifier1(tableName, 5, Boolean.FALSE);  //Aborted transaction
+    populateHBaseTableQualifier1(tableName, 6, Boolean.TRUE); //Committed transaction
+    populateHBaseTableQualifier1(tableName, 7, null); //Running Transaction
+    populateHBaseTableQualifier1(tableName, 8, Boolean.FALSE); //Aborted Transaction
+
+    Configuration conf = new Configuration(hcatConf);
+    conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
+      HCatUtil.serialize(getHiveConf().getAllProperties()));
+
+    Path outputDir = new Path(getTestDir(), "mapred/testHBaseTableIgnoreAbortedTransactions");
+    FileSystem fs = getFileSystem();
+    if (fs.exists(outputDir)) {
+      fs.delete(outputDir, true);
+    }
+    Job job = new Job(conf, "hbase-running-aborted-transaction");
+    job.setJarByClass(this.getClass());
+    job.setMapperClass(MapReadHTableRunningAbort.class);
+    job.setInputFormatClass(HCatInputFormat.class);
+    HCatInputFormat.setInput(job, MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+    job.setOutputFormatClass(TextOutputFormat.class);
+    TextOutputFormat.setOutputPath(job, outputDir);
+    job.setMapOutputKeyClass(BytesWritable.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setOutputKeyClass(BytesWritable.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(0);
+    assertTrue(job.waitForCompletion(true));
+    // Verify that the records do not contain running and aborted transaction
+    // and it fetches revision 2 for testQualifier1 and testQualifier2
+    assertFalse(MapReadHTableRunningAbort.error);
+    assertEquals(1, MapReadHTableRunningAbort.count);
+
+    String dropTableQuery = "DROP TABLE " + tableName;
+    CommandProcessorResponse responseThree = hcatDriver.run(dropTableQuery);
+    assertEquals(0, responseThree.getResponseCode());
+
+    boolean isHbaseTableThere = hAdmin.tableExists(tableName);
+    assertFalse(isHbaseTableThere);
+  }
+
+
+  static class MapReadHTable
+    extends
+    Mapper<ImmutableBytesWritable, HCatRecord, WritableComparable<?>, Text> {
+
+    static boolean error = false;
+    static int count = 0;
+
+    @Override
+    public void map(ImmutableBytesWritable key, HCatRecord value,
+            Context context) throws IOException, InterruptedException {
+      System.out.println("HCat record value" + value.toString());
+      boolean correctValues = (value.size() == 3)
+        && (value.get(0).toString()).equalsIgnoreCase("testRow")
+        && (value.get(1).toString()).equalsIgnoreCase("textValue-5")
+        && (value.get(2).toString()).equalsIgnoreCase("textValue-5");
+
+      if (correctValues == false) {
+        error = true;
+      }
+      count++;
+    }
+
+    public static void resetCounters() {
+      error = false;
+      count = 0;
+    }
+  }
+
+  static class MapReadProjHTable
+    extends
+    Mapper<ImmutableBytesWritable, HCatRecord, WritableComparable<?>, Text> {
+
+    static boolean error = false;
+    static int count = 0;
+
+    @Override
+    public void map(ImmutableBytesWritable key, HCatRecord value,
+            Context context) throws IOException, InterruptedException {
+      System.out.println("HCat record value" + value.toString());
+      boolean correctValues = (value.size() == 2)
+        && (value.get(0).toString()).equalsIgnoreCase("testRow")
+        && (value.get(1).toString()).equalsIgnoreCase("textValue-5");
+
+      if (correctValues == false) {
+        error = true;
+      }
+      count++;
+    }
+  }
+
+  static class MapReadProjectionHTable
+    implements org.apache.hadoop.mapred.Mapper<ImmutableBytesWritable, Result, WritableComparable<?>, Text> {
+
+    static boolean error = false;
+    static int count = 0;
+
+    @Override
+    public void configure(JobConf job) {
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public void map(ImmutableBytesWritable key, Result result,
+            OutputCollector<WritableComparable<?>, Text> output, Reporter reporter)
+      throws IOException {
+      System.out.println("Result " + result.toString());
+      List<KeyValue> list = result.list();
+      boolean correctValues = (list.size() == 1)
+        && (Bytes.toString(list.get(0).getRow())).equalsIgnoreCase("testRow")
+        && (Bytes.toString(list.get(0).getValue())).equalsIgnoreCase("textValue-5")
+        && (Bytes.toString(list.get(0).getFamily())).equalsIgnoreCase("testFamily")
+        && (Bytes.toString(list.get(0).getQualifier())).equalsIgnoreCase("testQualifier1");
+
+      if (correctValues == false) {
+        error = true;
+      }
+      count++;
+    }
+  }
+
+  static class MapReadHTableRunningAbort
+    extends
+    Mapper<ImmutableBytesWritable, HCatRecord, WritableComparable<?>, Text> {
+
+    static boolean error = false;
+    static int count = 0;
+
+    @Override
+    public void map(ImmutableBytesWritable key, HCatRecord value,
+            Context context) throws IOException, InterruptedException {
+      System.out.println("HCat record value" + value.toString());
+      boolean correctValues = (value.size() == 3)
+        && (value.get(0).toString()).equalsIgnoreCase("testRow")
+        && (value.get(1).toString()).equalsIgnoreCase("textValue-3")
+        && (value.get(2).toString()).equalsIgnoreCase("textValue-2");
+
+      if (correctValues == false) {
+        error = true;
+      }
+      count++;
+    }
+  }
+
+  private HCatSchema getProjectionSchema() throws HCatException {
+
+    HCatSchema schema = new HCatSchema(new ArrayList<HCatFieldSchema>());
+    schema.append(new HCatFieldSchema("key", HCatFieldSchema.Type.STRING,
+      ""));
+    schema.append(new HCatFieldSchema("testqualifier1",
+      HCatFieldSchema.Type.STRING, ""));
+    return schema;
+  }
 
 
 }

Modified: hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestSnapshots.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestSnapshots.java?rev=1522098&r1=1522097&r2=1522098&view=diff
==============================================================================
--- hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestSnapshots.java (original)
+++ hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestSnapshots.java Thu Sep 12 01:21:10 2013
@@ -34,108 +34,108 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hcatalog.cli.HCatDriver;
-import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
-import org.apache.hcatalog.common.HCatConstants;
-import org.apache.hcatalog.common.HCatUtil;
+import org.apache.hive.hcatalog.cli.HCatDriver;
+import org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
+import org.apache.hive.hcatalog.common.HCatConstants;
+import org.apache.hive.hcatalog.common.HCatUtil;
 import org.apache.hcatalog.hbase.snapshot.TableSnapshot;
-import org.apache.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hcatalog.mapreduce.InputJobInfo;
+import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.junit.Test;
 
 public class TestSnapshots extends SkeletonHBaseTest {
-    private static HiveConf hcatConf;
-    private static HCatDriver hcatDriver;
+  private static HiveConf hcatConf;
+  private static HCatDriver hcatDriver;
 
-    public void Initialize() throws Exception {
-        hcatConf = getHiveConf();
-        hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
-            HCatSemanticAnalyzer.class.getName());
-        URI fsuri = getFileSystem().getUri();
-        Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(),
-            getTestDir());
-        hcatConf.set(HiveConf.ConfVars.HADOOPFS.varname, fsuri.toString());
-        hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString());
-
-        //Add hbase properties
-
-        for (Map.Entry<String, String> el : getHbaseConf()) {
-            if (el.getKey().startsWith("hbase.")) {
-                hcatConf.set(el.getKey(), el.getValue());
-            }
-        }
+  public void Initialize() throws Exception {
+    hcatConf = getHiveConf();
+    hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
+      HCatSemanticAnalyzer.class.getName());
+    URI fsuri = getFileSystem().getUri();
+    Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(),
+      getTestDir());
+    hcatConf.set(HiveConf.ConfVars.HADOOPFS.varname, fsuri.toString());
+    hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString());
+
+    //Add hbase properties
+
+    for (Map.Entry<String, String> el : getHbaseConf()) {
+      if (el.getKey().startsWith("hbase.")) {
+        hcatConf.set(el.getKey(), el.getValue());
+      }
+    }
 
-        SessionState.start(new CliSessionState(hcatConf));
-        hcatDriver = new HCatDriver();
+    SessionState.start(new CliSessionState(hcatConf));
+    hcatDriver = new HCatDriver();
 
-    }
+  }
 
-    @Test
-    public void TestSnapshotConversion() throws Exception {
-        Initialize();
-        String tableName = newTableName("mytableOne");
-        String databaseName = newTableName("mydatabase");
-        String fullyQualTableName = databaseName + "." + tableName;
-        String db_dir = getTestDir() + "/hbasedb";
-        String dbquery = "CREATE DATABASE IF NOT EXISTS " + databaseName + " LOCATION '"
-            + db_dir + "'";
-        String tableQuery = "CREATE TABLE " + fullyQualTableName
-            + "(key string, value1 string, value2 string) STORED BY " +
-            "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-            + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:q1,cf2:q2')";
-
-        CommandProcessorResponse cmdResponse = hcatDriver.run(dbquery);
-        assertEquals(0, cmdResponse.getResponseCode());
-        cmdResponse = hcatDriver.run(tableQuery);
-        assertEquals(0, cmdResponse.getResponseCode());
-
-        Configuration conf = new Configuration(hcatConf);
-        conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
-            HCatUtil.serialize(getHiveConf().getAllProperties()));
-        Job job = new Job(conf);
-        Properties properties = new Properties();
-        properties.setProperty(HBaseConstants.PROPERTY_TABLE_SNAPSHOT_KEY, "dummysnapshot");
-        HCatInputFormat.setInput(job, databaseName, tableName).setProperties(properties);
-        String modifiedInputInfo = job.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO);
-        InputJobInfo inputInfo = (InputJobInfo) HCatUtil.deserialize(modifiedInputInfo);
-
-        Map<String, Long> revMap = new HashMap<String, Long>();
-        revMap.put("cf1", 3L);
-        revMap.put("cf2", 5L);
-        TableSnapshot hbaseSnapshot = new TableSnapshot(fullyQualTableName, revMap, -1);
-        HCatTableSnapshot hcatSnapshot = HBaseRevisionManagerUtil.convertSnapshot(hbaseSnapshot, inputInfo.getTableInfo());
-
-        assertEquals(hcatSnapshot.getRevision("value1"), 3);
-        assertEquals(hcatSnapshot.getRevision("value2"), 5);
-
-        String dropTable = "DROP TABLE " + fullyQualTableName;
-        cmdResponse = hcatDriver.run(dropTable);
-        assertEquals(0, cmdResponse.getResponseCode());
-
-        tableName = newTableName("mytableTwo");
-        fullyQualTableName = databaseName + "." + tableName;
-        tableQuery = "CREATE TABLE " + fullyQualTableName
-            + "(key string, value1 string, value2 string) STORED BY " +
-            "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-            + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:q1,cf1:q2')";
-        cmdResponse = hcatDriver.run(tableQuery);
-        assertEquals(0, cmdResponse.getResponseCode());
-        revMap.clear();
-        revMap.put("cf1", 3L);
-        hbaseSnapshot = new TableSnapshot(fullyQualTableName, revMap, -1);
-        HCatInputFormat.setInput(job, databaseName, tableName).setProperties(properties);
-        modifiedInputInfo = job.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO);
-        inputInfo = (InputJobInfo) HCatUtil.deserialize(modifiedInputInfo);
-        hcatSnapshot = HBaseRevisionManagerUtil.convertSnapshot(hbaseSnapshot, inputInfo.getTableInfo());
-        assertEquals(hcatSnapshot.getRevision("value1"), 3);
-        assertEquals(hcatSnapshot.getRevision("value2"), 3);
-
-        dropTable = "DROP TABLE " + fullyQualTableName;
-        cmdResponse = hcatDriver.run(dropTable);
-        assertEquals(0, cmdResponse.getResponseCode());
-
-        String dropDatabase = "DROP DATABASE IF EXISTS " + databaseName + "CASCADE";
-        cmdResponse = hcatDriver.run(dropDatabase);
-        assertEquals(0, cmdResponse.getResponseCode());
-    }
+  @Test
+  public void TestSnapshotConversion() throws Exception {
+    Initialize();
+    String tableName = newTableName("mytableOne");
+    String databaseName = newTableName("mydatabase");
+    String fullyQualTableName = databaseName + "." + tableName;
+    String db_dir = new Path(getTestDir(), "hbasedb").toString();
+    String dbquery = "CREATE DATABASE IF NOT EXISTS " + databaseName + " LOCATION '"
+      + db_dir + "'";
+    String tableQuery = "CREATE TABLE " + fullyQualTableName
+      + "(key string, value1 string, value2 string) STORED BY " +
+      "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+      + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:q1,cf2:q2')";
+
+    CommandProcessorResponse cmdResponse = hcatDriver.run(dbquery);
+    assertEquals(0, cmdResponse.getResponseCode());
+    cmdResponse = hcatDriver.run(tableQuery);
+    assertEquals(0, cmdResponse.getResponseCode());
+
+    Configuration conf = new Configuration(hcatConf);
+    conf.set(HCatConstants.HCAT_KEY_HIVE_CONF,
+      HCatUtil.serialize(getHiveConf().getAllProperties()));
+    Job job = new Job(conf);
+    Properties properties = new Properties();
+    properties.setProperty(HBaseConstants.PROPERTY_TABLE_SNAPSHOT_KEY, "dummysnapshot");
+    HCatInputFormat.setInput(job, databaseName, tableName).setProperties(properties);
+    String modifiedInputInfo = job.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO);
+    InputJobInfo inputInfo = (InputJobInfo) HCatUtil.deserialize(modifiedInputInfo);
+
+    Map<String, Long> revMap = new HashMap<String, Long>();
+    revMap.put("cf1", 3L);
+    revMap.put("cf2", 5L);
+    TableSnapshot hbaseSnapshot = new TableSnapshot(fullyQualTableName, revMap, -1);
+    HCatTableSnapshot hcatSnapshot = HBaseRevisionManagerUtil.convertSnapshot(hbaseSnapshot, inputInfo.getTableInfo());
+
+    assertEquals(hcatSnapshot.getRevision("value1"), 3);
+    assertEquals(hcatSnapshot.getRevision("value2"), 5);
+
+    String dropTable = "DROP TABLE " + fullyQualTableName;
+    cmdResponse = hcatDriver.run(dropTable);
+    assertEquals(0, cmdResponse.getResponseCode());
+
+    tableName = newTableName("mytableTwo");
+    fullyQualTableName = databaseName + "." + tableName;
+    tableQuery = "CREATE TABLE " + fullyQualTableName
+      + "(key string, value1 string, value2 string) STORED BY " +
+      "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
+      + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:q1,cf1:q2')";
+    cmdResponse = hcatDriver.run(tableQuery);
+    assertEquals(0, cmdResponse.getResponseCode());
+    revMap.clear();
+    revMap.put("cf1", 3L);
+    hbaseSnapshot = new TableSnapshot(fullyQualTableName, revMap, -1);
+    HCatInputFormat.setInput(job, databaseName, tableName).setProperties(properties);
+    modifiedInputInfo = job.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO);
+    inputInfo = (InputJobInfo) HCatUtil.deserialize(modifiedInputInfo);
+    hcatSnapshot = HBaseRevisionManagerUtil.convertSnapshot(hbaseSnapshot, inputInfo.getTableInfo());
+    assertEquals(hcatSnapshot.getRevision("value1"), 3);
+    assertEquals(hcatSnapshot.getRevision("value2"), 3);
+
+    dropTable = "DROP TABLE " + fullyQualTableName;
+    cmdResponse = hcatDriver.run(dropTable);
+    assertEquals(0, cmdResponse.getResponseCode());
+
+    String dropDatabase = "DROP DATABASE IF EXISTS " + databaseName + "CASCADE";
+    cmdResponse = hcatDriver.run(dropDatabase);
+    assertEquals(0, cmdResponse.getResponseCode());
+  }
 }

Modified: hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/IDGenClient.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/IDGenClient.java?rev=1522098&r1=1522097&r2=1522098&view=diff
==============================================================================
--- hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/IDGenClient.java (original)
+++ hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/IDGenClient.java Thu Sep 12 01:21:10 2013
@@ -24,49 +24,49 @@ import java.util.Random;
 
 public class IDGenClient extends Thread {
 
-    String connectionStr;
-    String base_dir;
-    ZKUtil zkutil;
-    Random sleepTime = new Random();
-    int runtime;
-    HashMap<Long, Long> idMap;
-    String tableName;
-
-    IDGenClient(String connectionStr, String base_dir, int time, String tableName) {
-        super();
-        this.connectionStr = connectionStr;
-        this.base_dir = base_dir;
-        this.zkutil = new ZKUtil(connectionStr, base_dir);
-        this.runtime = time;
-        idMap = new HashMap<Long, Long>();
-        this.tableName = tableName;
-    }
-
-    /*
-     * @see java.lang.Runnable#run()
-     */
-    @Override
-    public void run() {
-        long startTime = System.currentTimeMillis();
-        int timeElapsed = 0;
-        while( timeElapsed <= runtime){
-            try {
-                long id = zkutil.nextId(tableName);
-                idMap.put(System.currentTimeMillis(), id);
-
-                int sTime = sleepTime.nextInt(2);
-                Thread.sleep(sTime * 100);
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-
-            timeElapsed = (int) Math.ceil((System.currentTimeMillis() - startTime)/(double)1000);
-        }
+  String connectionStr;
+  String base_dir;
+  ZKUtil zkutil;
+  Random sleepTime = new Random();
+  int runtime;
+  HashMap<Long, Long> idMap;
+  String tableName;
+
+  IDGenClient(String connectionStr, String base_dir, int time, String tableName) {
+    super();
+    this.connectionStr = connectionStr;
+    this.base_dir = base_dir;
+    this.zkutil = new ZKUtil(connectionStr, base_dir);
+    this.runtime = time;
+    idMap = new HashMap<Long, Long>();
+    this.tableName = tableName;
+  }
+
+  /*
+   * @see java.lang.Runnable#run()
+   */
+  @Override
+  public void run() {
+    long startTime = System.currentTimeMillis();
+    int timeElapsed = 0;
+    while( timeElapsed <= runtime){
+      try {
+        long id = zkutil.nextId(tableName);
+        idMap.put(System.currentTimeMillis(), id);
+
+        int sTime = sleepTime.nextInt(2);
+        Thread.sleep(sTime * 100);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
 
+      timeElapsed = (int) Math.ceil((System.currentTimeMillis() - startTime)/(double)1000);
     }
 
-    Map<Long, Long> getIdMap(){
-        return idMap;
-    }
+  }
+
+  Map<Long, Long> getIdMap(){
+    return idMap;
+  }
 
 }

Modified: hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/TestIDGenerator.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/TestIDGenerator.java?rev=1522098&r1=1522097&r2=1522098&view=diff
==============================================================================
--- hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/TestIDGenerator.java (original)
+++ hive/branches/vectorization/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/TestIDGenerator.java Thu Sep 12 01:21:10 2013
@@ -30,70 +30,70 @@ import org.junit.Test;
 
 public class TestIDGenerator extends SkeletonHBaseTest {
 
-    @Test
-    public void testIDGeneration() throws Exception {
+  @Test
+  public void testIDGeneration() throws Exception {
 
-        int port = getHbaseConf().getInt("hbase.zookeeper.property.clientPort", 2181);
-        String servers = getHbaseConf().get("hbase.zookeeper.quorum");
-        String[] splits = servers.split(",");
-        StringBuffer sb = new StringBuffer();
-        for (String split : splits) {
-            sb.append(split);
-            sb.append(':');
-            sb.append(port);
-        }
-        ZKUtil zkutil = new ZKUtil(sb.toString(), "/rm_base");
-
-        String tableName = "myTable";
-        long initId = zkutil.nextId(tableName);
-        for (int i = 0; i < 10; i++) {
-            long id = zkutil.nextId(tableName);
-            Assert.assertEquals(initId + (i + 1), id);
-        }
-    }
-
-    @Test
-    public void testMultipleClients() throws InterruptedException {
-
-        int port = getHbaseConf().getInt("hbase.zookeeper.property.clientPort", 2181);
-        String servers = getHbaseConf().get("hbase.zookeeper.quorum");
-        String[] splits = servers.split(",");
-        StringBuffer sb = new StringBuffer();
-        for (String split : splits) {
-            sb.append(split);
-            sb.append(':');
-            sb.append(port);
-        }
-
-        ArrayList<IDGenClient> clients = new ArrayList<IDGenClient>();
-
-        for (int i = 0; i < 5; i++) {
-            IDGenClient idClient = new IDGenClient(sb.toString(), "/rm_base", 10, "testTable");
-            clients.add(idClient);
-        }
-
-        for (IDGenClient idClient : clients) {
-            idClient.run();
-        }
-
-        for (IDGenClient idClient : clients) {
-            idClient.join();
-        }
-
-        HashMap<Long, Long> idMap = new HashMap<Long, Long>();
-        for (IDGenClient idClient : clients) {
-            idMap.putAll(idClient.getIdMap());
-        }
-
-        ArrayList<Long> keys = new ArrayList<Long>(idMap.keySet());
-        Collections.sort(keys);
-        int startId = 1;
-        for (Long key : keys) {
-            Long id = idMap.get(key);
-            System.out.println("Key: " + key + " Value " + id);
-            assertTrue(id == startId);
-            startId++;
+    int port = getHbaseConf().getInt("hbase.zookeeper.property.clientPort", 2181);
+    String servers = getHbaseConf().get("hbase.zookeeper.quorum");
+    String[] splits = servers.split(",");
+    StringBuffer sb = new StringBuffer();
+    for (String split : splits) {
+      sb.append(split);
+      sb.append(':');
+      sb.append(port);
+    }
+    ZKUtil zkutil = new ZKUtil(sb.toString(), "/rm_base");
+
+    String tableName = "myTable";
+    long initId = zkutil.nextId(tableName);
+    for (int i = 0; i < 10; i++) {
+      long id = zkutil.nextId(tableName);
+      Assert.assertEquals(initId + (i + 1), id);
+    }
+  }
+
+  @Test
+  public void testMultipleClients() throws InterruptedException {
+
+    int port = getHbaseConf().getInt("hbase.zookeeper.property.clientPort", 2181);
+    String servers = getHbaseConf().get("hbase.zookeeper.quorum");
+    String[] splits = servers.split(",");
+    StringBuffer sb = new StringBuffer();
+    for (String split : splits) {
+      sb.append(split);
+      sb.append(':');
+      sb.append(port);
+    }
+
+    ArrayList<IDGenClient> clients = new ArrayList<IDGenClient>();
+
+    for (int i = 0; i < 5; i++) {
+      IDGenClient idClient = new IDGenClient(sb.toString(), "/rm_base", 10, "testTable");
+      clients.add(idClient);
+    }
+
+    for (IDGenClient idClient : clients) {
+      idClient.run();
+    }
+
+    for (IDGenClient idClient : clients) {
+      idClient.join();
+    }
+
+    HashMap<Long, Long> idMap = new HashMap<Long, Long>();
+    for (IDGenClient idClient : clients) {
+      idMap.putAll(idClient.getIdMap());
+    }
+
+    ArrayList<Long> keys = new ArrayList<Long>(idMap.keySet());
+    Collections.sort(keys);
+    int startId = 1;
+    for (Long key : keys) {
+      Long id = idMap.get(key);
+      System.out.println("Key: " + key + " Value " + id);
+      assertTrue(id == startId);
+      startId++;
 
-        }
     }
+  }
 }