You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hcatalog-commits@incubator.apache.org by to...@apache.org on 2011/11/24 01:34:46 UTC

svn commit: r1205693 - in /incubator/hcatalog/trunk: ./ src/java/org/apache/hcatalog/cli/SemanticAnalysis/ src/test/org/apache/hcatalog/cli/ storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/

Author: toffer
Date: Thu Nov 24 01:34:45 2011
New Revision: 1205693

URL: http://svn.apache.org/viewvc?rev=1205693&view=rev
Log:
HCATALOG-160 : Creating a table with HCatStorageHandler does not correctly store the hcat.isd and hcat.osd names (avandana via toffer)

Added:
    incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/DummyStorageHandler.java
    incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestStorageHandlerProperties.java
Modified:
    incubator/hcatalog/trunk/CHANGES.txt
    incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateTableHook.java
    incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseBulkOutputStorageDriver.java
    incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseDirectOutputStorageDriver.java
    incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java
    incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputStorageDriver.java

Modified: incubator/hcatalog/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/CHANGES.txt?rev=1205693&r1=1205692&r2=1205693&view=diff
==============================================================================
--- incubator/hcatalog/trunk/CHANGES.txt (original)
+++ incubator/hcatalog/trunk/CHANGES.txt Thu Nov 24 01:34:45 2011
@@ -79,6 +79,8 @@ Trunk (unreleased changes)
   OPTIMIZATIONS
 
   BUG FIXES
+  HCAT-169. Creating a table with HCatStorageHandler does not correctly store the hcat.isd and hcat.osd names (avandana via toffer)
+
   HCAT-160. HBaseDirectOutputStorageDriver outputVersion isn't consitent within the same MR job (toffer via hashutosh)
 
   HCAT-155. HBase bulkOSD requires value to be Put rather than HCatRecord (toffer via hashutosh) 

Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateTableHook.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateTableHook.java?rev=1205693&r1=1205692&r2=1205693&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateTableHook.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateTableHook.java Thu Nov 24 01:34:45 2011
@@ -249,24 +249,22 @@ final class CreateTableHook extends Abst
                         .getAuthorizationProvider();
 
                 // TBD: To pass in the exact read and write privileges.
-                auth.authorize(context.getHive().getTable(tableName), null,
-                        null);
+                String databaseName = context.getHive().newTable(desc.getTableName()).getDbName();
+                auth.authorize(context.getHive().getDatabase(databaseName), null, null);
 
                 tblProps.put(HCatConstants.HCAT_ISD_CLASS, storageHandlerInst
-                        .getInputStorageDriver().toString());
+                        .getInputStorageDriver().getName());
                 tblProps.put(HCatConstants.HCAT_OSD_CLASS, storageHandlerInst
-                        .getOutputStorageDriver().toString());
+                        .getOutputStorageDriver().getName());
 
             } catch (HiveException e) {
-                new SemanticException(e);
+                throw new SemanticException(e);
             }
 
         }
-        
         if (loader!=null) {
             tblProps.put(HCatConstants.HCAT_PIG_LOADER, loader);
         }
-        
         if (storer!=null) {
             tblProps.put(HCatConstants.HCAT_PIG_STORER, storer);
         }

