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 av...@apache.org on 2012/12/07 01:54:45 UTC

svn commit: r1418162 - in /incubator/hcatalog/branches/branch-0.4: ./ webhcat/java-client/src/main/java/org/apache/hcatalog/api/ webhcat/java-client/src/test/java/org/apache/hcatalog/api/

Author: avandana
Date: Fri Dec  7 01:54:44 2012
New Revision: 1418162

URL: http://svn.apache.org/viewvc?rev=1418162&view=rev
Log:
HCAT-560 HCatClient should support addition of new columns to a Table. (branch 0.4)

Modified:
    incubator/hcatalog/branches/branch-0.4/CHANGES.txt
    incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClient.java
    incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClientHMSImpl.java
    incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java

Modified: incubator/hcatalog/branches/branch-0.4/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/CHANGES.txt?rev=1418162&r1=1418161&r2=1418162&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/CHANGES.txt (original)
+++ incubator/hcatalog/branches/branch-0.4/CHANGES.txt Fri Dec  7 01:54:44 2012
@@ -38,6 +38,8 @@ Trunk (unreleased changes)
   HCAT-427 Document storage-based authorization (lefty via gates)
 
   IMPROVEMENTS
+  HCAT-560 HCatClient should support addition of new columns to a Table.(mithunr via avandana)
+
   HCAT-548 Move topic creation in NotificationListener to a separate method (amalakar via avandana)
 
   HCAT-538 HCatalogStorer fails for 100GB of data with dynamic partitioning, number of partition is 300 (amalakar via toffer)

Modified: incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClient.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClient.java?rev=1418162&r1=1418161&r2=1418162&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClient.java (original)
+++ incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClient.java Fri Dec  7 01:54:44 2012
@@ -17,7 +17,6 @@
  */
 package org.apache.hcatalog.api;
 
-import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
@@ -25,23 +24,24 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.metastore.api.PartitionEventType;
 import org.apache.hcatalog.common.HCatException;
