You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2010/01/21 08:31:27 UTC

svn commit: r901581 [8/10] - in /hadoop/hive/trunk: ./ cli/src/java/org/apache/hadoop/hive/cli/ common/src/java/org/apache/hadoop/hive/common/ common/src/java/org/apache/hadoop/hive/common/io/ contrib/src/java/org/apache/hadoop/hive/contrib/fileformat/...

Modified: hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java?rev=901581&r1=901580&r2=901581&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java (original)
+++ hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java Thu Jan 21 07:29:29 2010
@@ -40,7 +40,6 @@
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
-
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocol;
@@ -56,54 +55,57 @@
   private TTransport transport = null;
   private boolean open = false;
   private URI metastoreUris[];
-  private boolean standAloneClient = false;
+  private final boolean standAloneClient = false;
 
   // for thrift connects
   private int retries = 5;
 
   static final private Log LOG = LogFactory.getLog("hive.metastore");
 
-
   public HiveMetaStoreClient(HiveConf conf) throws MetaException {
-    if(conf == null) {
+    if (conf == null) {
       conf = new HiveConf(HiveMetaStoreClient.class);
     }
-    
+
     boolean localMetaStore = conf.getBoolean("hive.metastore.local", false);
-    if(localMetaStore) {
-      // instantiate the metastore server handler directly instead of connecting through the network
+    if (localMetaStore) {
+      // instantiate the metastore server handler directly instead of connecting
+      // through the network
       client = new HiveMetaStore.HMSHandler("hive client", conf);
-      this.open = true;
+      open = true;
       return;
     }
-    
+
     // get the number retries
     retries = conf.getInt("hive.metastore.connect.retries", 5);
 
     // user wants file store based configuration
-    if(conf.getVar(HiveConf.ConfVars.METASTOREURIS) != null) {
-      String metastoreUrisString []= conf.getVar(HiveConf.ConfVars.METASTOREURIS).split(",");
-      this.metastoreUris = new URI[metastoreUrisString.length];
+    if (conf.getVar(HiveConf.ConfVars.METASTOREURIS) != null) {
+      String metastoreUrisString[] = conf.getVar(
+          HiveConf.ConfVars.METASTOREURIS).split(",");
+      metastoreUris = new URI[metastoreUrisString.length];
       try {
         int i = 0;
-        for(String s: metastoreUrisString) {
+        for (String s : metastoreUrisString) {
           URI tmpUri = new URI(s);
-          if(tmpUri.getScheme() == null) {
-            throw new IllegalArgumentException("URI: "+s+" does not have a scheme");
+          if (tmpUri.getScheme() == null) {
+            throw new IllegalArgumentException("URI: " + s
+                + " does not have a scheme");
           }
-          this.metastoreUris[i++]= tmpUri;
+          metastoreUris[i++] = tmpUri;
 
         }
       } catch (IllegalArgumentException e) {
         throw (e);
-      } catch(Exception e) {
+      } catch (Exception e) {
         MetaStoreUtils.logAndThrowMetaException(e);
       }
-    } else if(conf.getVar(HiveConf.ConfVars.METASTOREDIRECTORY) != null) {
-      this.metastoreUris = new URI[1];
+    } else if (conf.getVar(HiveConf.ConfVars.METASTOREDIRECTORY) != null) {
+      metastoreUris = new URI[1];
       try {
-        this.metastoreUris[0] = new URI(conf.getVar(HiveConf.ConfVars.METASTOREDIRECTORY));
-      } catch(URISyntaxException e) {
+        metastoreUris[0] = new URI(conf
+            .getVar(HiveConf.ConfVars.METASTOREDIRECTORY));
+      } catch (URISyntaxException e) {
         MetaStoreUtils.logAndThrowMetaException(e);
       }
     } else {
@@ -111,9 +113,9 @@
       throw new MetaException("MetaStoreURIs not found in conf file");
     }
     // finally open the store
-    this.open();
+    open();
   }
-  
+
   /**
    * @param dbname
    * @param tbl_name
@@ -121,7 +123,8 @@
    * @throws InvalidOperationException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#alter_table(java.lang.String, java.lang.String, org.apache.hadoop.hive.metastore.api.Table)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#alter_table(java.lang.String,
+   *      java.lang.String, org.apache.hadoop.hive.metastore.api.Table)
    */
   public void alter_table(String dbname, String tbl_name, Table new_tbl)
       throws InvalidOperationException, MetaException, TException {
@@ -129,7 +132,7 @@
   }
 
   private void open() throws MetaException {
-    for(URI store: this.metastoreUris) {
+    for (URI store : metastoreUris) {
       LOG.info("Trying to connect to metastore with URI " + store);
       try {
         openStore(store);
@@ -141,63 +144,67 @@
         break;
       }
     }
-    if(!open) {
-      throw new MetaException("Could not connect to meta store using any of the URIs provided");
+    if (!open) {
+      throw new MetaException(
+          "Could not connect to meta store using any of the URIs provided");
     }
     LOG.info("Connected to metastore.");
   }
- 
+
   private void openStore(URI store) throws MetaException {
     open = false;
     transport = new TSocket(store.getHost(), store.getPort());
-    ((TSocket)transport).setTimeout(20000);
+    ((TSocket) transport).setTimeout(20000);
     TProtocol protocol = new TBinaryProtocol(transport);
     client = new ThriftHiveMetastore.Client(protocol);
 
-    for(int i = 0; i < retries && !this.open; ++i) {
+    for (int i = 0; i < retries && !open; ++i) {
       try {
         transport.open();
         open = true;
-      } catch(TTransportException e) {
+      } catch (TTransportException e) {
         LOG.warn("failed to connect to MetaStore, re-trying...");
         try {
           Thread.sleep(1000);
-        } catch(InterruptedException ignore) { }
+        } catch (InterruptedException ignore) {
+        }
       }
     }
-    if(!open) {
+    if (!open) {
       throw new MetaException("could not connect to meta store");
     }
   }
-  
+
   public void close() {
     open = false;
-    if((transport != null) && transport.isOpen()) {
+    if ((transport != null) && transport.isOpen()) {
       transport.close();
     }
-    if(standAloneClient) {
+    if (standAloneClient) {
       try {
         client.shutdown();
       } catch (TException e) {
-        //TODO:pc cleanup the exceptions
+        // TODO:pc cleanup the exceptions
         LOG.error("Unable to shutdown local metastore client");
         LOG.error(e.getStackTrace());
-        //throw new RuntimeException(e.getMessage());
+        // throw new RuntimeException(e.getMessage());
       }
     }
   }
 
-  public void dropTable(String tableName, boolean deleteData) throws MetaException, NoSuchObjectException {
+  public void dropTable(String tableName, boolean deleteData)
+      throws MetaException, NoSuchObjectException {
     // assume that it is default database
     try {
-      this.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, deleteData, false);
+      this.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName,
+          deleteData, false);
     } catch (NoSuchObjectException e) {
       throw e;
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
   }
-  
+
   /**
    * @param new_part
    * @return the added partition
@@ -207,8 +214,9 @@
    * @throws TException
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#add_partition(org.apache.hadoop.hive.metastore.api.Partition)
    */
-  public Partition add_partition(Partition new_part) throws InvalidObjectException,
-      AlreadyExistsException, MetaException, TException {
+  public Partition add_partition(Partition new_part)
+      throws InvalidObjectException, AlreadyExistsException, MetaException,
+      TException {
     return client.add_partition(new_part);
   }
 
@@ -221,10 +229,12 @@
    * @throws AlreadyExistsException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#append_partition(java.lang.String, java.lang.String, java.util.List)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#append_partition(java.lang.String,
+   *      java.lang.String, java.util.List)
    */
-  public Partition appendPartition(String db_name, String table_name, List<String> part_vals)
-      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+  public Partition appendPartition(String db_name, String table_name,
+      List<String> part_vals) throws InvalidObjectException,
+      AlreadyExistsException, MetaException, TException {
     return client.append_partition(db_name, table_name, part_vals);
   }
 
@@ -235,10 +245,11 @@
    * @throws AlreadyExistsException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_database(java.lang.String, java.lang.String)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_database(java.lang.String,
+   *      java.lang.String)
    */
-  public boolean createDatabase(String name, String location_uri) throws AlreadyExistsException,
-      MetaException, TException {
+  public boolean createDatabase(String name, String location_uri)
+      throws AlreadyExistsException, MetaException, TException {
     return client.create_database(name, location_uri);
   }
 
@@ -249,8 +260,8 @@
    * @throws TException
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_table(org.apache.hadoop.hive.metastore.api.Table)
    */
-  public void createTable(Table tbl) throws AlreadyExistsException, InvalidObjectException,
-  MetaException, NoSuchObjectException, TException {
+  public void createTable(Table tbl) throws AlreadyExistsException,
+      InvalidObjectException, MetaException, NoSuchObjectException, TException {
     client.create_table(tbl);
   }
 
@@ -263,8 +274,8 @@
    * @throws TException
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_type(org.apache.hadoop.hive.metastore.api.Type)
    */
-  public boolean createType(Type type) throws AlreadyExistsException, InvalidObjectException,
-      MetaException, TException {
+  public boolean createType(Type type) throws AlreadyExistsException,
+      InvalidObjectException, MetaException, TException {
     return client.create_type(type);
   }
 
@@ -287,29 +298,34 @@
    * @throws NoSuchObjectException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String, java.lang.String, java.util.List, boolean)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String,
+   *      java.lang.String, java.util.List, boolean)
    */
-  public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals)
-      throws NoSuchObjectException, MetaException, TException {
-        return dropPartition(db_name, tbl_name, part_vals, true);
-      }
+  public boolean dropPartition(String db_name, String tbl_name,
+      List<String> part_vals) throws NoSuchObjectException, MetaException,
+      TException {
+    return dropPartition(db_name, tbl_name, part_vals, true);
+  }
 
   /**
    * @param db_name
    * @param tbl_name
    * @param part_vals
-   * @param deleteData delete the underlying data or just delete the table in metadata
+   * @param deleteData
+   *          delete the underlying data or just delete the table in metadata
    * @return true or false
    * @throws NoSuchObjectException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String, java.lang.String, java.util.List, boolean)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String,
+   *      java.lang.String, java.util.List, boolean)
    */
-  public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData)
-      throws NoSuchObjectException, MetaException, TException {
+  public boolean dropPartition(String db_name, String tbl_name,
+      List<String> part_vals, boolean deleteData) throws NoSuchObjectException,
+      MetaException, TException {
     return client.drop_partition(db_name, tbl_name, part_vals, deleteData);
   }
-  
+
   /**
    * @param name
    * @param dbname
@@ -317,29 +333,33 @@
    * @throws ExistingDependentsException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, java.lang.String, boolean)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String,
+   *      java.lang.String, boolean)
    */
-  public void dropTable(String dbname, String name) throws NoSuchObjectException,
-      MetaException, TException {
-        dropTable(dbname, name, true, true);
-      }
+  public void dropTable(String dbname, String name)
+      throws NoSuchObjectException, MetaException, TException {
+    dropTable(dbname, name, true, true);
+  }
 
   /**
    * @param dbname
    * @param name
-   * @param deleteData delete the underlying data or just delete the table in metadata
+   * @param deleteData
+   *          delete the underlying data or just delete the table in metadata
    * @throws NoSuchObjectException
    * @throws ExistingDependentsException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, java.lang.String, boolean)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String,
+   *      java.lang.String, boolean)
    */
-  public void dropTable(String dbname, String name, boolean deleteData, boolean ignoreUknownTab) throws 
-      MetaException, TException, NoSuchObjectException {
+  public void dropTable(String dbname, String name, boolean deleteData,
+      boolean ignoreUknownTab) throws MetaException, TException,
+      NoSuchObjectException {
     try {
       client.drop_table(dbname, name, deleteData);
     } catch (NoSuchObjectException e) {
-      if(!ignoreUknownTab) {
+      if (!ignoreUknownTab) {
         throw e;
       }
     }
@@ -363,7 +383,8 @@
    * @throws TException
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_type_all(java.lang.String)
    */
-  public Map<String, Type> getTypeAll(String name) throws MetaException, TException {
+  public Map<String, Type> getTypeAll(String name) throws MetaException,
+      TException {
     return client.get_type_all(name);
   }
 
@@ -386,8 +407,8 @@
    * @throws MetaException
    * @throws TException
    */
-  public List<Partition> listPartitions(String db_name, String tbl_name, short max_parts)
-      throws NoSuchObjectException, MetaException, TException {
+  public List<Partition> listPartitions(String db_name, String tbl_name,
+      short max_parts) throws NoSuchObjectException, MetaException, TException {
     return client.get_partitions(db_name, tbl_name, max_parts);
   }
 
@@ -399,8 +420,8 @@
    * @throws TException
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_database(java.lang.String)
    */
-  public Database getDatabase(String name) throws NoSuchObjectException, MetaException,
-      TException {
+  public Database getDatabase(String name) throws NoSuchObjectException,
+      MetaException, TException {
     return client.get_database(name);
   }
 
@@ -411,13 +432,14 @@
    * @return the partition
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String, java.lang.String, java.util.List)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String,
+   *      java.lang.String, java.util.List)
    */
-  public Partition getPartition(String db_name, String tbl_name, List<String> part_vals)
-      throws MetaException, TException {
+  public Partition getPartition(String db_name, String tbl_name,
+      List<String> part_vals) throws MetaException, TException {
     return client.get_partition(db_name, tbl_name, part_vals);
   }
-  
+
   /**
    * @param name
    * @param dbname
@@ -425,10 +447,12 @@
    * @throws NoSuchObjectException
    * @throws MetaException
    * @throws TException
-   * @throws NoSuchObjectException 
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_table(java.lang.String, java.lang.String)
+   * @throws NoSuchObjectException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_table(java.lang.String,
+   *      java.lang.String)
    */
-  public Table getTable(String dbname, String name) throws MetaException, TException, NoSuchObjectException {
+  public Table getTable(String dbname, String name) throws MetaException,
+      TException, NoSuchObjectException {
     return client.get_table(dbname, name);
   }
 
@@ -442,23 +466,24 @@
   public Type getType(String name) throws MetaException, TException {
     return client.get_type(name);
   }
-  
-  public List<String> getTables(String dbname, String tablePattern) throws MetaException {
+
+  public List<String> getTables(String dbname, String tablePattern)
+      throws MetaException {
     try {
       return client.get_tables(dbname, tablePattern);
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
-    return null; 
+    return null;
   }
-  
+
   public List<String> getTables(String tablePattern) throws MetaException {
     String dbname = MetaStoreUtils.DEFAULT_DATABASE_NAME;
-    return this.getTables(dbname, tablePattern); 
+    return this.getTables(dbname, tablePattern);
   }
 
-  public boolean tableExists(String tableName) throws MetaException, TException,
-  UnknownDBException {
+  public boolean tableExists(String tableName) throws MetaException,
+      TException, UnknownDBException {
     try {
       client.get_table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
     } catch (NoSuchObjectException e) {
@@ -467,21 +492,21 @@
     return true;
   }
 
-  public Table getTable(String tableName) throws MetaException, TException, NoSuchObjectException {
+  public Table getTable(String tableName) throws MetaException, TException,
+      NoSuchObjectException {
     return getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
   }
 
-  public List<String> listPartitionNames(String dbName, String tblName, short max)
-      throws MetaException, TException {
+  public List<String> listPartitionNames(String dbName, String tblName,
+      short max) throws MetaException, TException {
     return client.get_partition_names(dbName, tblName, max);
   }
 
-  public void alter_partition(String dbName, String tblName,
-      Partition newPart) throws InvalidOperationException, MetaException,
-      TException {
+  public void alter_partition(String dbName, String tblName, Partition newPart)
+      throws InvalidOperationException, MetaException, TException {
     client.alter_partition(dbName, tblName, newPart);
   }
-  
+
   /**
    * @param db
    * @param tableName
@@ -489,10 +514,12 @@
    * @throws UnknownDBException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_fields(java.lang.String, java.lang.String)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_fields(java.lang.String,
+   *      java.lang.String)
    */
-  public List<FieldSchema> getFields(String db, String tableName) 
-      throws MetaException, TException, UnknownTableException, UnknownDBException {
+  public List<FieldSchema> getFields(String db, String tableName)
+      throws MetaException, TException, UnknownTableException,
+      UnknownDBException {
     return client.get_fields(db, tableName);
   }
 
@@ -503,14 +530,16 @@
    * @throws UnknownDBException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_schema(java.lang.String, java.lang.String)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_schema(java.lang.String,
+   *      java.lang.String)
    */
-  public List<FieldSchema> getSchema(String db, String tableName) 
-      throws MetaException, TException, UnknownTableException, UnknownDBException {
+  public List<FieldSchema> getSchema(String db, String tableName)
+      throws MetaException, TException, UnknownTableException,
+      UnknownDBException {
     return client.get_schema(db, tableName);
   }
 
-  public String getConfigValue(String name, String defaultValue) 
+  public String getConfigValue(String name, String defaultValue)
       throws TException, ConfigValSecurityException {
     return client.get_config_value(name, defaultValue);
   }

Modified: hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java?rev=901581&r1=901580&r2=901581&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java (original)
+++ hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java Thu Jan 21 07:29:29 2010
@@ -31,71 +31,99 @@
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
-
 import org.apache.thrift.TException;
 
 /**
- * TODO Unnecessary when the server sides for both dbstore and filestore are merged
+ * TODO Unnecessary when the server sides for both dbstore and filestore are
+ * merged
  */
 public interface IMetaStoreClient {
 
   public void close();
 
-  public List<String> getTables(String dbName, String tablePattern) throws MetaException, UnknownTableException, TException,
+  public List<String> getTables(String dbName, String tablePattern)
+      throws MetaException, UnknownTableException, TException,
       UnknownDBException;
 
   /**
    * Drop the table.
-   * @param tableName The table to drop
-   * @param deleteData Should we delete the underlying data
-   * @throws MetaException Could not drop table properly.
-   * @throws UnknownTableException The table wasn't found.
-   * @throws TException A thrift communication error occurred
-   * @throws NoSuchObjectException The table wasn't found.
+   * 
+   * @param tableName
+   *          The table to drop
+   * @param deleteData
+   *          Should we delete the underlying data
+   * @throws MetaException
+   *           Could not drop table properly.
+   * @throws UnknownTableException
+   *           The table wasn't found.
+   * @throws TException
+   *           A thrift communication error occurred
+   * @throws NoSuchObjectException
+   *           The table wasn't found.
    */
-  public void dropTable(String tableName, boolean deleteData) 
-    throws MetaException, UnknownTableException, TException, NoSuchObjectException;
+  public void dropTable(String tableName, boolean deleteData)
+      throws MetaException, UnknownTableException, TException,
+      NoSuchObjectException;
 
   /**
    * Drop the table.
-   * @param dbname The database for this table
-   * @param tableName The table to drop
-   * @throws MetaException Could not drop table properly.
-   * @throws NoSuchObjectException The table wasn't found.
-   * @throws TException A thrift communication error occurred
+   * 
+   * @param dbname
+   *          The database for this table
+   * @param tableName
+   *          The table to drop
+   * @throws MetaException
+   *           Could not drop table properly.
+   * @throws NoSuchObjectException
+   *           The table wasn't found.
+   * @throws TException
+   *           A thrift communication error occurred
    * @throws ExistingDependentsException
    */
-  public void dropTable(String dbname, String tableName, boolean deleteData, 
-      boolean ignoreUknownTab) throws  
-      MetaException, TException, NoSuchObjectException;
+  public void dropTable(String dbname, String tableName, boolean deleteData,
+      boolean ignoreUknownTab) throws MetaException, TException,
+      NoSuchObjectException;
 
-  //public void createTable(String tableName, Properties schema) throws MetaException, UnknownTableException,
-    //  TException;
+  // public void createTable(String tableName, Properties schema) throws
+  // MetaException, UnknownTableException,
+  // TException;
 
-  public boolean tableExists(String tableName) throws MetaException, TException, UnknownDBException;
+  public boolean tableExists(String tableName) throws MetaException,
+      TException, UnknownDBException;
 
   /**
-   * Get a table object. 
-   * @param tableName Name of the table to fetch.
+   * Get a table object.
+   * 
+   * @param tableName
+   *          Name of the table to fetch.
    * @return An object representing the table.
-   * @throws MetaException Could not fetch the table
-   * @throws TException A thrift communication error occurred 
-   * @throws NoSuchObjectException In case the table wasn't found.
-   */
-  public Table getTable(String tableName) throws MetaException, 
-    TException, NoSuchObjectException;
-  
-  /**
-   * Get a table object. 
-   * @param dbName The database the table is located in.
-   * @param tableName Name of the table to fetch.
+   * @throws MetaException
+   *           Could not fetch the table
+   * @throws TException
+   *           A thrift communication error occurred
+   * @throws NoSuchObjectException
+   *           In case the table wasn't found.
+   */
+  public Table getTable(String tableName) throws MetaException, TException,
+      NoSuchObjectException;
+
+  /**
+   * Get a table object.
+   * 
+   * @param dbName
+   *          The database the table is located in.
+   * @param tableName
+   *          Name of the table to fetch.
    * @return An object representing the table.
-   * @throws MetaException Could not fetch the table
-   * @throws TException A thrift communication error occurred 
-   * @throws NoSuchObjectException In case the table wasn't found.
+   * @throws MetaException
+   *           Could not fetch the table
+   * @throws TException
+   *           A thrift communication error occurred
+   * @throws NoSuchObjectException
+   *           In case the table wasn't found.
    */
-  public Table getTable(String dbName, String tableName) 
-    throws MetaException, TException, NoSuchObjectException;
+  public Table getTable(String dbName, String tableName) throws MetaException,
+      TException, NoSuchObjectException;
 
   /**
    * @param tableName
@@ -106,23 +134,31 @@
    * @throws AlreadyExistsException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#append_partition(java.lang.String, java.lang.String, java.util.List)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#append_partition(java.lang.String,
+   *      java.lang.String, java.util.List)
    */
-  public Partition appendPartition(String tableName, String dbName, List<String> partVals)
-      throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
-  
+  public Partition appendPartition(String tableName, String dbName,
+      List<String> partVals) throws InvalidObjectException,
+      AlreadyExistsException, MetaException, TException;
+
   /**
    * Add a partition to the table.
-   * @param partition The partition to add
+   * 
+   * @param partition
+   *          The partition to add
    * @return The partition added
-   * @throws InvalidObjectException Could not find table to add to
-   * @throws AlreadyExistsException Partition already exists
-   * @throws MetaException Could not add partition
-   * @throws TException Thrift exception
+   * @throws InvalidObjectException
+   *           Could not find table to add to
+   * @throws AlreadyExistsException
+   *           Partition already exists
+   * @throws MetaException
+   *           Could not add partition
+   * @throws TException
+   *           Thrift exception
    */
-  public Partition add_partition(Partition partition) 
-    throws InvalidObjectException, AlreadyExistsException, 
-      MetaException, TException;
+  public Partition add_partition(Partition partition)
+      throws InvalidObjectException, AlreadyExistsException, MetaException,
+      TException;
 
   /**
    * @param tblName
@@ -131,11 +167,12 @@
    * @return the partition object
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String, java.lang.String, java.util.List)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String,
+   *      java.lang.String, java.util.List)
    */
-  public Partition getPartition(String tblName, String dbName, List<String> partVals)
-      throws MetaException, TException ;
-  
+  public Partition getPartition(String tblName, String dbName,
+      List<String> partVals) throws MetaException, TException;
+
   /**
    * @param tbl_name
    * @param db_name
@@ -145,11 +182,12 @@
    * @throws MetaException
    * @throws TException
    */
-  public List<Partition> listPartitions(String db_name, String tbl_name, short max_parts)
-      throws NoSuchObjectException, MetaException, TException;
+  public List<Partition> listPartitions(String db_name, String tbl_name,
+      short max_parts) throws NoSuchObjectException, MetaException, TException;
+
+  public List<String> listPartitionNames(String db_name, String tbl_name,
+      short max_parts) throws MetaException, TException;
 
-  public List<String> listPartitionNames(String db_name, String tbl_name, short max_parts)
-    throws  MetaException, TException;
   /**
    * @param tbl
    * @throws AlreadyExistsException
@@ -159,40 +197,53 @@
    * @throws TException
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_table(org.apache.hadoop.hive.metastore.api.Table)
    */
-  public void createTable(Table tbl) throws AlreadyExistsException, InvalidObjectException,
-      MetaException, NoSuchObjectException, TException ;
+  public void createTable(Table tbl) throws AlreadyExistsException,
+      InvalidObjectException, MetaException, NoSuchObjectException, TException;
+
+  public void alter_table(String defaultDatabaseName, String tblName,
+      Table table) throws InvalidOperationException, MetaException, TException;
+
+  public boolean createDatabase(String name, String location_uri)
+      throws AlreadyExistsException, MetaException, TException;
 
-  public void alter_table(String defaultDatabaseName, String tblName, Table table) throws InvalidOperationException, MetaException, TException;
-  public boolean createDatabase(String name, String location_uri) throws AlreadyExistsException, MetaException, TException;
   public boolean dropDatabase(String name) throws MetaException, TException;
 
   /**
    * @param db_name
    * @param tbl_name
    * @param part_vals
-   * @param deleteData delete the underlying data or just delete the table in metadata
+   * @param deleteData
+   *          delete the underlying data or just delete the table in metadata
    * @return true or false
    * @throws NoSuchObjectException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String, java.lang.String, java.util.List, boolean)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String,
+   *      java.lang.String, java.util.List, boolean)
    */
-  public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData)
-      throws NoSuchObjectException, MetaException, TException;
+  public boolean dropPartition(String db_name, String tbl_name,
+      List<String> part_vals, boolean deleteData) throws NoSuchObjectException,
+      MetaException, TException;
 
   /**
    * updates a partition to new partition
-   * @param dbName database of the old partition
-   * @param tblName table name of the old partition
-   * @param newPart new partition
-   * @throws InvalidOperationException if the old partition does not exist
-   * @throws MetaException if error in updating metadata
-   * @throws TException if error in communicating with metastore server
+   * 
+   * @param dbName
+   *          database of the old partition
+   * @param tblName
+   *          table name of the old partition
+   * @param newPart
+   *          new partition
+   * @throws InvalidOperationException
+   *           if the old partition does not exist
+   * @throws MetaException
+   *           if error in updating metadata
+   * @throws TException
+   *           if error in communicating with metastore server
    */
-  public void alter_partition(String dbName, String tblName,
-      Partition newPart) throws InvalidOperationException, MetaException,
-      TException;
-  
+  public void alter_partition(String dbName, String tblName, Partition newPart)
+      throws InvalidOperationException, MetaException, TException;
+
   /**
    * @param db
    * @param tableName
@@ -200,10 +251,13 @@
    * @throws UnknownDBException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_fields(java.lang.String, java.lang.String)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_fields(java.lang.String,
+   *      java.lang.String)
    */
-  public List<FieldSchema> getFields(String db, String tableName) 
-      throws MetaException, TException, UnknownTableException, UnknownDBException;
+  public List<FieldSchema> getFields(String db, String tableName)
+      throws MetaException, TException, UnknownTableException,
+      UnknownDBException;
+
   /**
    * @param db
    * @param tableName
@@ -211,17 +265,21 @@
    * @throws UnknownDBException
    * @throws MetaException
    * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_schema(java.lang.String, java.lang.String)
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_schema(java.lang.String,
+   *      java.lang.String)
    */
-  public List<FieldSchema> getSchema(String db, String tableName) 
-      throws MetaException, TException, UnknownTableException, UnknownDBException;
-  
+  public List<FieldSchema> getSchema(String db, String tableName)
+      throws MetaException, TException, UnknownTableException,
+      UnknownDBException;
+
   /**
-   * @param name name of the configuration property to get the value of
-   * @param defaultValue the value to return if property with the given name doesn't exist
+   * @param name
+   *          name of the configuration property to get the value of
+   * @param defaultValue
+   *          the value to return if property with the given name doesn't exist
    * @return
    * @throws TException
-   * @throws ConfigValSecurityException 
+   * @throws ConfigValSecurityException
    */
   public String getConfigValue(String name, String defaultValue)
       throws TException, ConfigValSecurityException;

Modified: hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=901581&r1=901580&r2=901581&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Thu Jan 21 07:29:29 2010
@@ -62,24 +62,26 @@
 
   /**
    * printStackTrace
-   *
+   * 
    * Helper function to print an exception stack trace to the log and not stderr
-   *
-   * @param e the exception
-   *
+   * 
+   * @param e
+   *          the exception
+   * 
    */
   static public void printStackTrace(Exception e) {
-    for(StackTraceElement s: e.getStackTrace()) {
+    for (StackTraceElement s : e.getStackTrace()) {
       LOG.error(s);
     }
   }
 
-  public static Table createColumnsetSchema(String name, List<String> columns, List<String> partCols, Configuration conf) throws MetaException {
+  public static Table createColumnsetSchema(String name, List<String> columns,
+      List<String> partCols, Configuration conf) throws MetaException {
 
     if (columns == null) {
       throw new MetaException("columns not specified for table " + name);
     }
-    
+
     Table tTable = new Table();
     tTable.setTableName(name);
     tTable.setSd(new StorageDescriptor());
@@ -88,12 +90,14 @@
     SerDeInfo serdeInfo = sd.getSerdeInfo();
     serdeInfo.setSerializationLib(LazySimpleSerDe.class.getName());
     serdeInfo.setParameters(new HashMap<String, String>());
-    serdeInfo.getParameters().put(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
-    
-    List<FieldSchema>  fields = new ArrayList<FieldSchema>();
+    serdeInfo.getParameters().put(
+        org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
+
+    List<FieldSchema> fields = new ArrayList<FieldSchema>();
     sd.setCols(fields);
-    for (String col: columns) {
-      FieldSchema field = new FieldSchema(col, org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "'default'");
+    for (String col : columns) {
+      FieldSchema field = new FieldSchema(col,
+          org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "'default'");
       fields.add(field);
     }
 
@@ -101,57 +105,66 @@
     for (String partCol : partCols) {
       FieldSchema part = new FieldSchema();
       part.setName(partCol);
-      part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default partition key
+      part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default
+                                                                             // partition
+                                                                             // key
       tTable.getPartitionKeys().add(part);
     }
     sd.setNumBuckets(-1);
     return tTable;
   }
 
-
   /**
    * recursiveDelete
-   *
-   * just recursively deletes a dir - you'd think Java would have something to do this??
-   *
-   * @param f - the file/dir to delete
-   * @exception IOException propogate f.delete() exceptions
-   *
+   * 
+   * just recursively deletes a dir - you'd think Java would have something to
+   * do this??
+   * 
+   * @param f
+   *          - the file/dir to delete
+   * @exception IOException
+   *              propogate f.delete() exceptions
+   * 
    */
   static public void recursiveDelete(File f) throws IOException {
-    if(f.isDirectory()) {
-      File fs [] = f.listFiles();
-      for(File subf: fs) {
+    if (f.isDirectory()) {
+      File fs[] = f.listFiles();
+      for (File subf : fs) {
         recursiveDelete(subf);
       }
     }
-    if(!f.delete()) {
+    if (!f.delete()) {
       throw new IOException("could not delete: " + f.getPath());
     }
   }
 
-
   /**
    * getDeserializer
-   *
+   * 
    * Get the Deserializer for a table given its name and properties.
-   *
-   * @param conf hadoop config
-   * @param schema the properties to use to instantiate the deserializer
+   * 
+   * @param conf
+   *          hadoop config
+   * @param schema
+   *          the properties to use to instantiate the deserializer
    * @return the Deserializer
-   * @exception MetaException if any problems instantiating the Deserializer
-   *
-   * todo - this should move somewhere into serde.jar
-   *
+   * @exception MetaException
+   *              if any problems instantiating the Deserializer
+   * 
+   *              todo - this should move somewhere into serde.jar
+   * 
    */
-  static public Deserializer getDeserializer(Configuration conf, Properties schema) throws MetaException  {
-    String lib = schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB);
+  static public Deserializer getDeserializer(Configuration conf,
+      Properties schema) throws MetaException {
+    String lib = schema
+        .getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB);
     try {
       Deserializer deserializer = SerDeUtils.lookupDeserializer(lib);
-      ((Deserializer)deserializer).initialize(conf, schema);
+      (deserializer).initialize(conf, schema);
       return deserializer;
     } catch (Exception e) {
-      LOG.error("error in initSerDe: " + e.getClass().getName() + " " + e.getMessage());
+      LOG.error("error in initSerDe: " + e.getClass().getName() + " "
+          + e.getMessage());
       MetaStoreUtils.printStackTrace(e);
       throw new MetaException(e.getClass().getName() + " " + e.getMessage());
     }
@@ -159,18 +172,22 @@
 
   /**
    * getDeserializer
-   *
+   * 
    * Get the Deserializer for a table.
-   *
-   * @param conf - hadoop config
-   * @param table the table
+   * 
+   * @param conf
+   *          - hadoop config
+   * @param table
+   *          the table
    * @return the Deserializer
-   * @exception MetaException if any problems instantiating the Deserializer
-   *
-   * todo - this should move somewhere into serde.jar
-   *
+   * @exception MetaException
+   *              if any problems instantiating the Deserializer
+   * 
+   *              todo - this should move somewhere into serde.jar
+   * 
    */
-  static public Deserializer getDeserializer(Configuration conf, org.apache.hadoop.hive.metastore.api.Table table) throws MetaException  {
+  static public Deserializer getDeserializer(Configuration conf,
+      org.apache.hadoop.hive.metastore.api.Table table) throws MetaException {
     String lib = table.getSd().getSerdeInfo().getSerializationLib();
     try {
       Deserializer deserializer = SerDeUtils.lookupDeserializer(lib);
@@ -179,26 +196,30 @@
     } catch (RuntimeException e) {
       throw e;
     } catch (Exception e) {
-      LOG.error("error in initSerDe: " + e.getClass().getName() + " " + e.getMessage());
+      LOG.error("error in initSerDe: " + e.getClass().getName() + " "
+          + e.getMessage());
       MetaStoreUtils.printStackTrace(e);
       throw new MetaException(e.getClass().getName() + " " + e.getMessage());
     }
   }
-  
+
   /**
    * getDeserializer
-   *
+   * 
    * Get the Deserializer for a partition.
-   *
-   * @param conf - hadoop config
-   * @param partition the partition
+   * 
+   * @param conf
+   *          - hadoop config
+   * @param partition
+   *          the partition
    * @return the Deserializer
-   * @exception MetaException if any problems instantiating the Deserializer
-   *
+   * @exception MetaException
+   *              if any problems instantiating the Deserializer
+   * 
    */
-	static public Deserializer getDeserializer(Configuration conf,
-	    org.apache.hadoop.hive.metastore.api.Partition part,
-	    org.apache.hadoop.hive.metastore.api.Table table) throws MetaException {
+  static public Deserializer getDeserializer(Configuration conf,
+      org.apache.hadoop.hive.metastore.api.Partition part,
+      org.apache.hadoop.hive.metastore.api.Table table) throws MetaException {
     String lib = part.getSd().getSerdeInfo().getSerializationLib();
     try {
       Deserializer deserializer = SerDeUtils.lookupDeserializer(lib);
@@ -207,79 +228,86 @@
     } catch (RuntimeException e) {
       throw e;
     } catch (Exception e) {
-      LOG.error("error in initSerDe: " + e.getClass().getName() + " " + e.getMessage());
+      LOG.error("error in initSerDe: " + e.getClass().getName() + " "
+          + e.getMessage());
       MetaStoreUtils.printStackTrace(e);
       throw new MetaException(e.getClass().getName() + " " + e.getMessage());
     }
   }
-  
-  static public void deleteWHDirectory(Path path,Configuration conf, boolean use_trash) throws MetaException {
+
+  static public void deleteWHDirectory(Path path, Configuration conf,
+      boolean use_trash) throws MetaException {
 
     try {
-      if(!path.getFileSystem(conf).exists(path)) {
-        LOG.warn("drop data called on table/partition with no directory: " + path);
+      if (!path.getFileSystem(conf).exists(path)) {
+        LOG.warn("drop data called on table/partition with no directory: "
+            + path);
         return;
       }
 
-      if(use_trash) {
+      if (use_trash) {
 
         int count = 0;
-        Path newPath = new Path("/Trash/Current" + path.getParent().toUri().getPath());
+        Path newPath = new Path("/Trash/Current"
+            + path.getParent().toUri().getPath());
 
-        if(path.getFileSystem(conf).exists(newPath) == false) {
+        if (path.getFileSystem(conf).exists(newPath) == false) {
           path.getFileSystem(conf).mkdirs(newPath);
         }
 
         do {
-          newPath = new Path("/Trash/Current" + path.toUri().getPath() + "." + count);
-          if(path.getFileSystem(conf).exists(newPath)) {
+          newPath = new Path("/Trash/Current" + path.toUri().getPath() + "."
+              + count);
+          if (path.getFileSystem(conf).exists(newPath)) {
             count++;
             continue;
           }
-          if(path.getFileSystem(conf).rename(path, newPath)) {
+          if (path.getFileSystem(conf).rename(path, newPath)) {
             break;
           }
-        } while(++count < 50) ;
-        if(count >= 50) {
+        } while (++count < 50);
+        if (count >= 50) {
           throw new MetaException("Rename failed due to maxing out retries");
         }
       } else {
         // directly delete it
         path.getFileSystem(conf).delete(path, true);
       }
-    } catch(IOException e) {
+    } catch (IOException e) {
       LOG.error("Got exception trying to delete data dir: " + e);
       throw new MetaException(e.getMessage());
-    } catch(MetaException e) {
+    } catch (MetaException e) {
       LOG.error("Got exception trying to delete data dir: " + e);
       throw e;
     }
   }
 
-
   /**
    * validateName
-   *
-   * Checks the name conforms to our standars which are: "[a-zA-z_0-9]+".
-   * checks this is just characters and numbers and _ 
-   *
-   * @param name the name to validate
+   * 
+   * Checks the name conforms to our standars which are: "[a-zA-z_0-9]+". checks
+   * this is just characters and numbers and _
+   * 
+   * @param name
+   *          the name to validate
    * @return true or false depending on conformance
-   * @exception MetaException if it doesn't match the pattern.
+   * @exception MetaException
+   *              if it doesn't match the pattern.
    */
   static public boolean validateName(String name) {
     Pattern tpat = Pattern.compile("[\\w_]+");
     Matcher m = tpat.matcher(name);
-    if(m.matches()) {
+    if (m.matches()) {
       return true;
     }
     return false;
   }
-  
+
   static public boolean validateColNames(List<FieldSchema> cols) {
     for (FieldSchema fieldSchema : cols) {
-      if(!validateName(fieldSchema.getName()))
+      if (!validateName(fieldSchema.getName())) {
         return false;
+      }
     }
     return true;
   }
@@ -289,78 +317,114 @@
   }
 
   public static String getMapType(String k, String v) {
-    return "map<" + k +"," + v + ">";
+    return "map<" + k + "," + v + ">";
   }
 
-  public static Table getTable(Configuration conf, Properties schema) throws MetaException {
+  public static Table getTable(Configuration conf, Properties schema)
+      throws MetaException {
     Table t = new Table();
     t.setSd(new StorageDescriptor());
-    t.setTableName(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME));
-    t.getSd().setLocation(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_LOCATION));
-    t.getSd().setInputFormat(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_INPUT_FORMAT,
-          org.apache.hadoop.mapred.SequenceFileInputFormat.class.getName())); 
-    t.getSd().setOutputFormat(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT,
-          org.apache.hadoop.mapred.SequenceFileOutputFormat.class.getName())); 
+    t
+        .setTableName(schema
+            .getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME));
+    t
+        .getSd()
+        .setLocation(
+            schema
+                .getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_LOCATION));
+    t.getSd().setInputFormat(
+        schema.getProperty(
+            org.apache.hadoop.hive.metastore.api.Constants.FILE_INPUT_FORMAT,
+            org.apache.hadoop.mapred.SequenceFileInputFormat.class.getName()));
+    t.getSd().setOutputFormat(
+        schema.getProperty(
+            org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT,
+            org.apache.hadoop.mapred.SequenceFileOutputFormat.class.getName()));
     t.setPartitionKeys(new ArrayList<FieldSchema>());
     t.setDbName(MetaStoreUtils.DEFAULT_DATABASE_NAME);
-    String part_cols_str = schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS);
+    String part_cols_str = schema
+        .getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS);
     t.setPartitionKeys(new ArrayList<FieldSchema>());
     if (part_cols_str != null && (part_cols_str.trim().length() != 0)) {
-      String [] part_keys = part_cols_str.trim().split("/");
-      for (String key: part_keys) {
+      String[] part_keys = part_cols_str.trim().split("/");
+      for (String key : part_keys) {
         FieldSchema part = new FieldSchema();
         part.setName(key);
-        part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default partition key
+        part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default
+                                                                               // partition
+                                                                               // key
         t.getPartitionKeys().add(part);
       }
     }
-    t.getSd().setNumBuckets(Integer.parseInt(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_COUNT, "-1")));
-    String bucketFieldName = schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_FIELD_NAME);
+    t.getSd()
+        .setNumBuckets(
+            Integer.parseInt(schema.getProperty(
+                org.apache.hadoop.hive.metastore.api.Constants.BUCKET_COUNT,
+                "-1")));
+    String bucketFieldName = schema
+        .getProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_FIELD_NAME);
     t.getSd().setBucketCols(new ArrayList<String>(1));
     if ((bucketFieldName != null) && (bucketFieldName.trim().length() != 0)) {
       t.getSd().setBucketCols(new ArrayList<String>(1));
       t.getSd().getBucketCols().add(bucketFieldName);
     }
-    
+
     t.getSd().setSerdeInfo(new SerDeInfo());
     t.getSd().getSerdeInfo().setParameters(new HashMap<String, String>());
     t.getSd().getSerdeInfo().setName(t.getTableName());
-    t.getSd().getSerdeInfo().setSerializationLib(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB));
-    setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS);
-    setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT);
-    if(org.apache.commons.lang.StringUtils.isNotBlank(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS))) {
-      setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_SERDE);
+    t
+        .getSd()
+        .getSerdeInfo()
+        .setSerializationLib(
+            schema
+                .getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB));
+    setSerdeParam(t.getSd().getSerdeInfo(), schema,
+        org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS);
+    setSerdeParam(t.getSd().getSerdeInfo(), schema,
+        org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT);
+    if (org.apache.commons.lang.StringUtils
+        .isNotBlank(schema
+            .getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS))) {
+      setSerdeParam(t.getSd().getSerdeInfo(), schema,
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_SERDE);
     }
     // needed for MetadataTypedColumnSetSerDe and LazySimpleSerDe
-    setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS);
+    setSerdeParam(t.getSd().getSerdeInfo(), schema,
+        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS);
     // needed for LazySimpleSerDe
-    setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMN_TYPES);
+    setSerdeParam(t.getSd().getSerdeInfo(), schema,
+        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMN_TYPES);
     // needed for DynamicSerDe
-    setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.serde.Constants.SERIALIZATION_DDL);
-      
-    String colstr = schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS);
-    List<FieldSchema>  fields = new ArrayList<FieldSchema>();
-    if(colstr != null) {
-      String[] cols =  colstr.split(",");
+    setSerdeParam(t.getSd().getSerdeInfo(), schema,
+        org.apache.hadoop.hive.serde.Constants.SERIALIZATION_DDL);
+
+    String colstr = schema
+        .getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS);
+    List<FieldSchema> fields = new ArrayList<FieldSchema>();
+    if (colstr != null) {
+      String[] cols = colstr.split(",");
       for (String colName : cols) {
-        FieldSchema col = new FieldSchema(colName, org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "'default'");
+        FieldSchema col = new FieldSchema(colName,
+            org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME,
+            "'default'");
         fields.add(col);
       }
-    } 
-    
-    if(fields.size() == 0) {
+    }
+
+    if (fields.size() == 0) {
       // get the fields from serde
       try {
-        fields = getFieldsFromDeserializer(t.getTableName(), getDeserializer(conf, schema));
+        fields = getFieldsFromDeserializer(t.getTableName(), getDeserializer(
+            conf, schema));
       } catch (SerDeException e) {
         LOG.error(StringUtils.stringifyException(e));
         throw new MetaException("Invalid serde or schema. " + e.getMessage());
       }
     }
     t.getSd().setCols(fields);
-    
+
     t.setOwner(schema.getProperty("creator"));
-    
+
     // remove all the used up parameters to find out the remaining parameters
     schema.remove(Constants.META_TABLE_NAME);
     schema.remove(Constants.META_TABLE_LOCATION);
@@ -375,80 +439,105 @@
     schema.remove(Constants.META_TABLE_SERDE);
     schema.remove(Constants.META_TABLE_COLUMNS);
     schema.remove(Constants.META_TABLE_COLUMN_TYPES);
-    
+
     // add the remaining unknown parameters to the table's parameters
     t.setParameters(new HashMap<String, String>());
-    for(Entry<Object, Object> e : schema.entrySet()) {
-     t.getParameters().put(e.getKey().toString(), e.getValue().toString()); 
+    for (Entry<Object, Object> e : schema.entrySet()) {
+      t.getParameters().put(e.getKey().toString(), e.getValue().toString());
     }
 
     return t;
   }
 
-  public static void setSerdeParam(SerDeInfo sdi, Properties schema, String param) {
+  public static void setSerdeParam(SerDeInfo sdi, Properties schema,
+      String param) {
     String val = schema.getProperty(param);
-    if(org.apache.commons.lang.StringUtils.isNotBlank(val)) {
+    if (org.apache.commons.lang.StringUtils.isNotBlank(val)) {
       sdi.getParameters().put(param, val);
     }
   }
 
-  static HashMap<String, String> typeToThriftTypeMap; 
+  static HashMap<String, String> typeToThriftTypeMap;
   static {
     typeToThriftTypeMap = new HashMap<String, String>();
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.BOOLEAN_TYPE_NAME, "bool");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.TINYINT_TYPE_NAME, "byte");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.SMALLINT_TYPE_NAME, "i16");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.INT_TYPE_NAME, "i32");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.BIGINT_TYPE_NAME, "i64");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.DOUBLE_TYPE_NAME, "double");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.FLOAT_TYPE_NAME, "float");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.LIST_TYPE_NAME, "list");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.MAP_TYPE_NAME, "map");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "string");
-    // These 3 types are not supported yet. 
-    // We should define a complex type date in thrift that contains a single int member, and DynamicSerDe
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.BOOLEAN_TYPE_NAME, "bool");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.TINYINT_TYPE_NAME, "byte");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.SMALLINT_TYPE_NAME, "i16");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.INT_TYPE_NAME, "i32");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.BIGINT_TYPE_NAME, "i64");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.DOUBLE_TYPE_NAME, "double");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.FLOAT_TYPE_NAME, "float");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.LIST_TYPE_NAME, "list");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.MAP_TYPE_NAME, "map");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "string");
+    // These 3 types are not supported yet.
+    // We should define a complex type date in thrift that contains a single int
+    // member, and DynamicSerDe
     // should convert it to date type at runtime.
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.DATE_TYPE_NAME, "date");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.DATETIME_TYPE_NAME, "datetime");
-    typeToThriftTypeMap.put(org.apache.hadoop.hive.serde.Constants.TIMESTAMP_TYPE_NAME, "timestamp");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.DATE_TYPE_NAME, "date");
+    typeToThriftTypeMap.put(
+        org.apache.hadoop.hive.serde.Constants.DATETIME_TYPE_NAME, "datetime");
+    typeToThriftTypeMap
+        .put(org.apache.hadoop.hive.serde.Constants.TIMESTAMP_TYPE_NAME,
+            "timestamp");
   }
-  /** Convert type to ThriftType.  We do that by tokenizing the type and convert each token.
+
+  /**
+   * Convert type to ThriftType. We do that by tokenizing the type and convert
+   * each token.
    */
   public static String typeToThriftType(String type) {
     StringBuilder thriftType = new StringBuilder();
     int last = 0;
     boolean lastAlphaDigit = Character.isLetterOrDigit(type.charAt(last));
-    for(int i=1; i<=type.length(); i++) {
-      if (i == type.length() || Character.isLetterOrDigit(type.charAt(i)) != lastAlphaDigit) {
+    for (int i = 1; i <= type.length(); i++) {
+      if (i == type.length()
+          || Character.isLetterOrDigit(type.charAt(i)) != lastAlphaDigit) {
         String token = type.substring(last, i);
         last = i;
         String thriftToken = typeToThriftTypeMap.get(token);
-        thriftType.append(thriftToken == null? token : thriftToken);
+        thriftType.append(thriftToken == null ? token : thriftToken);
         lastAlphaDigit = !lastAlphaDigit;
-      }         
+      }
     }
     return thriftType.toString();
   }
-  /** 
+
+  /**
    * Convert FieldSchemas to Thrift DDL + column names and column types
    * 
-   * @param structName The name of the table
-   * @param fieldSchemas List of fields along with their schemas
-   * @return String containing "Thrift DDL#comma-separated-column-names#colon-separated-columntypes
-   *         Example: "struct result { a string, map<int,string> b}#a,b#string:map<int,string>"
+   * @param structName
+   *          The name of the table
+   * @param fieldSchemas
+   *          List of fields along with their schemas
+   * @return String containing "Thrift
+   *         DDL#comma-separated-column-names#colon-separated-columntypes
+   *         Example:
+   *         "struct result { a string, map<int,string> b}#a,b#string:map<int,string>"
    */
-  public static String getFullDDLFromFieldSchema(String structName, List<FieldSchema> fieldSchemas) {
+  public static String getFullDDLFromFieldSchema(String structName,
+      List<FieldSchema> fieldSchemas) {
     StringBuilder ddl = new StringBuilder();
     ddl.append(getDDLFromFieldSchema(structName, fieldSchemas));
     ddl.append('#');
     StringBuilder colnames = new StringBuilder();
     StringBuilder coltypes = new StringBuilder();
     boolean first = true;
-    for (FieldSchema col: fieldSchemas) {
+    for (FieldSchema col : fieldSchemas) {
       if (first) {
         first = false;
-      }
-      else {
+      } else {
         colnames.append(',');
         coltypes.append(':');
       }
@@ -460,16 +549,18 @@
     ddl.append(coltypes);
     return ddl.toString();
   }
-  
-  /** Convert FieldSchemas to Thrift DDL.
+
+  /**
+   * Convert FieldSchemas to Thrift DDL.
    */
-  public static String getDDLFromFieldSchema(String structName, List<FieldSchema> fieldSchemas) {
+  public static String getDDLFromFieldSchema(String structName,
+      List<FieldSchema> fieldSchemas) {
     StringBuilder ddl = new StringBuilder();
     ddl.append("struct ");
     ddl.append(structName);
     ddl.append(" { ");
     boolean first = true;
-    for (FieldSchema col: fieldSchemas) {
+    for (FieldSchema col : fieldSchemas) {
       if (first) {
         first = false;
       } else {
@@ -480,19 +571,24 @@
       ddl.append(col.getName());
     }
     ddl.append("}");
-    
+
     LOG.info("DDL: " + ddl);
     return ddl.toString();
   }
-  
-  public static Properties getSchema(org.apache.hadoop.hive.metastore.api.Table table) {
-  	return MetaStoreUtils.getSchema(table.getSd(),table.getSd(), table.getParameters(), table.getTableName(), table.getPartitionKeys());
-  }
-  
-  public static Properties getSchema(org.apache.hadoop.hive.metastore.api.Partition part, org.apache.hadoop.hive.metastore.api.Table table) {
-  	return MetaStoreUtils.getSchema(part.getSd(), table.getSd(), table.getParameters(), table.getTableName(), table.getPartitionKeys());
+
+  public static Properties getSchema(
+      org.apache.hadoop.hive.metastore.api.Table table) {
+    return MetaStoreUtils.getSchema(table.getSd(), table.getSd(), table
+        .getParameters(), table.getTableName(), table.getPartitionKeys());
   }
-  
+
+  public static Properties getSchema(
+      org.apache.hadoop.hive.metastore.api.Partition part,
+      org.apache.hadoop.hive.metastore.api.Table table) {
+    return MetaStoreUtils.getSchema(part.getSd(), table.getSd(), table
+        .getParameters(), table.getTableName(), table.getPartitionKeys());
+  }
+
   public static Properties getSchema(
       org.apache.hadoop.hive.metastore.api.StorageDescriptor sd,
       org.apache.hadoop.hive.metastore.api.StorageDescriptor tblsd,
@@ -500,31 +596,47 @@
       List<FieldSchema> partitionKeys) {
     Properties schema = new Properties();
     String inputFormat = sd.getInputFormat();
-    if(inputFormat == null || inputFormat.length() == 0) {
-      inputFormat = org.apache.hadoop.mapred.SequenceFileInputFormat.class.getName();
-    }
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_INPUT_FORMAT, inputFormat);
+    if (inputFormat == null || inputFormat.length() == 0) {
+      inputFormat = org.apache.hadoop.mapred.SequenceFileInputFormat.class
+          .getName();
+    }
+    schema.setProperty(
+        org.apache.hadoop.hive.metastore.api.Constants.FILE_INPUT_FORMAT,
+        inputFormat);
     String outputFormat = sd.getOutputFormat();
-    if(outputFormat == null || outputFormat.length() == 0) {
-      outputFormat = org.apache.hadoop.mapred.SequenceFileOutputFormat.class.getName();
-    }
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT, outputFormat);
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, tableName);
-    if(sd.getLocation() != null) {
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_LOCATION, sd.getLocation());
-    }
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_COUNT, Integer.toString(sd.getNumBuckets()));
+    if (outputFormat == null || outputFormat.length() == 0) {
+      outputFormat = org.apache.hadoop.mapred.SequenceFileOutputFormat.class
+          .getName();
+    }
+    schema.setProperty(
+        org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT,
+        outputFormat);
+    schema.setProperty(
+        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+        tableName);
+    if (sd.getLocation() != null) {
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_LOCATION,
+          sd.getLocation());
+    }
+    schema.setProperty(
+        org.apache.hadoop.hive.metastore.api.Constants.BUCKET_COUNT, Integer
+            .toString(sd.getNumBuckets()));
     if (sd.getBucketCols().size() > 0) {
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_FIELD_NAME, sd.getBucketCols().get(0));
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.BUCKET_FIELD_NAME, sd
+              .getBucketCols().get(0));
     }
     schema.putAll(sd.getSerdeInfo().getParameters());
-    if(sd.getSerdeInfo().getSerializationLib() != null) {
-      schema.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB, sd.getSerdeInfo().getSerializationLib());
+    if (sd.getSerdeInfo().getSerializationLib() != null) {
+      schema.setProperty(
+          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB, sd
+              .getSerdeInfo().getSerializationLib());
     }
     StringBuilder colNameBuf = new StringBuilder();
     StringBuilder colTypeBuf = new StringBuilder();
     boolean first = true;
-    for (FieldSchema col: tblsd.getCols()) {
+    for (FieldSchema col : tblsd.getCols()) {
       if (!first) {
         colNameBuf.append(",");
         colTypeBuf.append(":");
@@ -535,39 +647,49 @@
     }
     String colNames = colNameBuf.toString();
     String colTypes = colTypeBuf.toString();
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS, colNames);
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMN_TYPES, colTypes);
-    schema.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_DDL, 
+    schema.setProperty(
+        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS,
+        colNames);
+    schema.setProperty(
+        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMN_TYPES,
+        colTypes);
+    schema.setProperty(
+        org.apache.hadoop.hive.serde.Constants.SERIALIZATION_DDL,
         getDDLFromFieldSchema(tableName, sd.getCols()));
-    
+
     String partString = "";
     String partStringSep = "";
     for (FieldSchema partKey : partitionKeys) {
       partString = partString.concat(partStringSep);
       partString = partString.concat(partKey.getName());
-      if(partStringSep.length() == 0) {
+      if (partStringSep.length() == 0) {
         partStringSep = "/";
       }
     }
-    if(partString.length() > 0) {
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS, partString);
+    if (partString.length() > 0) {
+      schema
+          .setProperty(
+              org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS,
+              partString);
     }
-    
-		if (parameters != null) {
-			for(Entry<String, String> e: parameters.entrySet()) {
-	      schema.setProperty(e.getKey(), e.getValue());
-	    }
-		}
-    
+
+    if (parameters != null) {
+      for (Entry<String, String> e : parameters.entrySet()) {
+        schema.setProperty(e.getKey(), e.getValue());
+      }
+    }
+
     return schema;
   }
-  
-  /** Convert FieldSchemas to columnNames.
+
+  /**
+   * Convert FieldSchemas to columnNames.
    */
-  public static String getColumnNamesFromFieldSchema(List<FieldSchema> fieldSchemas) {
+  public static String getColumnNamesFromFieldSchema(
+      List<FieldSchema> fieldSchemas) {
     StringBuilder sb = new StringBuilder();
-    for (int i=0; i<fieldSchemas.size(); i++) {
-      if (i>0) {
+    for (int i = 0; i < fieldSchemas.size(); i++) {
+      if (i > 0) {
         sb.append(",");
       }
       sb.append(fieldSchemas.get(i).getName());
@@ -575,19 +697,21 @@
     return sb.toString();
   }
 
-  /** Convert FieldSchemas to columnTypes.
+  /**
+   * Convert FieldSchemas to columnTypes.
    */
-  public static String getColumnTypesFromFieldSchema(List<FieldSchema> fieldSchemas) {
+  public static String getColumnTypesFromFieldSchema(
+      List<FieldSchema> fieldSchemas) {
     StringBuilder sb = new StringBuilder();
-    for (int i=0; i<fieldSchemas.size(); i++) {
-      if (i>0) {
+    for (int i = 0; i < fieldSchemas.size(); i++) {
+      if (i > 0) {
         sb.append(",");
       }
       sb.append(fieldSchemas.get(i).getType());
     }
     return sb.toString();
   }
-  
+
   public static void makeDir(Path path, HiveConf hiveConf) throws MetaException {
     FileSystem fs;
     try {
@@ -597,19 +721,23 @@
       }
     } catch (IOException e) {
       throw new MetaException("Unable to : " + path);
-    } 
+    }
 
   }
 
   /**
    * Catches exceptions that can't be handled and bundles them to MetaException
+   * 
    * @param e
    * @throws MetaException
    */
   static void logAndThrowMetaException(Exception e) throws MetaException {
-    LOG.error("Got exception: " + e.getClass().getName() + " " + e.getMessage());
+    LOG
+        .error("Got exception: " + e.getClass().getName() + " "
+            + e.getMessage());
     LOG.error(StringUtils.stringifyException(e));
-    throw new MetaException("Got exception: " + e.getClass().getName() + " " + e.getMessage());
+    throw new MetaException("Got exception: " + e.getClass().getName() + " "
+        + e.getMessage());
   }
 
   /**
@@ -619,60 +747,64 @@
    * @throws SerDeException
    * @throws MetaException
    */
-  public static List<FieldSchema> getFieldsFromDeserializer(String tableName, Deserializer deserializer) throws SerDeException, MetaException {
+  public static List<FieldSchema> getFieldsFromDeserializer(String tableName,
+      Deserializer deserializer) throws SerDeException, MetaException {
     ObjectInspector oi = deserializer.getObjectInspector();
-    String [] names = tableName.split("\\.");
-    String last_name = names[names.length-1];
-    for(int i = 1; i < names.length; i++) {
+    String[] names = tableName.split("\\.");
+    String last_name = names[names.length - 1];
+    for (int i = 1; i < names.length; i++) {
 
       if (oi instanceof StructObjectInspector) {
-        StructObjectInspector soi = (StructObjectInspector)oi;
+        StructObjectInspector soi = (StructObjectInspector) oi;
         StructField sf = soi.getStructFieldRef(names[i]);
         if (sf == null) {
           throw new MetaException("Invalid Field " + names[i]);
         } else {
           oi = sf.getFieldObjectInspector();
         }
-      }
-      else if (oi instanceof ListObjectInspector && names[i].equalsIgnoreCase("$elem$")) {
-        ListObjectInspector loi = (ListObjectInspector)oi;
+      } else if (oi instanceof ListObjectInspector
+          && names[i].equalsIgnoreCase("$elem$")) {
+        ListObjectInspector loi = (ListObjectInspector) oi;
         oi = loi.getListElementObjectInspector();
-      }
-      else if (oi instanceof MapObjectInspector && names[i].equalsIgnoreCase("$key$")) {
-        MapObjectInspector moi = (MapObjectInspector)oi;
+      } else if (oi instanceof MapObjectInspector
+          && names[i].equalsIgnoreCase("$key$")) {
+        MapObjectInspector moi = (MapObjectInspector) oi;
         oi = moi.getMapKeyObjectInspector();
-      }
-      else if (oi instanceof MapObjectInspector && names[i].equalsIgnoreCase("$value$")) {
-        MapObjectInspector moi = (MapObjectInspector)oi;
+      } else if (oi instanceof MapObjectInspector
+          && names[i].equalsIgnoreCase("$value$")) {
+        MapObjectInspector moi = (MapObjectInspector) oi;
         oi = moi.getMapValueObjectInspector();
-       }
-      else {
+      } else {
         throw new MetaException("Unknown type for " + names[i]);
       }
     }
 
-    ArrayList<FieldSchema> str_fields = new ArrayList<FieldSchema>(); 
+    ArrayList<FieldSchema> str_fields = new ArrayList<FieldSchema>();
     // rules on how to recurse the ObjectInspector based on its type
     if (oi.getCategory() != Category.STRUCT) {
-      str_fields.add(new FieldSchema(last_name, oi.getTypeName(), "from deserializer"));
+      str_fields.add(new FieldSchema(last_name, oi.getTypeName(),
+          "from deserializer"));
     } else {
-      List<? extends StructField> fields = ((StructObjectInspector)oi).getAllStructFieldRefs();
-      for(int i=0; i<fields.size(); i++) {
+      List<? extends StructField> fields = ((StructObjectInspector) oi)
+          .getAllStructFieldRefs();
+      for (int i = 0; i < fields.size(); i++) {
         String fieldName = fields.get(i).getFieldName();
-        String fieldTypeName = fields.get(i).getFieldObjectInspector().getTypeName();
-        str_fields.add(new FieldSchema(fieldName, fieldTypeName, "from deserializer"));
+        String fieldTypeName = fields.get(i).getFieldObjectInspector()
+            .getTypeName();
+        str_fields.add(new FieldSchema(fieldName, fieldTypeName,
+            "from deserializer"));
       }
     }
     return str_fields;
   }
 
   /**
-   * Convert TypeInfo to FieldSchema. 
+   * Convert TypeInfo to FieldSchema.
    */
-  public static FieldSchema getFieldSchemaFromTypeInfo(String fieldName, TypeInfo typeInfo) {
-    return new FieldSchema(
-        fieldName, typeInfo.getTypeName(), "generated by TypeInfoUtils.getFieldSchemaFromTypeInfo"
-    );
+  public static FieldSchema getFieldSchemaFromTypeInfo(String fieldName,
+      TypeInfo typeInfo) {
+    return new FieldSchema(fieldName, typeInfo.getTypeName(),
+        "generated by TypeInfoUtils.getFieldSchemaFromTypeInfo");
   }
-  
+
 }