Added: incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/DummyStorageHandler.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/DummyStorageHandler.java?rev=1205693&view=auto
==============================================================================
--- incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/DummyStorageHandler.java (added)
+++ incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/DummyStorageHandler.java Thu Nov 24 01:34:45 2011
@@ -0,0 +1,215 @@
+/*
+ * 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.hcatalog.cli;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe;
+import org.apache.hcatalog.mapreduce.HCatInputStorageDriver;
+import org.apache.hcatalog.mapreduce.HCatOutputStorageDriver;
+import org.apache.hcatalog.storagehandler.HCatStorageHandler;
+
+class DummyStorageHandler extends HCatStorageHandler {
+
+    @Override
+    public Configuration getConf() {
+        return null;
+    }
+
+    @Override
+    public void setConf(Configuration conf) {
+    }
+
+    @Override
+    public void configureTableJobProperties(TableDesc arg0,
+            Map<String, String> arg1) {
+    }
+
+    @Override
+    public HiveMetaHook getMetaHook() {
+        return this;
+    }
+
+    @Override
+    public Class<? extends SerDe> getSerDeClass() {
+        return ColumnarSerDe.class;
+    }
+
+    @Override
+    public void preCreateTable(Table table) throws MetaException {
+    }
+
+    @Override
+    public void rollbackCreateTable(Table table) throws MetaException {
+    }
+
+    @Override
+    public void commitCreateTable(Table table) throws MetaException {
+    }
+
+    @Override
+    public void preDropTable(Table table) throws MetaException {
+    }
+
+    @Override
+    public void rollbackDropTable(Table table) throws MetaException {
+
+    }
+
+    @Override
+    public void commitDropTable(Table table, boolean deleteData)
+            throws MetaException {
+    }
+
+    @Override
+    public Class<? extends HCatInputStorageDriver> getInputStorageDriver() {
+        return HCatInputStorageDriver.class;
+    }
+
+    @Override
+    public Class<? extends HCatOutputStorageDriver> getOutputStorageDriver() {
+        return HCatOutputStorageDriver.class;
+    }
+
+    @Override
+    public HiveAuthorizationProvider getAuthorizationProvider()
+            throws HiveException {
+        return new DummyAuthProvider();
+    }
+
+    private class DummyAuthProvider implements HiveAuthorizationProvider {
+
+        @Override
+        public Configuration getConf() {
+            return null;
+        }
+
+        /* @param conf
+         * @see org.apache.hadoop.conf.Configurable#setConf(org.apache.hadoop.conf.Configuration)
+         */
+        @Override
+        public void setConf(Configuration conf) {
+        }
+
+        /* @param conf
+        /* @throws HiveException
+         * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#init(org.apache.hadoop.conf.Configuration)
+         */
+        @Override
+        public void init(Configuration conf) throws HiveException {
+        }
+
+        /* @return HiveAuthenticationProvider
+         * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#getAuthenticator()
+         */
+        @Override
+        public HiveAuthenticationProvider getAuthenticator() {
+            return null;
+        }
+
+        /* @param authenticator
+         * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#setAuthenticator(org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider)
+         */
+        @Override
+        public void setAuthenticator(HiveAuthenticationProvider authenticator) {
+        }
+
+        /* @param readRequiredPriv
+        /* @param writeRequiredPriv
+        /* @throws HiveException
+        /* @throws AuthorizationException
+         * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[])
+         */
+        @Override
+        public void authorize(Privilege[] readRequiredPriv,
+                Privilege[] writeRequiredPriv) throws HiveException,
+                AuthorizationException {
+        }
+
+        /* @param db
+        /* @param readRequiredPriv
+        /* @param writeRequiredPriv
+        /* @throws HiveException
+        /* @throws AuthorizationException
+         * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.metastore.api.Database, org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[])
+         */
+        @Override
+        public void authorize(Database db, Privilege[] readRequiredPriv,
+                Privilege[] writeRequiredPriv) throws HiveException,
+                AuthorizationException {
+        }
+
+        /* @param table
+        /* @param readRequiredPriv
+        /* @param writeRequiredPriv
+        /* @throws HiveException
+        /* @throws AuthorizationException
+         * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.ql.metadata.Table, org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[])
+         */
+        @Override
+        public void authorize(org.apache.hadoop.hive.ql.metadata.Table table, Privilege[] readRequiredPriv,
+                Privilege[] writeRequiredPriv) throws HiveException,
+                AuthorizationException {
+        }
+
+        /* @param part
+        /* @param readRequiredPriv
+        /* @param writeRequiredPriv
+        /* @throws HiveException
+        /* @throws AuthorizationException
+         * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.ql.metadata.Partition, org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[])
+         */
+        @Override
+        public void authorize(Partition part, Privilege[] readRequiredPriv,
+                Privilege[] writeRequiredPriv) throws HiveException,
+                AuthorizationException {
+        }
+
+        /* @param table
+        /* @param part
+        /* @param columns
+        /* @param readRequiredPriv
+        /* @param writeRequiredPriv
+        /* @throws HiveException
+        /* @throws AuthorizationException
+         * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.ql.metadata.Table, org.apache.hadoop.hive.ql.metadata.Partition, java.util.List, org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[])
+         */
+        @Override
+        public void authorize(org.apache.hadoop.hive.ql.metadata.Table table, Partition part, List<String> columns,
+                Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+                throws HiveException, AuthorizationException {
+        }
+
+    }
+
+}
+
+