+import org.apache.hcatalog.data.schema.HCatFieldSchema;
 
 /**
  * The abstract class HCatClient containing APIs for HCatalog DDL commands.
  */
 public abstract class HCatClient {
 
-    public enum DROP_DB_MODE { RESTRICT, CASCADE };
+    public enum DropDBMode { RESTRICT, CASCADE }
+
     public static final String HCAT_CLIENT_IMPL_CLASS = "hcat.client.impl.class";
     /**
      * Creates an instance of HCatClient.
      *
      * @param conf An instance of configuration.
      * @return An instance of HCatClient.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
-    public static HCatClient create(Configuration conf) throws HCatException,
-            ConnectionFailureException {
+    public static HCatClient create(Configuration conf) throws HCatException {
         HCatClient client = null;
         String className = conf.get(HCAT_CLIENT_IMPL_CLASS,
                 HCatClientHMSImpl.class.getName());
@@ -49,7 +49,7 @@ public abstract class HCatClient {
             Class<? extends HCatClient> clientClass = Class.forName(className,
                     true, JavaUtils.getClassLoader()).asSubclass(
                     HCatClient.class);
-            client = (HCatClient) clientClass.newInstance();
+            client = clientClass.newInstance();
         } catch (ClassNotFoundException e) {
             throw new HCatException(
                     "ClassNotFoundException while creating client class.", e);
@@ -66,36 +66,36 @@ public abstract class HCatClient {
         return client;
     }
 
-    abstract void initialize(Configuration conf) throws HCatException,ConnectionFailureException;
+    abstract void initialize(Configuration conf) throws HCatException;
 
     /**
      * Get all existing databases that match the given
      * pattern. The matching occurs as per Java regular expressions
      *
-     * @param databasePattern  java re pattern
+     * @param pattern  java re pattern
      * @return list of database names
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract List<String> listDatabaseNamesByPattern(String pattern)
-            throws HCatException, ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Gets the database.
      *
      * @param dbName The name of the database.
      * @return An instance of HCatDatabaseInfo.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
-    public abstract HCatDatabase getDatabase(String dbName) throws HCatException,ConnectionFailureException;
+    public abstract HCatDatabase getDatabase(String dbName) throws HCatException;
 
     /**
      * Creates the database.
      *
      * @param dbInfo An instance of HCatCreateDBDesc.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void createDatabase(HCatCreateDBDesc dbInfo)
-            throws HCatException,ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Drops a database.
@@ -106,21 +106,21 @@ public abstract class HCatClient {
      * @param mode This is set to either "restrict" or "cascade". Restrict will
      *             remove the schema if all the tables are empty. Cascade removes
      *             everything including data and definitions.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void dropDatabase(String dbName, boolean ifExists,
-            DROP_DB_MODE mode) throws HCatException, ConnectionFailureException;
+            DropDBMode mode) throws HCatException;
 
     /**
      * Returns all existing tables from the specified database which match the given
      * pattern. The matching occurs as per Java regular expressions.
-     * @param dbName
-     * @param tablePattern
+     * @param dbName The name of the DB (to be searched)
+     * @param tablePattern The regex for the table-name
      * @return list of table names
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract List<String> listTableNamesByPattern(String dbName, String tablePattern)
-            throws HCatException,ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Gets the table.
@@ -128,19 +128,30 @@ public abstract class HCatClient {
      * @param dbName The name of the database.
      * @param tableName The name of the table.
      * @return An instance of HCatTableInfo.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract HCatTable getTable(String dbName, String tableName)
-            throws HCatException,ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Creates the table.
      *
      * @param createTableDesc An instance of HCatCreateTableDesc class.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void createTable(HCatCreateTableDesc createTableDesc)
-            throws HCatException,ConnectionFailureException;
+            throws HCatException;
+
+    /**
+     * Updates the Table's column schema to the specified definition.
+     *
+     * @param dbName The name of the database.
+     * @param tableName The name of the table.
+     * @param columnSchema The (new) definition of the column schema (i.e. list of fields).
+     *
+     */
+    public abstract void updateTableSchema(String dbName, String tableName, List<HCatFieldSchema> columnSchema)
+            throws HCatException;
 
     /**
      * Creates the table like an existing table.
@@ -152,11 +163,11 @@ public abstract class HCatClient {
      * @param isExternal Set to "true", if table has be created at a different
      *                   location other than default.
      * @param location The location for the table.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void createTableLike(String dbName, String existingTblName,
             String newTableName, boolean ifNotExists, boolean isExternal,
-            String location) throws HCatException,ConnectionFailureException;
+            String location) throws HCatException;
 
     /**
      * Drop table.
@@ -165,10 +176,10 @@ public abstract class HCatClient {
      * @param tableName The name of the table.
      * @param ifExists Hive returns an error if the database specified does not exist,
      *                 unless ifExists is set to true.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void dropTable(String dbName, String tableName,
-            boolean ifExists) throws HCatException,ConnectionFailureException;
+            boolean ifExists) throws HCatException;
 
     /**
      * Renames a table.
@@ -176,10 +187,10 @@ public abstract class HCatClient {
      * @param dbName The name of the database.
      * @param oldName The name of the table to be renamed.
      * @param newName The new name of the table.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void renameTable(String dbName, String oldName,
-            String newName) throws HCatException, ConnectionFailureException;
+            String newName) throws HCatException;
 
     /**
      * Gets all the partitions.
@@ -187,10 +198,10 @@ public abstract class HCatClient {
      * @param dbName The name of the database.
      * @param tblName The name of the table.
      * @return A list of partitions.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract List<HCatPartition> getPartitions(String dbName, String tblName)
-            throws HCatException,ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Gets the partition.
@@ -199,29 +210,29 @@ public abstract class HCatClient {
      * @param tableName The table name.
      * @param partitionSpec The partition specification, {[col_name,value],[col_name2,value2]}.
      * @return An instance of HCatPartitionInfo.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract HCatPartition getPartition(String dbName, String tableName,
-            Map<String,String> partitionSpec) throws HCatException,ConnectionFailureException;
+            Map<String,String> partitionSpec) throws HCatException;
 
     /**
      * Adds the partition.
      *
      * @param partInfo An instance of HCatAddPartitionDesc.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void addPartition(HCatAddPartitionDesc partInfo)
-            throws HCatException, ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Adds a list of partitions.
      *
      * @param partInfoList A list of HCatAddPartitionDesc.
-     * @return
-     * @throws HCatException,ConnectionFailureException
+     * @return The number of partitions added.
+     * @throws HCatException
      */
     public abstract int addPartitions(List<HCatAddPartitionDesc> partInfoList)
-            throws HCatException, ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Drops partition.
@@ -230,11 +241,11 @@ public abstract class HCatClient {
      * @param tableName The table name.
      * @param partitionSpec The partition specification, {[col_name,value],[col_name2,value2]}.
      * @param ifExists Hive returns an error if the partition specified does not exist, unless ifExists is set to true.
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void dropPartition(String dbName, String tableName,
             Map<String, String> partitionSpec, boolean ifExists)
-            throws HCatException, ConnectionFailureException;
+            throws HCatException;
 
     /**
      * List partitions by filter.
@@ -245,10 +256,10 @@ public abstract class HCatClient {
      *    for example "part1 = \"p1_abc\" and part2 <= "\p2_test\"". Filtering can
      *    be done only on string partition keys.
      * @return list of partitions
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract List<HCatPartition> listPartitionsByFilter(String dbName, String tblName,
-            String filter) throws HCatException,ConnectionFailureException;
+            String filter) throws HCatException;
 
     /**
      * Mark partition for event.
@@ -257,11 +268,11 @@ public abstract class HCatClient {
      * @param tblName The table name.
      * @param partKVs the key-values associated with the partition.
      * @param eventType the event type
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void markPartitionForEvent(String dbName, String tblName,
             Map<String, String> partKVs, PartitionEventType eventType)
-            throws HCatException,ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Checks if a partition is marked for event.
@@ -271,11 +282,11 @@ public abstract class HCatClient {
      * @param partKVs the key-values associated with the partition.
      * @param eventType the event type
      * @return true, if is partition marked for event
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract boolean isPartitionMarkedForEvent(String dbName, String tblName,
             Map<String, String> partKVs, PartitionEventType eventType)
-            throws HCatException,ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Gets the delegation token.
@@ -283,30 +294,29 @@ public abstract class HCatClient {
      * @param owner the owner
      * @param renewerKerberosPrincipalName the renewer kerberos principal name
      * @return the delegation token
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract String getDelegationToken(String owner,
-            String renewerKerberosPrincipalName) throws HCatException,
-            ConnectionFailureException;
+            String renewerKerberosPrincipalName) throws HCatException;
 
     /**
      * Renew delegation token.
      *
      * @param tokenStrForm the token string
      * @return the new expiration time
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract long renewDelegationToken(String tokenStrForm)
-            throws HCatException, ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Cancel delegation token.
      *
      * @param tokenStrForm the token string
-     * @throws HCatException,ConnectionFailureException
+     * @throws HCatException
      */
     public abstract void cancelDelegationToken(String tokenStrForm)
-            throws HCatException, ConnectionFailureException;
+            throws HCatException;
 
     /**
      * Close the hcatalog client.

Modified: incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClientHMSImpl.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClientHMSImpl.java?rev=1418162&r1=1418161&r2=1418162&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClientHMSImpl.java (original)
+++ incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClientHMSImpl.java Fri Dec  7 01:54:44 2012
@@ -44,6 +44,8 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hcatalog.common.HCatException;
 import org.apache.hcatalog.common.HCatUtil;
+import org.apache.hcatalog.data.schema.HCatFieldSchema;
+import org.apache.hcatalog.data.schema.HCatSchemaUtils;
 import org.apache.thrift.TException;
 
 /**
@@ -58,7 +60,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public List<String> listDatabaseNamesByPattern(String pattern)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         List<String> dbNames = null;
         try {
             dbNames = hmsClient.getDatabases(pattern);
@@ -69,8 +71,7 @@ public class HCatClientHMSImpl extends H
     }
 
     @Override
-    public HCatDatabase getDatabase(String dbName) throws HCatException,
-            ConnectionFailureException {
+    public HCatDatabase getDatabase(String dbName) throws HCatException {
         HCatDatabase db = null;
         try {
             Database hiveDB = hmsClient.getDatabase(checkDB(dbName));
@@ -91,8 +92,7 @@ public class HCatClientHMSImpl extends H
     }
 
     @Override
-    public void createDatabase(HCatCreateDBDesc dbInfo) throws HCatException,
-            ConnectionFailureException {
+    public void createDatabase(HCatCreateDBDesc dbInfo) throws HCatException {
         try {
             hmsClient.createDatabase(dbInfo.toHiveDb());
         } catch (AlreadyExistsException exp) {
@@ -113,14 +113,9 @@ public class HCatClientHMSImpl extends H
     }
 
     @Override
-    public void dropDatabase(String dbName, boolean ifExists, DROP_DB_MODE mode)
-            throws HCatException, ConnectionFailureException {
-        boolean isCascade;
-        if (mode.toString().equalsIgnoreCase("cascade")) {
-            isCascade = true;
-        } else {
-            isCascade = false;
-        }
+    public void dropDatabase(String dbName, boolean ifExists, DropDBMode mode)
+            throws HCatException {
+        boolean isCascade = mode.toString().equalsIgnoreCase("cascade");
         try {
             hmsClient.dropDatabase(checkDB(dbName), true, ifExists, isCascade);
         } catch (NoSuchObjectException e) {
@@ -141,7 +136,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public List<String> listTableNamesByPattern(String dbName,
-            String tablePattern) throws HCatException, ConnectionFailureException {
+            String tablePattern) throws HCatException {
         List<String> tableNames = null;
         try {
             tableNames = hmsClient.getTables(checkDB(dbName), tablePattern);
@@ -154,7 +149,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public HCatTable getTable(String dbName, String tableName)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         HCatTable table = null;
         try {
             Table hiveTable = hmsClient.getTable(checkDB(dbName), tableName);
@@ -175,11 +170,11 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public void createTable(HCatCreateTableDesc createTableDesc)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         try {
             hmsClient.createTable(createTableDesc.toHiveTable(hiveConfig));
         } catch (AlreadyExistsException e) {
-            if (createTableDesc.getIfNotExists() == false) {
+            if (!createTableDesc.getIfNotExists()) {
                 throw new HCatException(
                         "AlreadyExistsException while creating table.", e);
             }
@@ -201,9 +196,32 @@ public class HCatClientHMSImpl extends H
     }
 
     @Override
+    public void updateTableSchema(String dbName, String tableName, List<HCatFieldSchema> columnSchema) throws HCatException {
+        try {
+            Table table = hmsClient.getTable(dbName, tableName);
+            table.getSd().setCols(HCatSchemaUtils.getFieldSchemas(columnSchema));
+            hmsClient.alter_table(dbName, tableName, table);
+        }
+        catch (InvalidOperationException e) {
+            throw new HCatException("InvalidOperationException while updating table schema.", e);
+        }
+        catch (MetaException e) {
+            throw new HCatException("MetaException while updating table schema.", e);
+        }
+        catch (NoSuchObjectException e) {
+            throw new HCatException(
+                    "NoSuchObjectException while updating table schema.", e);
+        }
+        catch (TException e) {
+            throw new ConnectionFailureException(
+                    "TException while updating table schema.", e);
+        }
+    }
+
+    @Override
     public void createTableLike(String dbName, String existingTblName,
             String newTableName, boolean ifNotExists, boolean isExternal,
-            String location) throws HCatException, ConnectionFailureException {
+            String location) throws HCatException {
 
         Table hiveTable = getHiveTableLike(checkDB(dbName), existingTblName,
                 newTableName, ifNotExists, location);
@@ -236,7 +254,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public void dropTable(String dbName, String tableName, boolean ifExists)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         try {
             hmsClient.dropTable(checkDB(dbName), tableName,true, ifExists);
         } catch (NoSuchObjectException e) {
@@ -254,7 +272,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public void renameTable(String dbName, String oldName, String newName)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         Table tbl;
         try {
             Table oldtbl = hmsClient.getTable(checkDB(dbName), oldName);
@@ -286,7 +304,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public List<HCatPartition> getPartitions(String dbName, String tblName)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         List<HCatPartition> hcatPtns = new ArrayList<HCatPartition>();
         try {
             List<Partition> hivePtns = hmsClient.listPartitions(
@@ -309,8 +327,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public HCatPartition getPartition(String dbName, String tableName,
-            Map<String, String> partitionSpec) throws HCatException,
-            ConnectionFailureException {
+            Map<String, String> partitionSpec) throws HCatException {
         HCatPartition partition = null;
         try {
             ArrayList<String> ptnValues = new ArrayList<String>();
@@ -335,7 +352,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public void addPartition(HCatAddPartitionDesc partInfo)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         Table tbl = null;
         try {
             tbl = hmsClient.getTable(partInfo.getDatabaseName(),
@@ -367,7 +384,7 @@ public class HCatClientHMSImpl extends H
     @Override
     public void dropPartition(String dbName, String tableName,
             Map<String, String> partitionSpec, boolean ifExists)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         try {
             List<String> ptnValues = new ArrayList<String>();
             ptnValues.addAll(partitionSpec.values());
@@ -389,8 +406,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public List<HCatPartition> listPartitionsByFilter(String dbName,
-            String tblName, String filter) throws HCatException,
-            ConnectionFailureException {
+            String tblName, String filter) throws HCatException {
         List<HCatPartition> hcatPtns = new ArrayList<HCatPartition>();
         try {
             List<Partition> hivePtns = hmsClient.listPartitionsByFilter(
@@ -414,7 +430,7 @@ public class HCatClientHMSImpl extends H
     @Override
     public void markPartitionForEvent(String dbName, String tblName,
             Map<String, String> partKVs, PartitionEventType eventType)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         try {
             hmsClient.markPartitionForEvent(checkDB(dbName), tblName, partKVs,
                     eventType);
@@ -449,7 +465,7 @@ public class HCatClientHMSImpl extends H
     @Override
     public boolean isPartitionMarkedForEvent(String dbName, String tblName,
             Map<String, String> partKVs, PartitionEventType eventType)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         boolean isMarked = false;
         try {
             isMarked = hmsClient.isPartitionMarkedForEvent(checkDB(dbName),
@@ -485,8 +501,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public String getDelegationToken(String owner,
-            String renewerKerberosPrincipalName) throws HCatException,
-            ConnectionFailureException {
+            String renewerKerberosPrincipalName) throws HCatException {
         String token = null;
         try {
             token = hmsClient.getDelegationToken(owner,
@@ -503,8 +518,7 @@ public class HCatClientHMSImpl extends H
     }
 
     @Override
-    public long renewDelegationToken(String tokenStrForm) throws HCatException,
-            ConnectionFailureException {
+    public long renewDelegationToken(String tokenStrForm) throws HCatException {
         long time = 0;
         try {
             time = hmsClient.renewDelegationToken(tokenStrForm);
@@ -521,7 +535,7 @@ public class HCatClientHMSImpl extends H
 
     @Override
     public void cancelDelegationToken(String tokenStrForm)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         try {
             hmsClient.cancelDelegationToken(tokenStrForm);
         } catch (MetaException e) {
@@ -541,8 +555,7 @@ public class HCatClientHMSImpl extends H
      * Configuration)
      */
     @Override
-    void initialize(Configuration conf) throws HCatException,
-            ConnectionFailureException {
+    void initialize(Configuration conf) throws HCatException {
         this.config = conf;
         try {
             hiveConfig = HCatUtil.getHiveConf(config);
@@ -559,7 +572,7 @@ public class HCatClientHMSImpl extends H
 
     private Table getHiveTableLike(String dbName, String existingTblName,
             String newTableName, boolean isExternal, String location)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         Table oldtbl = null;
         Table newTable = null;
         try {
@@ -626,7 +639,7 @@ public class HCatClientHMSImpl extends H
      */
     @Override
     public int addPartitions(List<HCatAddPartitionDesc> partInfoList)
-            throws HCatException, ConnectionFailureException {
+            throws HCatException {
         int numPartitions = -1;
         if ((partInfoList == null) || (partInfoList.size() == 0)) {
             throw new HCatException("The partition list is null or empty.");

Modified: incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java?rev=1418162&r1=1418161&r2=1418162&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java (original)
+++ incubator/hcatalog/branches/branch-0.4/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java Fri Dec  7 01:54:44 2012
@@ -19,6 +19,7 @@ package org.apache.hcatalog.api;
 
 import java.math.BigInteger;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -45,6 +46,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 public class TestHCatClient {
@@ -101,7 +104,7 @@ public class TestHCatClient {
         String tableOne = "testTable1";
         String tableTwo = "testTable2";
         HCatClient client = HCatClient.create(new Configuration(hcatConf));
-        client.dropDatabase(db, true, HCatClient.DROP_DB_MODE.CASCADE);
+        client.dropDatabase(db, true, HCatClient.DropDBMode.CASCADE);
 
         HCatCreateDBDesc dbDesc = HCatCreateDBDesc.create(db).ifNotExists(false)
                 .build();
@@ -160,7 +163,7 @@ public class TestHCatClient {
         HCatClient client = HCatClient.create(new Configuration(hcatConf));
         String dbName = "ptnDB";
         String tableName = "pageView";
-        client.dropDatabase(dbName, true, HCatClient.DROP_DB_MODE.CASCADE);
+        client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE);
 
         HCatCreateDBDesc dbDesc = HCatCreateDBDesc.create(dbName)
                 .ifNotExists(true).build();
@@ -231,7 +234,7 @@ public class TestHCatClient {
     public void testDatabaseLocation() throws Exception{
         HCatClient client = HCatClient.create(new Configuration(hcatConf));
         String dbName = "locationDB";
-        client.dropDatabase(dbName, true, HCatClient.DROP_DB_MODE.CASCADE);
+        client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE);
 
         HCatCreateDBDesc dbDesc = HCatCreateDBDesc.create(dbName)
                 .ifNotExists(true).location("/tmp/"+dbName).build();
@@ -370,4 +373,34 @@ public class TestHCatClient {
             assertTrue("The expected exception was never thrown.", isExceptionCaught);
         }
     }
+
+    @Test
+    public void testUpdateTableSchema() throws Exception {
+        try {
+            HCatClient client = HCatClient.create(new Configuration(hcatConf));
+            final String dbName = "myDb";
+            final String tableName = "myTable";
+
+            client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE);
+
+            client.createDatabase(HCatCreateDBDesc.create(dbName).build());
+            List<HCatFieldSchema> oldSchema = Arrays.asList(new HCatFieldSchema("foo", Type.INT, ""),
+                    new HCatFieldSchema("bar", Type.STRING, ""));
+            client.createTable(HCatCreateTableDesc.create(dbName, tableName, oldSchema).build());
+
+            List<HCatFieldSchema> newSchema = Arrays.asList(new HCatFieldSchema("completely", Type.DOUBLE, ""),
+                    new HCatFieldSchema("new", Type.FLOAT, ""),
+                    new HCatFieldSchema("fields", Type.STRING, ""));
+
+            client.updateTableSchema(dbName, tableName, newSchema);
+
+            assertArrayEquals(newSchema.toArray(), client.getTable(dbName, tableName).getCols().toArray());
+
+            client.dropDatabase(dbName, false, HCatClient.DropDBMode.CASCADE);
+        }
+        catch (Exception exception) {
+            LOG.error("Unexpected exception.", exception);
+            assertTrue("Unexpected exception: " + exception.getMessage(), false);
+        }
+    }
 }