Added: incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestStorageHandlerProperties.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestStorageHandlerProperties.java?rev=1205693&view=auto
==============================================================================
--- incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestStorageHandlerProperties.java (added)
+++ incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestStorageHandlerProperties.java Thu Nov 24 01:34:45 2011
@@ -0,0 +1,84 @@
+/*
+ * 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.hcatalog.cli;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
+import org.apache.hcatalog.common.HCatConstants;
+import org.apache.thrift.TException;
+
+import junit.framework.TestCase;
+
+public class TestStorageHandlerProperties extends TestCase {
+
+    private Driver hcatDriver;
+    private Driver hiveDriver;
+    private HiveMetaStoreClient msc;
+
+    protected void setUp() throws Exception {
+        HiveConf hcatConf = new HiveConf(this.getClass());
+        hcatConf.set(ConfVars.PREEXECHOOKS.varname, "");
+        hcatConf.set(ConfVars.POSTEXECHOOKS.varname, "");
+        hcatConf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+
+        HiveConf hiveConf = new HiveConf(hcatConf,this.getClass());
+        hiveDriver = new Driver(hiveConf);
+
+        hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName());
+        hcatDriver = new Driver(hcatConf);
+
+        msc = new HiveMetaStoreClient(hcatConf);
+        SessionState.start(new CliSessionState(hcatConf));
+    }
+
+    public void testTableProperties() throws CommandNeedRetryException, MetaException ,TException, NoSuchObjectException{
+        hcatDriver.run("drop table test_table");
+        CommandProcessorResponse response = hcatDriver
+                .run("create table test_table(key int, value string) STORED BY " +
+                     "'org.apache.hcatalog.cli.DummyStorageHandler' ");
+
+        assertEquals(0, response.getResponseCode());
+        Table tbl = msc.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "test_table");
+        DummyStorageHandler dsh = new DummyStorageHandler();
+        assertTrue(tbl.getParameters().containsKey(HCatConstants.HCAT_ISD_CLASS));
+        assertTrue(tbl.getParameters().containsKey(HCatConstants.HCAT_OSD_CLASS));
+        assertEquals(tbl.getParameters().get(HCatConstants.HCAT_ISD_CLASS), dsh.getInputStorageDriver().getName());
+        assertEquals(tbl.getParameters().get(HCatConstants.HCAT_OSD_CLASS), dsh.getOutputStorageDriver().getName());
+    }
+
+    /* @throws java.lang.Exception
+     * @see junit.framework.TestCase#tearDown()
+     */
+    protected void tearDown() throws Exception {
+        super.tearDown();
+    }
+
+}

Modified: incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseBulkOutputStorageDriver.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseBulkOutputStorageDriver.java?rev=1205693&r1=1205692&r2=1205693&view=diff
==============================================================================
--- incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseBulkOutputStorageDriver.java (original)
+++ incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseBulkOutputStorageDriver.java Thu Nov 24 01:34:45 2011
@@ -270,9 +270,7 @@ public class TestHBaseBulkOutputStorageD
         String tableQuery = "CREATE TABLE " + databaseName + "." + tableName +
                               "(key int, english string, spanish string) STORED BY " +
                               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'" +
-                              "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                              "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                              "'hbase.columns.mapping'=':key,"+familyName+":english,"+familyName+":spanish')" ;
+                              "TBLPROPERTIES ('hbase.columns.mapping'=':key,"+familyName+":english,"+familyName+":spanish')" ;
 
         assertEquals(0, hcatDriver.run(dbquery).getResponseCode());
         assertEquals(0, hcatDriver.run(tableQuery).getResponseCode());
@@ -375,9 +373,7 @@ public class TestHBaseBulkOutputStorageD
         String tableQuery = "CREATE TABLE " + databaseName + "." + tableName +
                               "(key int, english string, spanish string) STORED BY " +
                               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'" +
-                              "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                              "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                              "'hbase.columns.mapping'=':key,"+familyName+":english,"+familyName+":spanish')" ;
+                              "TBLPROPERTIES ('hbase.columns.mapping'=':key,"+familyName+":english,"+familyName+":spanish')" ;
 
         assertEquals(0, hcatDriver.run(dbquery).getResponseCode());
         assertEquals(0, hcatDriver.run(tableQuery).getResponseCode());
@@ -463,9 +459,7 @@ public class TestHBaseBulkOutputStorageD
         String tableQuery = "CREATE TABLE " + databaseName + "." + tableName +
                               "(key int, english string, spanish string) STORED BY " +
                               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'" +
-                              "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                              "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                              "'hbase.columns.mapping'=':key,"+familyName+":english,"+familyName+":spanish')" ;
+                              "TBLPROPERTIES ('hbase.columns.mapping'=':key,"+familyName+":english,"+familyName+":spanish')" ;
 
         assertEquals(0, hcatDriver.run(dbquery).getResponseCode());
         assertEquals(0, hcatDriver.run(tableQuery).getResponseCode());

Modified: incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseDirectOutputStorageDriver.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseDirectOutputStorageDriver.java?rev=1205693&r1=1205692&r2=1205693&view=diff
==============================================================================
--- incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseDirectOutputStorageDriver.java (original)
+++ incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseDirectOutputStorageDriver.java Thu Nov 24 01:34:45 2011
@@ -166,9 +166,7 @@ public class TestHBaseDirectOutputStorag
         String tableQuery = "CREATE TABLE " + databaseName + "." + tableName +
                               "(key int, english string, spanish string) STORED BY " +
                               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'" +
-                              "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                              "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                              "'"+HBaseConstants.PROPERTY_OSD_BULK_MODE_KEY+"'='false',"+
+                              "TBLPROPERTIES ('"+HBaseConstants.PROPERTY_OSD_BULK_MODE_KEY+"'='false',"+
                               "'hbase.columns.mapping'=':key,"+familyName+":english,"+familyName+":spanish')" ;
 
         assertEquals(0, hcatDriver.run(dbquery).getResponseCode());
@@ -264,9 +262,7 @@ public class TestHBaseDirectOutputStorag
         String tableQuery = "CREATE TABLE " + databaseName + "." + tableName +
                               "(key int, english string, spanish string) STORED BY " +
                               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'" +
-                              "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                              "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                              "'"+HBaseConstants.PROPERTY_OSD_BULK_MODE_KEY+"'='false',"+
+                              "TBLPROPERTIES ('"+HBaseConstants.PROPERTY_OSD_BULK_MODE_KEY+"'='false',"+
                               "'hbase.columns.mapping'=':key,"+familyName+":english,"+familyName+":spanish')" ;
 
         assertEquals(0, hcatDriver.run(dbquery).getResponseCode());

Modified: incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java?rev=1205693&r1=1205692&r2=1205693&view=diff
==============================================================================
--- incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java (original)
+++ incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseHCatStorageHandler.java Thu Nov 24 01:34:45 2011
@@ -76,9 +76,7 @@ public class TestHBaseHCatStorageHandler
         CommandProcessorResponse response = hcatDriver
                 .run("create table test_table(key int, value string) STORED BY " +
                 		     "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                    + "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                    		"'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                    		"'hbase.columns.mapping'=':key,cf1:val')");
+                    + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val')");
 
         assertEquals(0, response.getResponseCode());
 
@@ -102,9 +100,7 @@ public class TestHBaseHCatStorageHandler
         CommandProcessorResponse response = hcatDriver
                 .run("create table mytable(key int, value string) STORED BY " +
                      "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                    + "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                    "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                    "'hbase.columns.mapping'=':key,cf1:val')");
+                    + "TBLPROPERTIES ('hbase.columns.mapping'=':key,cf1:val')");
 
         assertEquals(0, response.getResponseCode());
 
@@ -140,13 +136,11 @@ public class TestHBaseHCatStorageHandler
         boolean doesTableExist = hAdmin.tableExists(tableName);
         assertTrue(doesTableExist);
 
-        hcatDriver.run("drop table mytable");
+        hcatDriver.run("drop table mytabletwo");
         CommandProcessorResponse response = hcatDriver
-                .run("create external table mytable(key int, valueone string, valuetwo string) STORED BY " +
+                .run("create external table mytabletwo(key int, valueone string, valuetwo string) STORED BY " +
                      "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                    + "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                    "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                    "'hbase.columns.mapping'=':key,familyone:val,familytwo:val'," +
+                    + "TBLPROPERTIES ('hbase.columns.mapping'=':key,familyone:val,familytwo:val'," +
                     "'hbase.table.name'='testTable')");
 
         assertEquals(0, response.getResponseCode());

Modified: incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputStorageDriver.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputStorageDriver.java?rev=1205693&r1=1205692&r2=1205693&view=diff
==============================================================================
--- incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputStorageDriver.java (original)
+++ incubator/hcatalog/trunk/storage-drivers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseInputStorageDriver.java Thu Nov 24 01:34:45 2011
@@ -120,9 +120,7 @@ public class TestHBaseInputStorageDriver
         String tableQuery = "CREATE TABLE " + databaseName + "." + tableName
                               + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
                               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                              + "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                              "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                              "'hbase.columns.mapping'=':key,testFamily:testQualifier1,testFamily:testQualifier2')" ;
+                              + "TBLPROPERTIES ('hbase.columns.mapping'=':key,testFamily:testQualifier1,testFamily:testQualifier2')" ;
 
         CommandProcessorResponse responseOne = hcatDriver.run(dbquery);
         assertEquals(0, responseOne.getResponseCode());
@@ -184,9 +182,8 @@ public class TestHBaseInputStorageDriver
         String tableQuery = "CREATE TABLE " + tableName
                               + "(key string, testqualifier1 string, testqualifier2 string) STORED BY " +
                               "'org.apache.hcatalog.hbase.HBaseHCatStorageHandler'"
-                              + "TBLPROPERTIES ('hcat.isd'='org.apache.hcatalog.hbase.HBaseInputStorageDriver', " +
-                              "'hcat.osd'='org.apache.hcatalog.hbase.HBaseOutputStorageDriver'," +
-                              "'hbase.columns.mapping'=':key,testFamily:testQualifier1,testFamily:testQualifier2')" ;
+                              + "TBLPROPERTIES ('hbase.columns.mapping'=':key," +
+                              		"testFamily:testQualifier1,testFamily:testQualifier2')" ;
 
         CommandProcessorResponse responseTwo = hcatDriver.run(tableQuery);
         assertEquals(0, responseTwo.getResponseCode());
@@ -284,4 +281,6 @@ public class TestHBaseInputStorageDriver
                 HCatFieldSchema.Type.STRING, ""));
         return schema;
     }
+
+
 }