You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/09/22 23:31:42 UTC

[01/52] [abbrv] hive git commit: HIVE-11300 HBase metastore: Support token and master key methods (gates)

Repository: hive
Updated Branches:
  refs/heads/llap 47187618b -> f84eefa52


http://git-wip-us.apache.org/repos/asf/hive/blob/a310524c/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index 332e30a..ae73feb 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -80,6 +80,8 @@ class HBaseReadWrite {
   @VisibleForTesting final static String PART_TABLE = "HBMS_PARTITIONS";
   @VisibleForTesting final static String ROLE_TABLE = "HBMS_ROLES";
   @VisibleForTesting final static String SD_TABLE = "HBMS_SDS";
+  @VisibleForTesting final static String SECURITY_TABLE = "HBMS_SECURITY";
+  @VisibleForTesting final static String SEQUENCES_TABLE = "HBMS_SEQUENCES";
   @VisibleForTesting final static String TABLE_TABLE = "HBMS_TBLS";
   @VisibleForTesting final static String USER_TO_ROLE_TABLE = "HBMS_USER_TO_ROLE";
   @VisibleForTesting final static byte[] CATALOG_CF = "c".getBytes(HBaseUtils.ENCODING);
@@ -90,7 +92,7 @@ class HBaseReadWrite {
    */
   final static String[] tableNames = { AGGR_STATS_TABLE, DB_TABLE, FUNC_TABLE, GLOBAL_PRIVS_TABLE,
                                        PART_TABLE, USER_TO_ROLE_TABLE, ROLE_TABLE, SD_TABLE,
-                                       TABLE_TABLE  };
+                                       SECURITY_TABLE, SEQUENCES_TABLE, TABLE_TABLE};
   final static Map<String, List<byte[]>> columnFamilies =
       new HashMap<String, List<byte[]>> (tableNames.length);
 
@@ -103,6 +105,8 @@ class HBaseReadWrite {
     columnFamilies.put(USER_TO_ROLE_TABLE, Arrays.asList(CATALOG_CF));
     columnFamilies.put(ROLE_TABLE, Arrays.asList(CATALOG_CF));
     columnFamilies.put(SD_TABLE, Arrays.asList(CATALOG_CF));
+    columnFamilies.put(SECURITY_TABLE, Arrays.asList(CATALOG_CF));
+    columnFamilies.put(SEQUENCES_TABLE, Arrays.asList(CATALOG_CF));
     columnFamilies.put(TABLE_TABLE, Arrays.asList(CATALOG_CF, STATS_CF));
   }
 
@@ -110,12 +114,16 @@ class HBaseReadWrite {
    * Stores the bloom filter for the aggregated stats, to determine what partitions are in this
    * aggregate.
    */
+  final static byte[] MASTER_KEY_SEQUENCE = "mk".getBytes(HBaseUtils.ENCODING);
   final static byte[] AGGR_STATS_BLOOM_COL = "b".getBytes(HBaseUtils.ENCODING);
   private final static byte[] CATALOG_COL = "c".getBytes(HBaseUtils.ENCODING);
   private final static byte[] ROLES_COL = "roles".getBytes(HBaseUtils.ENCODING);
   private final static byte[] REF_COUNT_COL = "ref".getBytes(HBaseUtils.ENCODING);
+  private final static byte[] DELEGATION_TOKEN_COL = "dt".getBytes(HBaseUtils.ENCODING);
+  private final static byte[] MASTER_KEY_COL = "mk".getBytes(HBaseUtils.ENCODING);
   private final static byte[] AGGR_STATS_STATS_COL = "s".getBytes(HBaseUtils.ENCODING);
   private final static byte[] GLOBAL_PRIVS_KEY = "gp".getBytes(HBaseUtils.ENCODING);
+  private final static byte[] SEQUENCES_KEY = "seq".getBytes(HBaseUtils.ENCODING);
   private final static int TABLES_TO_CACHE = 10;
   // False positives are very bad here because they cause us to invalidate entries we shouldn't.
   // Space used and # of hash functions grows in proportion to ln of num bits so a 10x increase
@@ -226,7 +234,7 @@ class HBaseReadWrite {
     sdHits = new Counter("storage descriptor cache hits");
     sdMisses = new Counter("storage descriptor cache misses");
     sdOverflows = new Counter("storage descriptor cache overflows");
-    counters = new ArrayList<Counter>();
+    counters = new ArrayList<>();
     counters.add(tableHits);
     counters.add(tableMisses);
     counters.add(tableOverflows);
@@ -241,18 +249,16 @@ class HBaseReadWrite {
     // (storage descriptors are shared, so 99% should be the same for a given table)
     int sdsCacheSize = totalCatalogObjectsToCache / 100;
     if (conf.getBoolean(NO_CACHE_CONF, false)) {
-      tableCache = new BogusObjectCache<ObjectPair<String, String>, Table>();
-      sdCache = new BogusObjectCache<ByteArrayWrapper, StorageDescriptor>();
+      tableCache = new BogusObjectCache<>();
+      sdCache = new BogusObjectCache<>();
       partCache = new BogusPartitionCache();
     } else {
-      tableCache = new ObjectCache<ObjectPair<String, String>, Table>(TABLES_TO_CACHE, tableHits,
-          tableMisses, tableOverflows);
-      sdCache = new ObjectCache<ByteArrayWrapper, StorageDescriptor>(sdsCacheSize, sdHits,
-          sdMisses, sdOverflows);
+      tableCache = new ObjectCache<>(TABLES_TO_CACHE, tableHits, tableMisses, tableOverflows);
+      sdCache = new ObjectCache<>(sdsCacheSize, sdHits, sdMisses, sdOverflows);
       partCache = new PartitionCache(totalCatalogObjectsToCache, partHits, partMisses, partOverflows);
     }
     statsCache = StatsCache.getInstance(conf);
-    roleCache = new HashMap<String, HbaseMetastoreProto.RoleGrantInfoList>();
+    roleCache = new HashMap<>();
     entireRoleTableInCache = false;
   }
 
@@ -338,7 +344,7 @@ class HBaseReadWrite {
     }
     Iterator<Result> iter =
         scan(DB_TABLE, CATALOG_CF, CATALOG_COL, filter);
-    List<Database> databases = new ArrayList<Database>();
+    List<Database> databases = new ArrayList<>();
     while (iter.hasNext()) {
       Result result = iter.next();
       databases.add(HBaseUtils.deserializeDatabase(result.getRow(),
@@ -404,7 +410,7 @@ class HBaseReadWrite {
     }
     Iterator<Result> iter =
         scan(FUNC_TABLE, keyPrefix, HBaseUtils.getEndPrefix(keyPrefix), CATALOG_CF, CATALOG_COL, filter);
-    List<Function> functions = new ArrayList<Function>();
+    List<Function> functions = new ArrayList<>();
     while (iter.hasNext()) {
       Result result = iter.next();
       functions.add(HBaseUtils.deserializeFunction(result.getRow(),
@@ -489,8 +495,8 @@ class HBaseReadWrite {
    */
    List<Partition> getPartitions(String dbName, String tableName, List<List<String>> partValLists)
        throws IOException {
-     List<Partition> parts = new ArrayList<Partition>(partValLists.size());
-     List<Get> gets = new ArrayList<Get>(partValLists.size());
+     List<Partition> parts = new ArrayList<>(partValLists.size());
+     List<Get> gets = new ArrayList<>(partValLists.size());
      for (List<String> partVals : partValLists) {
        byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName, partVals);
        Get get = new Get(key);
@@ -556,7 +562,7 @@ class HBaseReadWrite {
    * @throws IOException
    */
   void putPartitions(List<Partition> partitions) throws IOException {
-    List<Put> puts = new ArrayList<Put>(partitions.size());
+    List<Put> puts = new ArrayList<>(partitions.size());
     for (Partition partition : partitions) {
       byte[] hash = putStorageDescriptor(partition.getSd());
       byte[][] serialized = HBaseUtils.serializePartition(partition, hash);
@@ -615,8 +621,8 @@ class HBaseReadWrite {
     Collection<Partition> cached = partCache.getAllForTable(dbName, tableName);
     if (cached != null) {
       return maxPartitions < cached.size()
-          ? new ArrayList<Partition>(cached).subList(0, maxPartitions)
-          : new ArrayList<Partition>(cached);
+          ? new ArrayList<>(cached).subList(0, maxPartitions)
+          : new ArrayList<>(cached);
     }
     byte[] keyPrefix = HBaseUtils.buildKeyWithTrailingSeparator(dbName, tableName);
     List<Partition> parts = scanPartitionsWithFilter(keyPrefix, HBaseUtils.getEndPrefix(keyPrefix), -1, null);
@@ -645,7 +651,7 @@ class HBaseReadWrite {
   List<Partition> scanPartitions(String dbName, String tableName, List<String> partVals,
                                  int maxPartitions) throws IOException, NoSuchObjectException {
     // First, build as much of the key as we can so that we make the scan as tight as possible.
-    List<String> keyElements = new ArrayList<String>();
+    List<String> keyElements = new ArrayList<>();
     keyElements.add(dbName);
     keyElements.add(tableName);
 
@@ -712,7 +718,7 @@ class HBaseReadWrite {
 
   List<Partition> scanPartitions(String dbName, String tableName, byte[] keyStart, byte[] keyEnd,
       Filter filter, int maxPartitions) throws IOException, NoSuchObjectException {
-    List<String> keyElements = new ArrayList<String>();
+    List<String> keyElements = new ArrayList<>();
     keyElements.add(dbName);
     keyElements.add(tableName);
 
@@ -780,7 +786,7 @@ class HBaseReadWrite {
       throws IOException {
     Iterator<Result> iter =
         scan(PART_TABLE, startRow, endRow, CATALOG_CF, CATALOG_COL, filter);
-    List<Partition> parts = new ArrayList<Partition>();
+    List<Partition> parts = new ArrayList<>();
     int numToFetch = maxResults < 0 ? Integer.MAX_VALUE : maxResults;
     for (int i = 0; i < numToFetch && iter.hasNext(); i++) {
       Result result = iter.next();
@@ -821,7 +827,7 @@ class HBaseReadWrite {
       throws IOException {
     buildRoleCache();
 
-    Set<String> rolesFound = new HashSet<String>();
+    Set<String> rolesFound = new HashSet<>();
     for (Map.Entry<String, HbaseMetastoreProto.RoleGrantInfoList> e : roleCache.entrySet()) {
       for (HbaseMetastoreProto.RoleGrantInfo giw : e.getValue().getGrantInfoList()) {
         if (HBaseUtils.convertPrincipalTypes(giw.getPrincipalType()) == type &&
@@ -831,8 +837,8 @@ class HBaseReadWrite {
         }
       }
     }
-    List<Role> directRoles = new ArrayList<Role>(rolesFound.size());
-    List<Get> gets = new ArrayList<Get>();
+    List<Role> directRoles = new ArrayList<>(rolesFound.size());
+    List<Get> gets = new ArrayList<>();
     HTableInterface htab = conn.getHBaseTable(ROLE_TABLE);
     for (String roleFound : rolesFound) {
       byte[] key = HBaseUtils.buildKey(roleFound);
@@ -880,7 +886,7 @@ class HBaseReadWrite {
    */
   Set<String> findAllUsersInRole(String roleName) throws IOException {
     // Walk the userToRole table and collect every user that matches this role.
-    Set<String> users = new HashSet<String>();
+    Set<String> users = new HashSet<>();
     Iterator<Result> iter = scan(USER_TO_ROLE_TABLE, CATALOG_CF, CATALOG_COL);
     while (iter.hasNext()) {
       Result result = iter.next();
@@ -907,8 +913,7 @@ class HBaseReadWrite {
   void addPrincipalToRole(String roleName, HbaseMetastoreProto.RoleGrantInfo grantInfo)
       throws IOException, NoSuchObjectException {
     HbaseMetastoreProto.RoleGrantInfoList proto = getRolePrincipals(roleName);
-    List<HbaseMetastoreProto.RoleGrantInfo> rolePrincipals =
-        new ArrayList<HbaseMetastoreProto.RoleGrantInfo>();
+    List<HbaseMetastoreProto.RoleGrantInfo> rolePrincipals = new ArrayList<>();
     if (proto != null) {
       rolePrincipals.addAll(proto.getGrantInfoList());
     }
@@ -937,8 +942,7 @@ class HBaseReadWrite {
       throws NoSuchObjectException, IOException {
     HbaseMetastoreProto.RoleGrantInfoList proto = getRolePrincipals(roleName);
     if (proto == null) return;
-    List<HbaseMetastoreProto.RoleGrantInfo> rolePrincipals =
-        new ArrayList<HbaseMetastoreProto.RoleGrantInfo>();
+    List<HbaseMetastoreProto.RoleGrantInfo> rolePrincipals = new ArrayList<>();
     rolePrincipals.addAll(proto.getGrantInfoList());
 
     for (int i = 0; i < rolePrincipals.size(); i++) {
@@ -976,8 +980,8 @@ class HBaseReadWrite {
     LOG.debug("Building role map for " + userName);
 
     // Second, find every role the user participates in directly.
-    Set<String> rolesToAdd = new HashSet<String>();
-    Set<String> rolesToCheckNext = new HashSet<String>();
+    Set<String> rolesToAdd = new HashSet<>();
+    Set<String> rolesToCheckNext = new HashSet<>();
     for (Map.Entry<String, HbaseMetastoreProto.RoleGrantInfoList> e : roleCache.entrySet()) {
       for (HbaseMetastoreProto.RoleGrantInfo grantInfo : e.getValue().getGrantInfoList()) {
         if (HBaseUtils.convertPrincipalTypes(grantInfo.getPrincipalType()) == PrincipalType.USER &&
@@ -993,7 +997,7 @@ class HBaseReadWrite {
     // Third, find every role the user participates in indirectly (that is, they have been
     // granted into role X and role Y has been granted into role X).
     while (rolesToCheckNext.size() > 0) {
-      Set<String> tmpRolesToCheckNext = new HashSet<String>();
+      Set<String> tmpRolesToCheckNext = new HashSet<>();
       for (String roleName : rolesToCheckNext) {
         HbaseMetastoreProto.RoleGrantInfoList grantInfos = roleCache.get(roleName);
         if (grantInfos == null) continue;  // happens when a role contains no grants
@@ -1010,7 +1014,7 @@ class HBaseReadWrite {
     }
 
     byte[] key = HBaseUtils.buildKey(userName);
-    byte[] serialized = HBaseUtils.serializeRoleList(new ArrayList<String>(rolesToAdd));
+    byte[] serialized = HBaseUtils.serializeRoleList(new ArrayList<>(rolesToAdd));
     store(USER_TO_ROLE_TABLE, key, CATALOG_CF, CATALOG_COL, serialized);
   }
 
@@ -1022,12 +1026,11 @@ class HBaseReadWrite {
   void removeRoleGrants(String roleName) throws IOException {
     buildRoleCache();
 
-    List<Put> puts = new ArrayList<Put>();
+    List<Put> puts = new ArrayList<>();
     // First, walk the role table and remove any references to this role
     for (Map.Entry<String, HbaseMetastoreProto.RoleGrantInfoList> e : roleCache.entrySet()) {
       boolean madeAChange = false;
-      List<HbaseMetastoreProto.RoleGrantInfo> rgil =
-          new ArrayList<HbaseMetastoreProto.RoleGrantInfo>();
+      List<HbaseMetastoreProto.RoleGrantInfo> rgil = new ArrayList<>();
       rgil.addAll(e.getValue().getGrantInfoList());
       for (int i = 0; i < rgil.size(); i++) {
         if (HBaseUtils.convertPrincipalTypes(rgil.get(i).getPrincipalType()) == PrincipalType.ROLE &&
@@ -1066,7 +1069,7 @@ class HBaseReadWrite {
     // Now, walk the db table
     puts.clear();
     List<Database> dbs = scanDatabases(null);
-    if (dbs == null) dbs = new ArrayList<Database>(); // rare, but can happen
+    if (dbs == null) dbs = new ArrayList<>(); // rare, but can happen
     for (Database db : dbs) {
       if (db.getPrivileges() != null &&
           db.getPrivileges().getRolePrivileges() != null &&
@@ -1130,7 +1133,7 @@ class HBaseReadWrite {
    */
   List<Role> scanRoles() throws IOException {
     Iterator<Result> iter = scan(ROLE_TABLE, CATALOG_CF, CATALOG_COL);
-    List<Role> roles = new ArrayList<Role>();
+    List<Role> roles = new ArrayList<>();
     while (iter.hasNext()) {
       Result result = iter.next();
       roles.add(HBaseUtils.deserializeRole(result.getRow(),
@@ -1199,11 +1202,11 @@ class HBaseReadWrite {
   List<Table> getTables(String dbName, List<String> tableNames) throws IOException {
     // I could implement getTable in terms of this method.  But it is such a core function
     // that I don't want to slow it down for the much less common fetching of multiple tables.
-    List<Table> results = new ArrayList<Table>(tableNames.size());
+    List<Table> results = new ArrayList<>(tableNames.size());
     ObjectPair<String, String>[] hashKeys = new ObjectPair[tableNames.size()];
     boolean atLeastOneMissing = false;
     for (int i = 0; i < tableNames.size(); i++) {
-      hashKeys[i] = new ObjectPair<String, String>(dbName, tableNames.get(i));
+      hashKeys[i] = new ObjectPair<>(dbName, tableNames.get(i));
       // The result may be null, but we still want to add it so that we have a slot in the list
       // for it.
       results.add(tableCache.get(hashKeys[i]));
@@ -1212,7 +1215,7 @@ class HBaseReadWrite {
     if (!atLeastOneMissing) return results;
 
     // Now build a single get that will fetch the remaining tables
-    List<Get> gets = new ArrayList<Get>();
+    List<Get> gets = new ArrayList<>();
     HTableInterface htab = conn.getHBaseTable(TABLE_TABLE);
     for (int i = 0; i < tableNames.size(); i++) {
       if (results.get(i) != null) continue;
@@ -1261,7 +1264,7 @@ class HBaseReadWrite {
     Iterator<Result> iter =
         scan(TABLE_TABLE, keyPrefix, HBaseUtils.getEndPrefix(keyPrefix),
             CATALOG_CF, CATALOG_COL, filter);
-    List<Table> tables = new ArrayList<Table>();
+    List<Table> tables = new ArrayList<>();
     while (iter.hasNext()) {
       Result result = iter.next();
       HBaseUtils.StorageDescriptorParts sdParts =
@@ -1284,7 +1287,7 @@ class HBaseReadWrite {
     byte[] hash = putStorageDescriptor(table.getSd());
     byte[][] serialized = HBaseUtils.serializeTable(table, hash);
     store(TABLE_TABLE, serialized[0], CATALOG_CF, CATALOG_COL, serialized[1]);
-    tableCache.put(new ObjectPair<String, String>(table.getDbName(), table.getTableName()), table);
+    tableCache.put(new ObjectPair<>(table.getDbName(), table.getTableName()), table);
   }
 
   /**
@@ -1323,7 +1326,7 @@ class HBaseReadWrite {
 
   private void deleteTable(String dbName, String tableName, boolean decrementRefCnt)
       throws IOException {
-    tableCache.remove(new ObjectPair<String, String>(dbName, tableName));
+    tableCache.remove(new ObjectPair<>(dbName, tableName));
     if (decrementRefCnt) {
       // Find the table so I can get the storage descriptor and drop it
       Table t = getTable(dbName, tableName, false);
@@ -1335,7 +1338,7 @@ class HBaseReadWrite {
 
   private Table getTable(String dbName, String tableName, boolean populateCache)
       throws IOException {
-    ObjectPair<String, String> hashKey = new ObjectPair<String, String>(dbName, tableName);
+    ObjectPair<String, String> hashKey = new ObjectPair<>(dbName, tableName);
     Table cached = tableCache.get(hashKey);
     if (cached != null) return cached;
     byte[] key = HBaseUtils.buildKey(dbName, tableName);
@@ -1623,6 +1626,7 @@ class HBaseReadWrite {
     byte[] serialized = read(AGGR_STATS_TABLE, key, CATALOG_CF, AGGR_STATS_STATS_COL);
     if (serialized == null) return null;
     return HBaseUtils.deserializeAggrStats(serialized);
+
   }
 
   /**
@@ -1696,6 +1700,134 @@ class HBaseReadWrite {
   }
 
   /**********************************************************************************************
+   * Security related methods
+   *********************************************************************************************/
+
+  /**
+   * Fetch a delegation token
+   * @param tokId identifier of the token to fetch
+   * @return the delegation token, or null if there is no such delegation token
+   * @throws IOException
+   */
+  String getDelegationToken(String tokId) throws IOException {
+    byte[] key = HBaseUtils.buildKey(tokId);
+    byte[] serialized = read(SECURITY_TABLE, key, CATALOG_CF, DELEGATION_TOKEN_COL);
+    if (serialized == null) return null;
+    return HBaseUtils.deserializeDelegationToken(serialized);
+  }
+
+  /**
+   * Get all delegation token ids
+   * @return list of all delegation token identifiers
+   * @throws IOException
+   */
+  List<String> scanDelegationTokenIdentifiers() throws IOException {
+    Iterator<Result> iter = scan(SECURITY_TABLE, CATALOG_CF, DELEGATION_TOKEN_COL);
+    List<String> ids = new ArrayList<>();
+    while (iter.hasNext()) {
+      Result result = iter.next();
+      byte[] serialized = result.getValue(CATALOG_CF, DELEGATION_TOKEN_COL);
+      if (serialized != null) {
+        // Don't deserialize the value, as what we're after is the key.  We just had to check the
+        // value wasn't null in order to check this is a record with a delegation token and not a
+        // master key.
+        ids.add(new String(result.getRow(), HBaseUtils.ENCODING));
+
+      }
+    }
+    return ids;
+  }
+
+  /**
+   * Store a delegation token
+   * @param tokId token id
+   * @param token delegation token to store
+   * @throws IOException
+   */
+  void putDelegationToken(String tokId, String token) throws IOException {
+    byte[][] serialized = HBaseUtils.serializeDelegationToken(tokId, token);
+    store(SECURITY_TABLE, serialized[0], CATALOG_CF, DELEGATION_TOKEN_COL, serialized[1]);
+  }
+
+  /**
+   * Delete a delegation token
+   * @param tokId identifier of token to drop
+   * @throws IOException
+   */
+  void deleteDelegationToken(String tokId) throws IOException {
+    byte[] key = HBaseUtils.buildKey(tokId);
+    delete(SECURITY_TABLE, key, CATALOG_CF, DELEGATION_TOKEN_COL);
+  }
+
+  /**
+   * Fetch a master key
+   * @param seqNo sequence number of the master key
+   * @return the master key, or null if there is no such master key
+   * @throws IOException
+   */
+  String getMasterKey(Integer seqNo) throws IOException {
+    byte[] key = HBaseUtils.buildKey(seqNo.toString());
+    byte[] serialized = read(SECURITY_TABLE, key, CATALOG_CF, MASTER_KEY_COL);
+    if (serialized == null) return null;
+    return HBaseUtils.deserializeMasterKey(serialized);
+  }
+
+  /**
+   * Get all master keys
+   * @return list of all master keys
+   * @throws IOException
+   */
+  List<String> scanMasterKeys() throws IOException {
+    Iterator<Result> iter = scan(SECURITY_TABLE, CATALOG_CF, MASTER_KEY_COL);
+    List<String> keys = new ArrayList<>();
+    while (iter.hasNext()) {
+      Result result = iter.next();
+      byte[] serialized = result.getValue(CATALOG_CF, MASTER_KEY_COL);
+      if (serialized != null) {
+        keys.add(HBaseUtils.deserializeMasterKey(serialized));
+
+      }
+    }
+    return keys;
+  }
+
+  /**
+   * Store a master key
+   * @param seqNo sequence number
+   * @param key master key to store
+   * @throws IOException
+   */
+  void putMasterKey(Integer seqNo, String key) throws IOException {
+    byte[][] serialized = HBaseUtils.serializeMasterKey(seqNo, key);
+    store(SECURITY_TABLE, serialized[0], CATALOG_CF, MASTER_KEY_COL, serialized[1]);
+  }
+
+  /**
+   * Delete a master key
+   * @param seqNo sequence number of master key to delete
+   * @throws IOException
+   */
+  void deleteMasterKey(Integer seqNo) throws IOException {
+    byte[] key = HBaseUtils.buildKey(seqNo.toString());
+    delete(SECURITY_TABLE, key, CATALOG_CF, MASTER_KEY_COL);
+  }
+
+  /**********************************************************************************************
+   * Sequence methods
+   *********************************************************************************************/
+
+  long getNextSequence(byte[] sequence) throws IOException {
+    byte[] serialized = read(SEQUENCES_TABLE, SEQUENCES_KEY, CATALOG_CF, sequence);
+    long val = 0;
+    if (serialized != null) {
+      val = Long.valueOf(new String(serialized, HBaseUtils.ENCODING));
+    }
+    byte[] incrSerialized = new Long(val + 1).toString().getBytes(HBaseUtils.ENCODING);
+    store(SEQUENCES_TABLE, SEQUENCES_KEY, CATALOG_CF, sequence, incrSerialized);
+    return val;
+  }
+
+  /**********************************************************************************************
    * Cache methods
    *********************************************************************************************/
 
@@ -1772,8 +1904,7 @@ class HBaseReadWrite {
     htab.delete(d);
   }
 
-  private Iterator<Result> scan(String table, byte[] colFam,
-      byte[] colName) throws IOException {
+  private Iterator<Result> scan(String table, byte[] colFam, byte[] colName) throws IOException {
     return scan(table, null, null, colFam, colName, null);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a310524c/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index 9782859..744070d 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -1613,43 +1613,128 @@ public class HBaseStore implements RawStore {
 
   @Override
   public boolean addToken(String tokenIdentifier, String delegationToken) {
-    throw new UnsupportedOperationException();
+    boolean commit = false;
+    openTransaction();
+    try {
+      getHBase().putDelegationToken(tokenIdentifier, delegationToken);
+      commit = true;
+      return commit; // See HIVE-11302, for now always returning true
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
   public boolean removeToken(String tokenIdentifier) {
-    throw new UnsupportedOperationException();
+    boolean commit = false;
+    openTransaction();
+    try {
+      getHBase().deleteDelegationToken(tokenIdentifier);
+      commit = true;
+      return commit; // See HIVE-11302, for now always returning true
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
   public String getToken(String tokenIdentifier) {
-    throw new UnsupportedOperationException();
+    boolean commit = false;
+    openTransaction();
+    try {
+      String token = getHBase().getDelegationToken(tokenIdentifier);
+      commit = true;
+      return token;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
   public List<String> getAllTokenIdentifiers() {
-    throw new UnsupportedOperationException();
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<String> ids = getHBase().scanDelegationTokenIdentifiers();
+      commit = true;
+      return ids;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
   public int addMasterKey(String key) throws MetaException {
-    throw new UnsupportedOperationException();
+    boolean commit = false;
+    openTransaction();
+    try {
+      long seq = getHBase().getNextSequence(HBaseReadWrite.MASTER_KEY_SEQUENCE);
+      getHBase().putMasterKey((int) seq, key);
+      commit = true;
+      return (int)seq;
+    } catch (IOException e) {
+      LOG.error("Unable to add master key", e);
+      throw new MetaException("Failed adding master key, " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
   public void updateMasterKey(Integer seqNo, String key) throws NoSuchObjectException,
       MetaException {
-    throw new UnsupportedOperationException();
+    boolean commit = false;
+    openTransaction();
+    try {
+      if (getHBase().getMasterKey(seqNo) == null) {
+        throw new NoSuchObjectException("No key found with keyId: " + seqNo);
+      }
+      getHBase().putMasterKey(seqNo, key);
+      commit = true;
+    } catch (IOException e) {
+      LOG.error("Unable to update master key", e);
+      throw new MetaException("Failed updating master key, " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
   public boolean removeMasterKey(Integer keySeq) {
-    throw new UnsupportedOperationException();
+    boolean commit = false;
+    openTransaction();
+    try {
+      getHBase().deleteMasterKey(keySeq);
+      commit = true;
+      return true;  // See HIVE-11302, for now always returning true
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
   public String[] getMasterKeys() {
-    throw new UnsupportedOperationException();
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<String> keys = getHBase().scanMasterKeys();
+      commit = true;
+      return keys.toArray(new String[keys.size()]);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/a310524c/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index 4d57af2..62bb4de 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -104,8 +104,7 @@ class HBaseUtils {
   }
 
   private static HbaseMetastoreProto.Parameters buildParameters(Map<String, String> params) {
-    List<HbaseMetastoreProto.ParameterEntry> entries =
-        new ArrayList<HbaseMetastoreProto.ParameterEntry>();
+    List<HbaseMetastoreProto.ParameterEntry> entries = new ArrayList<>();
     for (Map.Entry<String, String> e : params.entrySet()) {
       entries.add(
           HbaseMetastoreProto.ParameterEntry.newBuilder()
@@ -119,7 +118,7 @@ class HBaseUtils {
   }
 
   private static Map<String, String> buildParameters(HbaseMetastoreProto.Parameters protoParams) {
-    Map<String, String> params = new HashMap<String, String>();
+    Map<String, String> params = new HashMap<>();
     for (HbaseMetastoreProto.ParameterEntry pe : protoParams.getParameterList()) {
       params.put(pe.getKey(), pe.getValue());
     }
@@ -129,8 +128,7 @@ class HBaseUtils {
 
   private static List<HbaseMetastoreProto.PrincipalPrivilegeSetEntry>
   buildPrincipalPrivilegeSetEntry(Map<String, List<PrivilegeGrantInfo>> entries) {
-    List<HbaseMetastoreProto.PrincipalPrivilegeSetEntry> results =
-        new ArrayList<HbaseMetastoreProto.PrincipalPrivilegeSetEntry>();
+    List<HbaseMetastoreProto.PrincipalPrivilegeSetEntry> results = new ArrayList<>();
     for (Map.Entry<String, List<PrivilegeGrantInfo>> entry : entries.entrySet()) {
       results.add(HbaseMetastoreProto.PrincipalPrivilegeSetEntry.newBuilder()
           .setPrincipalName(entry.getKey())
@@ -142,8 +140,7 @@ class HBaseUtils {
 
   private static List<HbaseMetastoreProto.PrivilegeGrantInfo> buildPrivilegeGrantInfo(
       List<PrivilegeGrantInfo> privileges) {
-    List<HbaseMetastoreProto.PrivilegeGrantInfo> results =
-        new ArrayList<HbaseMetastoreProto.PrivilegeGrantInfo>();
+    List<HbaseMetastoreProto.PrivilegeGrantInfo> results = new ArrayList<>();
     for (PrivilegeGrantInfo privilege : privileges) {
       HbaseMetastoreProto.PrivilegeGrantInfo.Builder builder =
           HbaseMetastoreProto.PrivilegeGrantInfo.newBuilder();
@@ -187,8 +184,7 @@ class HBaseUtils {
 
   private static Map<String, List<PrivilegeGrantInfo>> convertPrincipalPrivilegeSetEntries(
       List<HbaseMetastoreProto.PrincipalPrivilegeSetEntry> entries) {
-    Map<String, List<PrivilegeGrantInfo>> map =
-        new HashMap<String, List<PrivilegeGrantInfo>>();
+    Map<String, List<PrivilegeGrantInfo>> map = new HashMap<>();
     for (HbaseMetastoreProto.PrincipalPrivilegeSetEntry entry : entries) {
       map.put(entry.getPrincipalName(), convertPrivilegeGrantInfos(entry.getPrivilegesList()));
     }
@@ -197,7 +193,7 @@ class HBaseUtils {
 
   private static List<PrivilegeGrantInfo> convertPrivilegeGrantInfos(
       List<HbaseMetastoreProto.PrivilegeGrantInfo> privileges) {
-    List<PrivilegeGrantInfo> results = new ArrayList<PrivilegeGrantInfo>();
+    List<PrivilegeGrantInfo> results = new ArrayList<>();
     for (HbaseMetastoreProto.PrivilegeGrantInfo proto : privileges) {
       PrivilegeGrantInfo pgi = new PrivilegeGrantInfo();
       if (proto.hasPrivilege()) pgi.setPrivilege(proto.getPrivilege());
@@ -316,7 +312,7 @@ class HBaseUtils {
 
   static List<String> deserializeRoleList(byte[] value) throws InvalidProtocolBufferException {
     HbaseMetastoreProto.RoleList proto = HbaseMetastoreProto.RoleList.parseFrom(value);
-    return new ArrayList<String>(proto.getRoleList());
+    return new ArrayList<>(proto.getRoleList());
   }
 
   /**
@@ -491,7 +487,7 @@ class HBaseUtils {
 
   private static List<FieldSchema>
   convertFieldSchemaListFromProto(List<HbaseMetastoreProto.FieldSchema> protoList) {
-    List<FieldSchema> schemas = new ArrayList<FieldSchema>(protoList.size());
+    List<FieldSchema> schemas = new ArrayList<>(protoList.size());
     for (HbaseMetastoreProto.FieldSchema proto : protoList) {
       schemas.add(new FieldSchema(proto.getName(), proto.getType(),
           proto.hasComment() ? proto.getComment() : null));
@@ -501,8 +497,7 @@ class HBaseUtils {
 
   private static List<HbaseMetastoreProto.FieldSchema>
   convertFieldSchemaListToProto(List<FieldSchema> schemas) {
-    List<HbaseMetastoreProto.FieldSchema> protoList =
-        new ArrayList<HbaseMetastoreProto.FieldSchema>(schemas.size());
+    List<HbaseMetastoreProto.FieldSchema> protoList = new ArrayList<>(schemas.size());
     for (FieldSchema fs : schemas) {
       HbaseMetastoreProto.FieldSchema.Builder builder =
           HbaseMetastoreProto.FieldSchema.newBuilder();
@@ -552,8 +547,7 @@ class HBaseUtils {
     }
     if (sd.getSortCols() != null) {
       List<Order> orders = sd.getSortCols();
-      List<HbaseMetastoreProto.StorageDescriptor.Order> protoList =
-          new ArrayList<HbaseMetastoreProto.StorageDescriptor.Order>(orders.size());
+      List<HbaseMetastoreProto.StorageDescriptor.Order> protoList = new ArrayList<>(orders.size());
       for (Order order : orders) {
         protoList.add(HbaseMetastoreProto.StorageDescriptor.Order.newBuilder()
             .setColumnName(order.getCol())
@@ -625,7 +619,7 @@ class HBaseUtils {
         md.update(serde.getSerializationLib().getBytes(ENCODING));
       }
       if (serde.getParameters() != null) {
-        SortedMap<String, String> params = new TreeMap<String, String>(serde.getParameters());
+        SortedMap<String, String> params = new TreeMap<>(serde.getParameters());
         for (Map.Entry<String, String> param : params.entrySet()) {
           md.update(param.getKey().getBytes(ENCODING));
           md.update(param.getValue().getBytes(ENCODING));
@@ -633,11 +627,11 @@ class HBaseUtils {
       }
     }
     if (sd.getBucketCols() != null) {
-      SortedSet<String> bucketCols = new TreeSet<String>(sd.getBucketCols());
+      SortedSet<String> bucketCols = new TreeSet<>(sd.getBucketCols());
       for (String bucket : bucketCols) md.update(bucket.getBytes(ENCODING));
     }
     if (sd.getSortCols() != null) {
-      SortedSet<Order> orders = new TreeSet<Order>(sd.getSortCols());
+      SortedSet<Order> orders = new TreeSet<>(sd.getSortCols());
       for (Order order : orders) {
         md.update(order.getCol().getBytes(ENCODING));
         md.update(Integer.toString(order.getOrder()).getBytes(ENCODING));
@@ -646,21 +640,21 @@ class HBaseUtils {
     if (sd.getSkewedInfo() != null) {
       SkewedInfo skewed = sd.getSkewedInfo();
       if (skewed.getSkewedColNames() != null) {
-        SortedSet<String> colnames = new TreeSet<String>(skewed.getSkewedColNames());
+        SortedSet<String> colnames = new TreeSet<>(skewed.getSkewedColNames());
         for (String colname : colnames) md.update(colname.getBytes(ENCODING));
       }
       if (skewed.getSkewedColValues() != null) {
-        SortedSet<String> sortedOuterList = new TreeSet<String>();
+        SortedSet<String> sortedOuterList = new TreeSet<>();
         for (List<String> innerList : skewed.getSkewedColValues()) {
-          SortedSet<String> sortedInnerList = new TreeSet<String>(innerList);
+          SortedSet<String> sortedInnerList = new TreeSet<>(innerList);
           sortedOuterList.add(StringUtils.join(sortedInnerList, "."));
         }
         for (String colval : sortedOuterList) md.update(colval.getBytes(ENCODING));
       }
       if (skewed.getSkewedColValueLocationMaps() != null) {
-        SortedMap<String, String> sortedMap = new TreeMap<String, String>();
+        SortedMap<String, String> sortedMap = new TreeMap<>();
         for (Map.Entry<List<String>, String> smap : skewed.getSkewedColValueLocationMaps().entrySet()) {
-          SortedSet<String> sortedKey = new TreeSet<String>(smap.getKey());
+          SortedSet<String> sortedKey = new TreeSet<>(smap.getKey());
           sortedMap.put(StringUtils.join(sortedKey, "."), smap.getValue());
         }
         for (Map.Entry<String, String> e : sortedMap.entrySet()) {
@@ -690,8 +684,8 @@ class HBaseUtils {
       serde.setParameters(buildParameters(proto.getSerdeInfo().getParameters()));
       sd.setSerdeInfo(serde);
     }
-    sd.setBucketCols(new ArrayList<String>(proto.getBucketColsList()));
-    List<Order> sortCols = new ArrayList<Order>();
+    sd.setBucketCols(new ArrayList<>(proto.getBucketColsList()));
+    List<Order> sortCols = new ArrayList<>();
     for (HbaseMetastoreProto.StorageDescriptor.Order protoOrder : proto.getSortColsList()) {
       sortCols.add(new Order(protoOrder.getColumnName(), protoOrder.getOrder()));
     }
@@ -699,15 +693,15 @@ class HBaseUtils {
     if (proto.hasSkewedInfo()) {
       SkewedInfo skewed = new SkewedInfo();
       skewed
-          .setSkewedColNames(new ArrayList<String>(proto.getSkewedInfo().getSkewedColNamesList()));
+          .setSkewedColNames(new ArrayList<>(proto.getSkewedInfo().getSkewedColNamesList()));
       for (HbaseMetastoreProto.StorageDescriptor.SkewedInfo.SkewedColValueList innerList :
           proto.getSkewedInfo().getSkewedColValuesList()) {
-        skewed.addToSkewedColValues(new ArrayList<String>(innerList.getSkewedColValueList()));
+        skewed.addToSkewedColValues(new ArrayList<>(innerList.getSkewedColValueList()));
       }
-      Map<List<String>, String> colMaps = new HashMap<List<String>, String>();
+      Map<List<String>, String> colMaps = new HashMap<>();
       for (HbaseMetastoreProto.StorageDescriptor.SkewedInfo.SkewedColValueLocationMap map :
           proto.getSkewedInfo().getSkewedColValueLocationMapsList()) {
-        colMaps.put(new ArrayList<String>(map.getKeyList()), map.getValue());
+        colMaps.put(new ArrayList<>(map.getKeyList()), map.getValue());
       }
       skewed.setSkewedColValueLocationMaps(colMaps);
       sd.setSkewedInfo(skewed);
@@ -742,7 +736,7 @@ class HBaseUtils {
   }
 
   static byte[] buildPartitionKey(String dbName, String tableName, List<String> partVals) {
-    Deque<String> keyParts = new ArrayDeque<String>(partVals);
+    Deque<String> keyParts = new ArrayDeque<>(partVals);
     keyParts.addFirst(tableName);
     keyParts.addFirst(dbName);
     return buildKey(keyParts.toArray(new String[keyParts.size()]));
@@ -1135,6 +1129,61 @@ class HBaseUtils {
   }
 
   /**
+   * Serialize a delegation token
+   * @param tokenIdentifier
+   * @param delegationToken
+   * @return two byte arrays, first contains the key, the second the serialized value.
+   */
+  static byte[][] serializeDelegationToken(String tokenIdentifier, String delegationToken) {
+    byte[][] result = new byte[2][];
+    result[0] = buildKey(tokenIdentifier);
+    result[1] = HbaseMetastoreProto.DelegationToken.newBuilder()
+        .setTokenStr(delegationToken)
+        .build()
+        .toByteArray();
+    return result;
+  }
+
+  /**
+   * Deserialize a delegation token.
+   * @param value value fetched from hbase
+   * @return A delegation token.
+   * @throws InvalidProtocolBufferException
+   */
+  static String deserializeDelegationToken(byte[] value) throws InvalidProtocolBufferException {
+    HbaseMetastoreProto.DelegationToken protoToken =
+        HbaseMetastoreProto.DelegationToken.parseFrom(value);
+    return protoToken.getTokenStr();
+  }
+
+  /**
+   * Serialize a master key
+   * @param seqNo
+   * @param key
+   * @return two byte arrays, first contains the key, the second the serialized value.
+   */
+  static byte[][] serializeMasterKey(Integer seqNo, String key) {
+    byte[][] result = new byte[2][];
+    result[0] = buildKey(seqNo.toString());
+    result[1] = HbaseMetastoreProto.MasterKey.newBuilder()
+        .setMasterKey(key)
+        .build()
+        .toByteArray();
+    return result;
+  }
+
+  /**
+   * Deserialize a master key.
+   * @param value value fetched from hbase
+   * @return A master key
+   * @throws InvalidProtocolBufferException
+   */
+  static String deserializeMasterKey(byte[] value) throws InvalidProtocolBufferException {
+    HbaseMetastoreProto.MasterKey protoKey = HbaseMetastoreProto.MasterKey.parseFrom(value);
+    return protoKey.getMasterKey();
+  }
+
+  /**
    * @param keyStart byte array representing the start prefix
    * @return byte array corresponding to the next possible prefix
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/a310524c/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
----------------------------------------------------------------------
diff --git a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
index 3cd8867..cba3671 100644
--- a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
+++ b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
@@ -104,6 +104,10 @@ message Database {
   optional PrincipalType owner_type = 6;
 }
 
+message DelegationToken {
+  required string token_str = 1;
+}
+
 message FieldSchema {
   required string name = 1;
   required string type = 2;
@@ -133,6 +137,10 @@ message Function {
   repeated ResourceUri resource_uris = 6;
 }
 
+message MasterKey {
+  required string master_key = 1;
+}
+
 message ParameterEntry {
   required string key = 1;
   required string value = 2;
@@ -247,8 +255,3 @@ message Table {
   optional PrincipalPrivilegeSet privileges = 13;
   optional bool is_temporary = 14;
 }
-
-
-
-
-   

http://git-wip-us.apache.org/repos/asf/hive/blob/a310524c/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
index 9878499..fac7dcc 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
@@ -23,7 +23,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
@@ -1246,56 +1245,6 @@ public class TestHBaseStore {
     Assert.assertEquals(decimalData.getNumDVs(), decimalDataFromDB.getNumDVs());
   }
 
-  // TODO: Activate this test, when we are able to mock the HBaseReadWrite.NO_CACHE_CONF set to false
-  // Right now, I have tested this by using aggrStatsCache despite NO_CACHE_CONF set to true
-  // Also need to add tests for other data types + refactor a lot of duplicate code in stats testing
-  //@Test
-  public void AggrStats() throws Exception {
-    int numParts = 3;
-    ColumnStatistics stats;
-    ColumnStatisticsDesc desc;
-    ColumnStatisticsObj obj;
-    List<String> partNames = new ArrayList<String>();
-    List<String> colNames = new ArrayList<String>();
-    colNames.add(BOOLEAN_COL);
-    // Add boolean col stats to DB for numParts partitions:
-    // PART_VALS(0), PART_VALS(1) & PART_VALS(2) for PART_KEYS(0)
-    for (int i = 0; i < numParts; i++) {
-      stats = new ColumnStatistics();
-      // Get a default ColumnStatisticsDesc for partition level stats
-      desc = getMockPartColStatsDesc(0, i);
-      stats.setStatsDesc(desc);
-      partNames.add(desc.getPartName());
-      // Get one of the pre-created ColumnStatisticsObj
-      obj = booleanColStatsObjs.get(i);
-      stats.addToStatsObj(obj);
-      // Add to DB
-      List<String> parVals = new ArrayList<String>();
-      parVals.add(PART_VALS.get(i));
-      store.updatePartitionColumnStatistics(stats, parVals);
-    }
-    // Read aggregate stats
-    AggrStats aggrStatsFromDB = store.get_aggr_stats_for(DB, TBL, partNames, colNames);
-    // Verify
-    Assert.assertEquals(1, aggrStatsFromDB.getColStatsSize());
-    ColumnStatisticsObj objFromDB = aggrStatsFromDB.getColStats().get(0);
-    Assert.assertNotNull(objFromDB);
-    // Aggregate our mock values
-    long numTrues = 0, numFalses = 0, numNulls = 0;
-    BooleanColumnStatsData boolData;;
-    for (int i = 0; i < numParts; i++) {
-      boolData = booleanColStatsObjs.get(i).getStatsData().getBooleanStats();
-      numTrues = numTrues + boolData.getNumTrues();
-      numFalses = numFalses + boolData.getNumFalses();
-      numNulls = numNulls + boolData.getNumNulls();
-    }
-    // Compare with what we got from the method call
-    BooleanColumnStatsData boolDataFromDB = objFromDB.getStatsData().getBooleanStats();
-    Assert.assertEquals(numTrues, boolDataFromDB.getNumTrues());
-    Assert.assertEquals(numFalses, boolDataFromDB.getNumFalses());
-    Assert.assertEquals(numNulls, boolDataFromDB.getNumNulls());
-  }
-
   /**
    * Returns a dummy table level ColumnStatisticsDesc with default values
    */


[12/52] [abbrv] hive git commit: HIVE-11568 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index f8042fc,0000000..0204f37
mode 100644,000000..100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@@ -1,2225 -1,0 +1,2241 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + */
 +package org.apache.hadoop.hive.metastore.hbase;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.cache.CacheLoader;
 +import org.apache.commons.lang.StringUtils;
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hive.common.FileUtils;
 +import org.apache.hadoop.hive.metastore.HiveMetaStore;
 +import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
 +import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
 +import org.apache.hadoop.hive.metastore.RawStore;
 +import org.apache.hadoop.hive.metastore.api.AggrStats;
 +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 +import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 +import org.apache.hadoop.hive.metastore.api.Database;
 +import org.apache.hadoop.hive.metastore.api.FieldSchema;
 +import org.apache.hadoop.hive.metastore.api.Function;
 +import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 +import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 +import org.apache.hadoop.hive.metastore.api.HiveObjectType;
 +import org.apache.hadoop.hive.metastore.api.Index;
 +import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 +import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 +import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
 +import org.apache.hadoop.hive.metastore.api.MetaException;
 +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 +import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 +import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
 +import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 +import org.apache.hadoop.hive.metastore.api.Partition;
 +import org.apache.hadoop.hive.metastore.api.PartitionEventType;
 +import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 +import org.apache.hadoop.hive.metastore.api.PrincipalType;
 +import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 +import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
 +import org.apache.hadoop.hive.metastore.api.Role;
 +import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 +import org.apache.hadoop.hive.metastore.api.Table;
 +import org.apache.hadoop.hive.metastore.api.Type;
 +import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 +import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 +import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 +import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.PlanResult;
 +import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.ScanPlan;
 +import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
 +import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 +import org.apache.thrift.TException;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +/**
 + * Implementation of RawStore that stores data in HBase
 + */
 +public class HBaseStore implements RawStore {
 +  static final private Log LOG = LogFactory.getLog(HBaseStore.class.getName());
 +
 +  // Do not access this directly, call getHBase to make sure it is initialized.
 +  private HBaseReadWrite hbase = null;
 +  private Configuration conf;
 +  private int txnNestLevel = 0;
 +  private PartitionExpressionProxy expressionProxy = null;
 +
 +  public HBaseStore() {
 +  }
 +
 +  @Override
 +  public void shutdown() {
 +    try {
 +      if (txnNestLevel != 0) rollbackTransaction();
 +      getHBase().close();
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +
 +  @Override
 +  public boolean openTransaction() {
 +    if (txnNestLevel++ <= 0) {
 +      LOG.debug("Opening HBase transaction");
 +      getHBase().begin();
 +      txnNestLevel = 1;
 +    }
 +    return true;
 +  }
 +
 +  @Override
 +  public boolean commitTransaction() {
 +    if (--txnNestLevel == 0) {
 +      LOG.debug("Committing HBase transaction");
 +      getHBase().commit();
 +    }
 +    return true;
 +  }
 +
 +  @Override
 +  public void rollbackTransaction() {
 +    txnNestLevel = 0;
 +    LOG.debug("Rolling back HBase transaction");
 +    getHBase().rollback();
 +  }
 +
 +  @Override
 +  public void createDatabase(Database db) throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +
 +      // HiveMetaStore already checks for existence of the database, don't recheck
 +      getHBase().putDb(db);
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to create database ", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +
 +  }
 +
 +  @Override
 +  public Database getDatabase(String name) throws NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Database db = getHBase().getDb(name);
 +      if (db == null) {
 +        throw new NoSuchObjectException("Unable to find db " + name);
 +      }
 +      commit = true;
 +      return db;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get db", e);
 +      throw new NoSuchObjectException("Error reading db " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean dropDatabase(String dbname) throws NoSuchObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().deleteDb(dbname);
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to delete db" + e);
 +      throw new MetaException("Unable to drop database " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean alterDatabase(String dbname, Database db) throws NoSuchObjectException,
 +      MetaException {
 +    // ObjectStore fetches the old db before updating it, but I can't see the possible value of
 +    // that since the caller will have needed to call getDatabase to have the db object.
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().putDb(db);
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to alter database ", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<String> getDatabases(String pattern) throws MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Database> dbs = getHBase().scanDatabases(likeToRegex(pattern));
 +      List<String> dbNames = new ArrayList<String>(dbs.size());
 +      for (Database db : dbs) dbNames.add(db.getName());
 +      commit = true;
 +      return dbNames;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get databases ", e);
 +      throw new MetaException("Unable to get databases, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<String> getAllDatabases() throws MetaException {
 +    return getDatabases(null);
 +  }
 +
 +  @Override
 +  public boolean createType(Type type) {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public Type getType(String typeName) {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public boolean dropType(String typeName) {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void createTable(Table tbl) throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    // HiveMetaStore above us checks if the table already exists, so we can blindly store it here.
 +    try {
 +      getHBase().putTable(tbl);
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to create table ", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean dropTable(String dbName, String tableName) throws MetaException,
 +      NoSuchObjectException, InvalidObjectException, InvalidInputException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().deleteTable(dbName, tableName);
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to delete db" + e);
 +      throw new MetaException("Unable to drop table " + tableNameForErrorMsg(dbName, tableName));
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public Table getTable(String dbName, String tableName) throws MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Table table = getHBase().getTable(dbName, tableName);
 +      if (table == null) {
 +        LOG.debug("Unable to find table " + tableNameForErrorMsg(dbName, tableName));
 +      }
 +      commit = true;
 +      return table;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get table", e);
 +      throw new MetaException("Error reading table " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean addPartition(Partition part) throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().putPartition(part);
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to add partition", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean addPartitions(String dbName, String tblName, List<Partition> parts)
 +      throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().putPartitions(parts);
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to add partitions", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec,
 +                               boolean ifNotExists) throws InvalidObjectException, MetaException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public Partition getPartition(String dbName, String tableName, List<String> part_vals) throws
 +      MetaException, NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Partition part = getHBase().getPartition(dbName, tableName, part_vals);
 +      if (part == null) {
 +        throw new NoSuchObjectException("Unable to find partition " +
 +            partNameForErrorMsg(dbName, tableName, part_vals));
 +      }
 +      commit = true;
 +      return part;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get partition", e);
 +      throw new MetaException("Error reading partition " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean doesPartitionExist(String dbName, String tableName, List<String> part_vals) throws
 +      MetaException, NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      boolean exists = getHBase().getPartition(dbName, tableName, part_vals) != null;
 +      commit = true;
 +      return exists;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get partition", e);
 +      throw new MetaException("Error reading partition " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean dropPartition(String dbName, String tableName, List<String> part_vals) throws
 +      MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().deletePartition(dbName, tableName, part_vals);
 +      // Drop any cached stats that reference this partitions
 +      getHBase().getStatsCache().invalidate(dbName, tableName,
 +          buildExternalPartName(dbName, tableName, part_vals));
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to delete db" + e);
 +      throw new MetaException("Unable to drop partition " + partNameForErrorMsg(dbName, tableName,
 +          part_vals));
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<Partition> getPartitions(String dbName, String tableName, int max) throws
 +      MetaException, NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Partition> parts = getHBase().scanPartitionsInTable(dbName, tableName, max);
 +      commit = true;
 +      return parts;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get partitions", e);
 +      throw new MetaException("Error scanning partitions");
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public void alterTable(String dbname, String name, Table newTable) throws InvalidObjectException,
 +      MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().replaceTable(getHBase().getTable(dbname, name), newTable);
 +      if (newTable.getPartitionKeys() != null && newTable.getPartitionKeys().size() > 0
 +          && !name.equals(newTable.getTableName())) {
 +        // They renamed the table, so we need to change each partition as well, since it changes
 +        // the key.
 +        try {
 +          List<Partition> oldParts = getPartitions(dbname, name, -1);
 +          List<Partition> newParts = new ArrayList<>(oldParts.size());
 +          for (Partition oldPart : oldParts) {
 +            Partition newPart = oldPart.deepCopy();
 +            newPart.setTableName(newTable.getTableName());
 +            newParts.add(newPart);
 +          }
 +          getHBase().replacePartitions(oldParts, newParts);
 +        } catch (NoSuchObjectException e) {
 +          LOG.debug("No partitions found for old table so not worrying about it");
 +        }
 +
 +      }
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to alter table " + tableNameForErrorMsg(dbname, name), e);
 +      throw new MetaException("Unable to alter table " + tableNameForErrorMsg(dbname, name));
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<String> getTables(String dbName, String pattern) throws MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Table> tables = getHBase().scanTables(dbName, likeToRegex(pattern));
 +      List<String> tableNames = new ArrayList<String>(tables.size());
 +      for (Table table : tables) tableNames.add(table.getTableName());
 +      commit = true;
 +      return tableNames;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get tables ", e);
 +      throw new MetaException("Unable to get tables, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<Table> getTableObjectsByName(String dbname, List<String> tableNames) throws
 +      MetaException, UnknownDBException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Table> tables = getHBase().getTables(dbname, tableNames);
 +      commit = true;
 +      return tables;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get tables ", e);
 +      throw new MetaException("Unable to get tables, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<String> getAllTables(String dbName) throws MetaException {
 +    return getTables(dbName, null);
 +  }
 +
 +  @Override
 +  public List<String> listTableNamesByFilter(String dbName, String filter, short max_tables) throws
 +      MetaException, UnknownDBException {
 +    // TODO needs to wait until we support pushing filters into HBase.
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public List<String> listPartitionNames(String db_name, String tbl_name, short max_parts) throws
 +      MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Partition> parts = getHBase().scanPartitionsInTable(db_name, tbl_name, max_parts);
 +      if (parts == null) return null;
 +      List<String> names = new ArrayList<String>(parts.size());
 +      Table table = getHBase().getTable(db_name, tbl_name);
 +      for (Partition p : parts) {
 +        names.add(buildExternalPartName(table, p));
 +      }
 +      commit = true;
 +      return names;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get partitions", e);
 +      throw new MetaException("Error scanning partitions");
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<String> listPartitionNamesByFilter(String db_name, String tbl_name, String filter,
 +                                                 short max_parts) throws MetaException {
 +    // TODO needs to wait until we support pushing filters into HBase.
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void alterPartition(String db_name, String tbl_name, List<String> part_vals,
 +                             Partition new_part) throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Partition oldPart = getHBase().getPartition(db_name, tbl_name, part_vals);
 +      getHBase().replacePartition(oldPart, new_part);
 +      // Drop any cached stats that reference this partitions
 +      getHBase().getStatsCache().invalidate(db_name, tbl_name,
 +          buildExternalPartName(db_name, tbl_name, part_vals));
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to add partition", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public void alterPartitions(String db_name, String tbl_name, List<List<String>> part_vals_list,
 +                              List<Partition> new_parts) throws InvalidObjectException,
 +      MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Partition> oldParts = getHBase().getPartitions(db_name, tbl_name, part_vals_list);
 +      getHBase().replacePartitions(oldParts, new_parts);
 +      for (List<String> part_vals : part_vals_list) {
 +        getHBase().getStatsCache().invalidate(db_name, tbl_name,
 +            buildExternalPartName(db_name, tbl_name, part_vals));
 +      }
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to add partition", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean addIndex(Index index) throws InvalidObjectException, MetaException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public Index getIndex(String dbName, String origTableName, String indexName) throws
 +      MetaException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public boolean dropIndex(String dbName, String origTableName, String indexName) throws
 +      MetaException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public List<Index> getIndexes(String dbName, String origTableName, int max) throws MetaException {
 +    // TODO - Index not currently supported.  But I need to return an empty list or else drop
 +    // table cores.
 +    return new ArrayList<Index>();
 +  }
 +
 +  @Override
 +  public List<String> listIndexNames(String dbName, String origTableName, short max) throws
 +      MetaException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void alterIndex(String dbname, String baseTblName, String name, Index newIndex) throws
 +      InvalidObjectException, MetaException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public List<Partition> getPartitionsByFilter(String dbName, String tblName, String filter,
 +                                               short maxParts) throws MetaException,
 +      NoSuchObjectException {
 +    final ExpressionTree exprTree = (filter != null && !filter.isEmpty()) ? PartFilterExprUtil
 +        .getFilterParser(filter).tree : ExpressionTree.EMPTY_TREE;
 +    List<Partition> result = new ArrayList<Partition>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getPartitionsByExprInternal(dbName, tblName, exprTree, maxParts, result);
 +      return result;
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr,
 +                                     String defaultPartitionName, short maxParts,
 +                                     List<Partition> result) throws TException {
 +    final ExpressionTree exprTree = PartFilterExprUtil.makeExpressionTree(expressionProxy, expr);
 +    // TODO: investigate if there should be any role for defaultPartitionName in this
 +    // implementation. direct sql code path in ObjectStore does not use it.
 +
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      return getPartitionsByExprInternal(dbName, tblName, exprTree, maxParts, result);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  private boolean getPartitionsByExprInternal(String dbName, String tblName,
 +      ExpressionTree exprTree, short maxParts, List<Partition> result) throws MetaException,
 +      NoSuchObjectException {
 +
 +    Table table = getTable(dbName, tblName);
 +    if (table == null) {
 +      throw new NoSuchObjectException("Unable to find table " + dbName + "." + tblName);
 +    }
 +    String firstPartitionColumn = table.getPartitionKeys().get(0).getName();
 +    // general hbase filter plan from expression tree
 +    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(exprTree, firstPartitionColumn);
 +
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Hbase Filter Plan generated : " + planRes.plan);
 +    }
 +
 +    // results from scans need to be merged as there can be overlapping results between
 +    // the scans. Use a map of list of partition values to partition for this.
 +    Map<List<String>, Partition> mergedParts = new HashMap<List<String>, Partition>();
 +    for (ScanPlan splan : planRes.plan.getPlans()) {
 +      try {
 +        List<Partition> parts = getHBase().scanPartitions(dbName, tblName,
 +            splan.getStartRowSuffix(), splan.getEndRowSuffix(), null, -1);
 +        boolean reachedMax = false;
 +        for (Partition part : parts) {
 +          mergedParts.put(part.getValues(), part);
 +          if (mergedParts.size() == maxParts) {
 +            reachedMax = true;
 +            break;
 +          }
 +        }
 +        if (reachedMax) {
 +          break;
 +        }
 +      } catch (IOException e) {
 +        LOG.error("Unable to get partitions", e);
 +        throw new MetaException("Error scanning partitions" + tableNameForErrorMsg(dbName, tblName)
 +            + ": " + e);
 +      }
 +    }
 +    for (Entry<List<String>, Partition> mp : mergedParts.entrySet()) {
 +      result.add(mp.getValue());
 +    }
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Matched partitions " + result);
 +    }
 +
 +    // return true if there might be some additional partitions that don't match filter conditions
 +    // being returned
 +    return !planRes.hasUnsupportedCondition;
 +  }
 +
 +  @Override
 +  public List<Partition> getPartitionsByNames(String dbName, String tblName,
 +                                              List<String> partNames) throws MetaException,
 +      NoSuchObjectException {
 +    List<Partition> parts = new ArrayList<Partition>();
 +    for (String partName : partNames) {
 +      parts.add(getPartition(dbName, tblName, partNameToVals(partName)));
 +    }
 +    return parts;
 +  }
 +
 +  @Override
 +  public Table markPartitionForEvent(String dbName, String tblName, Map<String, String> partVals,
 +                                     PartitionEventType evtType) throws MetaException,
 +      UnknownTableException, InvalidPartitionException, UnknownPartitionException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public boolean isPartitionMarkedForEvent(String dbName, String tblName,
 +                                           Map<String, String> partName,
 +                                           PartitionEventType evtType) throws MetaException,
 +      UnknownTableException, InvalidPartitionException, UnknownPartitionException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  /*
 +   * The design for roles.  Roles are a pain because of their hierarchical nature.  When a user
 +   * comes in and we need to be able to determine all roles he is a part of, we do not want to
 +   * have to walk the hierarchy in the database.  This means we need to flatten the role map for
 +   * each user.  But we also have to track how the roles are connected for each user, in case one
 +   * role is revoked from another (e.g. if role1 is included in role2 but then revoked
 +   * from it and user1 was granted both role2 and role1 we cannot remove user1 from role1
 +   * because he was granted that separately).
 +   *
 +   * We want to optimize for the read case and put the cost on grant and revoke of roles, since
 +   * we assume that is the less common case.  So we lay out the roles data as follows:
 +   *
 +   * There is a ROLES table that records each role, plus what other principals have been granted
 +   * into it, along with the info on grantor, etc.
 +   *
 +   * There is a USER_TO_ROLES table that contains the mapping of each user to every role he is a
 +   * part of.
 +   *
 +   * This makes determining what roles a user participates in very quick, as USER_TO_ROLE is a
 +   * simple list for each user.  It makes granting users into roles expensive, and granting roles
 +   * into roles very expensive.  Each time a user is granted into a role, we need to walk the
 +   * hierarchy in the role table (which means moving through that table multiple times) to
 +   * determine every role the user participates in.  Each a role is granted into another role
 +   * this hierarchical walk must be done for every principal in the role being granted into.  To
 +   * mitigate this pain somewhat whenever doing these mappings we cache the entire ROLES table in
 +   * memory since we assume it is not large.
 +   *
 +   * On a related note, whenever a role is dropped we must walk not only all these role tables
 +   * above (equivalent to a role being revoked from another role, since we have to rebuilding
 +   * mappings for any users in roles that contained that role and any users directly in that
 +   * role), but we also have to remove all the privileges associated with that role directly.
 +   * That means a walk of the DBS table and of the TBLS table.
 +   */
 +
 +  @Override
 +  public boolean addRole(String roleName, String ownerName) throws InvalidObjectException,
 +      MetaException, NoSuchObjectException {
 +    int now = (int)(System.currentTimeMillis()/1000);
 +    Role role = new Role(roleName, now, ownerName);
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      if (getHBase().getRole(roleName) != null) {
 +        throw new InvalidObjectException("Role " + roleName + " already exists");
 +      }
 +      getHBase().putRole(role);
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to create role ", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean removeRole(String roleName) throws MetaException, NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Set<String> usersInRole = getHBase().findAllUsersInRole(roleName);
 +      getHBase().deleteRole(roleName);
 +      getHBase().removeRoleGrants(roleName);
 +      for (String user : usersInRole) {
 +        getHBase().buildRoleMapForUser(user);
 +      }
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to delete role" + e);
 +      throw new MetaException("Unable to drop role " + roleName);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean grantRole(Role role, String userName, PrincipalType principalType, String grantor,
 +                           PrincipalType grantorType, boolean grantOption)
 +      throws MetaException, NoSuchObjectException, InvalidObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Set<String> usersToRemap = findUsersToRemapRolesFor(role, userName, principalType);
 +      HbaseMetastoreProto.RoleGrantInfo.Builder builder =
 +          HbaseMetastoreProto.RoleGrantInfo.newBuilder();
 +      if (userName != null) builder.setPrincipalName(userName);
 +      if (principalType != null) {
 +        builder.setPrincipalType(HBaseUtils.convertPrincipalTypes(principalType));
 +      }
 +      builder.setAddTime((int)(System.currentTimeMillis() / 1000));
 +      if (grantor != null) builder.setGrantor(grantor);
 +      if (grantorType != null) {
 +        builder.setGrantorType(HBaseUtils.convertPrincipalTypes(grantorType));
 +      }
 +      builder.setGrantOption(grantOption);
 +
 +      getHBase().addPrincipalToRole(role.getRoleName(), builder.build());
 +      for (String user : usersToRemap) {
 +        getHBase().buildRoleMapForUser(user);
 +      }
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to grant role", e);
 +      throw new MetaException("Unable to grant role " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean revokeRole(Role role, String userName, PrincipalType principalType,
 +                            boolean grantOption) throws MetaException, NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    // This can have a couple of different meanings.  If grantOption is true, then this is only
 +    // revoking the grant option, the role itself doesn't need to be removed.  If it is false
 +    // then we need to remove the userName from the role altogether.
 +    try {
 +      if (grantOption) {
 +        // If this is a grant only change, we don't need to rebuild the user mappings.
 +        getHBase().dropPrincipalFromRole(role.getRoleName(), userName, principalType, grantOption);
 +      } else {
 +        Set<String> usersToRemap = findUsersToRemapRolesFor(role, userName, principalType);
 +        getHBase().dropPrincipalFromRole(role.getRoleName(), userName, principalType, grantOption);
 +        for (String user : usersToRemap) {
 +          getHBase().buildRoleMapForUser(user);
 +        }
 +      }
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to revoke role " + role.getRoleName() + " from " + userName, e);
 +      throw new MetaException("Unable to revoke role " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public PrincipalPrivilegeSet getUserPrivilegeSet(String userName, List<String> groupNames)
 +      throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      PrincipalPrivilegeSet pps = new PrincipalPrivilegeSet();
 +      PrincipalPrivilegeSet global = getHBase().getGlobalPrivs();
 +      if (global == null) return null;
 +      List<PrivilegeGrantInfo> pgi;
 +      if (global.getUserPrivileges() != null) {
 +        pgi = global.getUserPrivileges().get(userName);
 +        if (pgi != null) {
 +          pps.putToUserPrivileges(userName, pgi);
 +        }
 +      }
 +
 +      if (global.getRolePrivileges() != null) {
 +        List<String> roles = getHBase().getUserRoles(userName);
 +        if (roles != null) {
 +          for (String role : roles) {
 +            pgi = global.getRolePrivileges().get(role);
 +            if (pgi != null) {
 +              pps.putToRolePrivileges(role, pgi);
 +            }
 +          }
 +        }
 +      }
 +      commit = true;
 +      return pps;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get db privileges for user", e);
 +      throw new MetaException("Unable to get db privileges for user, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public PrincipalPrivilegeSet getDBPrivilegeSet(String dbName, String userName,
 +                                                 List<String> groupNames)
 +      throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      PrincipalPrivilegeSet pps = new PrincipalPrivilegeSet();
 +      Database db = getHBase().getDb(dbName);
 +      if (db.getPrivileges() != null) {
 +        List<PrivilegeGrantInfo> pgi;
 +        // Find the user privileges for this db
 +        if (db.getPrivileges().getUserPrivileges() != null) {
 +          pgi = db.getPrivileges().getUserPrivileges().get(userName);
 +          if (pgi != null) {
 +            pps.putToUserPrivileges(userName, pgi);
 +          }
 +        }
 +
 +        if (db.getPrivileges().getRolePrivileges() != null) {
 +          List<String> roles = getHBase().getUserRoles(userName);
 +          if (roles != null) {
 +            for (String role : roles) {
 +              pgi = db.getPrivileges().getRolePrivileges().get(role);
 +              if (pgi != null) {
 +                pps.putToRolePrivileges(role, pgi);
 +              }
 +            }
 +          }
 +        }
 +      }
 +      commit = true;
 +      return pps;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get db privileges for user", e);
 +      throw new MetaException("Unable to get db privileges for user, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public PrincipalPrivilegeSet getTablePrivilegeSet(String dbName, String tableName,
 +                                                    String userName, List<String> groupNames)
 +      throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      PrincipalPrivilegeSet pps = new PrincipalPrivilegeSet();
 +      Table table = getHBase().getTable(dbName, tableName);
 +      List<PrivilegeGrantInfo> pgi;
 +      if (table.getPrivileges() != null) {
 +        if (table.getPrivileges().getUserPrivileges() != null) {
 +          pgi = table.getPrivileges().getUserPrivileges().get(userName);
 +          if (pgi != null) {
 +            pps.putToUserPrivileges(userName, pgi);
 +          }
 +        }
 +
 +        if (table.getPrivileges().getRolePrivileges() != null) {
 +          List<String> roles = getHBase().getUserRoles(userName);
 +          if (roles != null) {
 +            for (String role : roles) {
 +              pgi = table.getPrivileges().getRolePrivileges().get(role);
 +              if (pgi != null) {
 +                pps.putToRolePrivileges(role, pgi);
 +              }
 +            }
 +          }
 +        }
 +      }
 +      commit = true;
 +      return pps;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get db privileges for user", e);
 +      throw new MetaException("Unable to get db privileges for user, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String dbName, String tableName,
 +                                                        String partition, String userName,
 +                                                        List<String> groupNames) throws
 +      InvalidObjectException, MetaException {
 +    // We don't support partition privileges
 +    return null;
 +  }
 +
 +  @Override
 +  public PrincipalPrivilegeSet getColumnPrivilegeSet(String dbName, String tableName,
 +                                                     String partitionName, String columnName,
 +                                                     String userName,
 +                                                     List<String> groupNames) throws
 +      InvalidObjectException, MetaException {
 +    // We don't support column level privileges
 +    return null;
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalGlobalGrants(String principalName,
 +                                                             PrincipalType principalType) {
 +    List<PrivilegeGrantInfo> grants;
 +    List<HiveObjectPrivilege> privileges = new ArrayList<HiveObjectPrivilege>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      PrincipalPrivilegeSet pps = getHBase().getGlobalPrivs();
 +      if (pps == null) return privileges;
 +      Map<String, List<PrivilegeGrantInfo>> map;
 +      switch (principalType) {
 +        case USER:
 +          map = pps.getUserPrivileges();
 +          break;
 +
 +        case ROLE:
 +          map = pps.getRolePrivileges();
 +          break;
 +
 +        default:
 +          throw new RuntimeException("Unknown or unsupported principal type " +
 +              principalType.toString());
 +      }
 +      if (map == null) return privileges;
 +      grants = map.get(principalName);
 +
 +      if (grants == null || grants.size() == 0) return privileges;
 +      for (PrivilegeGrantInfo pgi : grants) {
 +        privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.GLOBAL, null,
 +            null, null, null), principalName, principalType, pgi));
 +      }
 +      commit = true;
 +      return privileges;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
 +                                                         PrincipalType principalType,
 +                                                         String dbName) {
 +    List<PrivilegeGrantInfo> grants;
 +    List<HiveObjectPrivilege> privileges = new ArrayList<HiveObjectPrivilege>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Database db = getHBase().getDb(dbName);
 +      if (db == null) return privileges;
 +      PrincipalPrivilegeSet pps = db.getPrivileges();
 +      if (pps == null) return privileges;
 +      Map<String, List<PrivilegeGrantInfo>> map;
 +      switch (principalType) {
 +        case USER:
 +          map = pps.getUserPrivileges();
 +          break;
 +
 +        case ROLE:
 +          map = pps.getRolePrivileges();
 +          break;
 +
 +        default:
 +          throw new RuntimeException("Unknown or unsupported principal type " +
 +              principalType.toString());
 +      }
 +      if (map == null) return privileges;
 +      grants = map.get(principalName);
 +
 +      if (grants == null || grants.size() == 0) return privileges;
 +      for (PrivilegeGrantInfo pgi : grants) {
 +        privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.DATABASE, dbName,
 +         null, null, null), principalName, principalType, pgi));
 +      }
 +      commit = true;
 +      return privileges;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
 +                                                      PrincipalType principalType,
 +                                                      String dbName,
 +                                                      String tableName) {
 +    List<PrivilegeGrantInfo> grants;
 +    List<HiveObjectPrivilege> privileges = new ArrayList<HiveObjectPrivilege>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Table table = getHBase().getTable(dbName, tableName);
 +      if (table == null) return privileges;
 +      PrincipalPrivilegeSet pps = table.getPrivileges();
 +      if (pps == null) return privileges;
 +      Map<String, List<PrivilegeGrantInfo>> map;
 +      switch (principalType) {
 +        case USER:
 +          map = pps.getUserPrivileges();
 +          break;
 +
 +        case ROLE:
 +          map = pps.getRolePrivileges();
 +          break;
 +
 +        default:
 +          throw new RuntimeException("Unknown or unsupported principal type " +
 +              principalType.toString());
 +      }
 +      if (map == null) return privileges;
 +      grants = map.get(principalName);
 +
 +      if (grants == null || grants.size() == 0) return privileges;
 +      for (PrivilegeGrantInfo pgi : grants) {
 +        privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.TABLE, dbName,
 +            tableName, null, null), principalName, principalType, pgi));
 +      }
 +      commit = true;
 +      return privileges;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalPartitionGrants(String principalName,
 +                                                                PrincipalType principalType,
 +                                                                String dbName,
 +                                                                String tableName,
 +                                                                List<String> partValues,
 +                                                                String partName) {
 +    // We don't support partition grants
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(String principalName,
 +                                                                  PrincipalType principalType,
 +                                                                  String dbName, String tableName,
 +                                                                  String columnName) {
 +    // We don't support column grants
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(String principalName,
 +                                                                      PrincipalType principalType,
 +                                                                      String dbName,
 +                                                                      String tableName,
 +                                                                      List<String> partVals,
 +                                                                      String partName,
 +                                                                      String columnName) {
 +    // We don't support column grants
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public boolean grantPrivileges(PrivilegeBag privileges)
 +      throws InvalidObjectException, MetaException, NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      for (HiveObjectPrivilege priv : privileges.getPrivileges()) {
 +        // Locate the right object to deal with
 +        PrivilegeInfo privilegeInfo = findPrivilegeToGrantOrRevoke(priv);
 +
 +        // Now, let's see if we've already got this privilege
 +        for (PrivilegeGrantInfo info : privilegeInfo.grants) {
 +          if (info.getPrivilege().equals(priv.getGrantInfo().getPrivilege())) {
 +            throw new InvalidObjectException(priv.getPrincipalName() + " already has " +
 +                priv.getGrantInfo().getPrivilege() + " on " + privilegeInfo.typeErrMsg);
 +          }
 +        }
 +        privilegeInfo.grants.add(priv.getGrantInfo());
 +
 +        writeBackGrantOrRevoke(priv, privilegeInfo);
 +      }
 +      commit = true;
 +      return true;
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean revokePrivileges(PrivilegeBag privileges, boolean grantOption) throws
 +      InvalidObjectException, MetaException, NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      for (HiveObjectPrivilege priv : privileges.getPrivileges()) {
 +        PrivilegeInfo privilegeInfo = findPrivilegeToGrantOrRevoke(priv);
 +
 +        for (int i = 0; i < privilegeInfo.grants.size(); i++) {
 +          if (privilegeInfo.grants.get(i).getPrivilege().equals(
 +              priv.getGrantInfo().getPrivilege())) {
 +            if (grantOption) privilegeInfo.grants.get(i).setGrantOption(false);
 +            else privilegeInfo.grants.remove(i);
 +            break;
 +          }
 +        }
 +        writeBackGrantOrRevoke(priv, privilegeInfo);
 +      }
 +      commit = true;
 +      return true;
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  private static class PrivilegeInfo {
 +    Database db;
 +    Table table;
 +    List<PrivilegeGrantInfo> grants;
 +    String typeErrMsg;
 +    PrincipalPrivilegeSet privSet;
 +  }
 +
 +  private PrivilegeInfo findPrivilegeToGrantOrRevoke(HiveObjectPrivilege privilege)
 +      throws MetaException, NoSuchObjectException, InvalidObjectException {
 +    PrivilegeInfo result = new PrivilegeInfo();
 +    switch (privilege.getHiveObject().getObjectType()) {
 +      case GLOBAL:
 +        try {
 +          result.privSet = createOnNull(getHBase().getGlobalPrivs());
 +        } catch (IOException e) {
 +          LOG.error("Unable to fetch global privileges", e);
 +          throw new MetaException("Unable to fetch global privileges, " + e.getMessage());
 +        }
 +        result.typeErrMsg = "global";
 +        break;
 +
 +      case DATABASE:
 +        result.db = getDatabase(privilege.getHiveObject().getDbName());
 +        result.typeErrMsg = "database " + result.db.getName();
 +        result.privSet = createOnNull(result.db.getPrivileges());
 +        break;
 +
 +      case TABLE:
 +        result.table = getTable(privilege.getHiveObject().getDbName(),
 +            privilege.getHiveObject().getObjectName());
 +        result.typeErrMsg = "table " + result.table.getTableName();
 +        result.privSet = createOnNull(result.table.getPrivileges());
 +        break;
 +
 +      case PARTITION:
 +      case COLUMN:
 +        throw new RuntimeException("HBase metastore does not support partition or column " +
 +            "permissions");
 +
 +      default:
 +        throw new RuntimeException("Woah bad, unknown object type " +
 +            privilege.getHiveObject().getObjectType());
 +    }
 +
 +    // Locate the right PrivilegeGrantInfo
 +    Map<String, List<PrivilegeGrantInfo>> grantInfos;
 +    switch (privilege.getPrincipalType()) {
 +      case USER:
 +        grantInfos = result.privSet.getUserPrivileges();
 +        result.typeErrMsg = "user";
 +        break;
 +
 +      case GROUP:
 +        throw new RuntimeException("HBase metastore does not support group permissions");
 +
 +      case ROLE:
 +        grantInfos = result.privSet.getRolePrivileges();
 +        result.typeErrMsg = "role";
 +        break;
 +
 +      default:
 +        throw new RuntimeException("Woah bad, unknown principal type " +
 +            privilege.getPrincipalType());
 +    }
 +
 +    // Find the requested name in the grantInfo
 +    result.grants = grantInfos.get(privilege.getPrincipalName());
 +    if (result.grants == null) {
 +      // Means we don't have any grants for this user yet.
 +      result.grants = new ArrayList<PrivilegeGrantInfo>();
 +      grantInfos.put(privilege.getPrincipalName(), result.grants);
 +    }
 +    return result;
 +  }
 +
 +  private PrincipalPrivilegeSet createOnNull(PrincipalPrivilegeSet pps) {
 +    // If this is the first time a user has been granted a privilege set will be null.
 +    if (pps == null) {
 +      pps = new PrincipalPrivilegeSet();
 +    }
 +    if (pps.getUserPrivileges() == null) {
 +      pps.setUserPrivileges(new HashMap<String, List<PrivilegeGrantInfo>>());
 +    }
 +    if (pps.getRolePrivileges() == null) {
 +      pps.setRolePrivileges(new HashMap<String, List<PrivilegeGrantInfo>>());
 +    }
 +    return pps;
 +  }
 +
 +  private void writeBackGrantOrRevoke(HiveObjectPrivilege priv, PrivilegeInfo pi)
 +      throws MetaException, NoSuchObjectException, InvalidObjectException {
 +    // Now write it back
 +    switch (priv.getHiveObject().getObjectType()) {
 +      case GLOBAL:
 +        try {
 +          getHBase().putGlobalPrivs(pi.privSet);
 +        } catch (IOException e) {
 +          LOG.error("Unable to write global privileges", e);
 +          throw new MetaException("Unable to write global privileges, " + e.getMessage());
 +        }
 +        break;
 +
 +      case DATABASE:
 +        pi.db.setPrivileges(pi.privSet);
 +        alterDatabase(pi.db.getName(), pi.db);
 +        break;
 +
 +      case TABLE:
 +        pi.table.setPrivileges(pi.privSet);
 +        alterTable(pi.table.getDbName(), pi.table.getTableName(), pi.table);
 +        break;
 +
 +      default:
 +        throw new RuntimeException("Dude, you missed the second switch!");
 +    }
 +  }
 +
 +  @Override
 +  public Role getRole(String roleName) throws NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Role role = getHBase().getRole(roleName);
 +      if (role == null) {
 +        throw new NoSuchObjectException("Unable to find role " + roleName);
 +      }
 +      commit = true;
 +      return role;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get role", e);
 +      throw new NoSuchObjectException("Error reading table " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<String> listRoleNames() {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Role> roles = getHBase().scanRoles();
 +      List<String> roleNames = new ArrayList<String>(roles.size());
 +      for (Role role : roles) roleNames.add(role.getRoleName());
 +      commit = true;
 +      return roleNames;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<Role> listRoles(String principalName, PrincipalType principalType) {
 +    List<Role> roles = new ArrayList<Role>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      try {
 +        roles.addAll(getHBase().getPrincipalDirectRoles(principalName, principalType));
 +      } catch (IOException e) {
 +        throw new RuntimeException(e);
 +      }
 +      // Add the public role if this is a user
 +      if (principalType == PrincipalType.USER) {
 +        roles.add(new Role(HiveMetaStore.PUBLIC, 0, null));
 +      }
 +      commit = true;
 +      return roles;
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<RolePrincipalGrant> listRolesWithGrants(String principalName,
 +                                                      PrincipalType principalType) {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Role> roles = listRoles(principalName, principalType);
 +      List<RolePrincipalGrant> rpgs = new ArrayList<RolePrincipalGrant>(roles.size());
 +      for (Role role : roles) {
 +        HbaseMetastoreProto.RoleGrantInfoList grants = getHBase().getRolePrincipals(role.getRoleName());
 +        if (grants != null) {
 +          for (HbaseMetastoreProto.RoleGrantInfo grant : grants.getGrantInfoList()) {
 +            if (grant.getPrincipalType() == HBaseUtils.convertPrincipalTypes(principalType) &&
 +                grant.getPrincipalName().equals(principalName)) {
 +              rpgs.add(new RolePrincipalGrant(role.getRoleName(), principalName, principalType,
 +                  grant.getGrantOption(), (int) grant.getAddTime(), grant.getGrantor(),
 +                  HBaseUtils.convertPrincipalTypes(grant.getGrantorType())));
 +            }
 +          }
 +        }
 +      }
 +      commit = true;
 +      return rpgs;
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<RolePrincipalGrant> listRoleMembers(String roleName) {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      HbaseMetastoreProto.RoleGrantInfoList gil = getHBase().getRolePrincipals(roleName);
 +      List<RolePrincipalGrant> roleMaps = new ArrayList<RolePrincipalGrant>(gil.getGrantInfoList().size());
 +      for (HbaseMetastoreProto.RoleGrantInfo giw : gil.getGrantInfoList()) {
 +        roleMaps.add(new RolePrincipalGrant(roleName, giw.getPrincipalName(),
 +            HBaseUtils.convertPrincipalTypes(giw.getPrincipalType()),
 +            giw.getGrantOption(), (int)giw.getAddTime(), giw.getGrantor(),
 +            HBaseUtils.convertPrincipalTypes(giw.getGrantorType())));
 +      }
 +      commit = true;
 +      return roleMaps;
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public Partition getPartitionWithAuth(String dbName, String tblName, List<String> partVals,
 +                                        String user_name, List<String> group_names)
 +      throws MetaException, NoSuchObjectException, InvalidObjectException {
 +    // We don't do authorization checks for partitions.
 +    return getPartition(dbName, tblName, partVals);
 +  }
 +
 +  @Override
 +  public List<Partition> getPartitionsWithAuth(String dbName, String tblName, short maxParts,
 +                                               String userName, List<String> groupNames)
 +      throws MetaException, NoSuchObjectException, InvalidObjectException {
 +    // We don't do authorization checks for partitions.
 +    return getPartitions(dbName, tblName, maxParts);
 +  }
 +
 +  @Override
 +  public List<String> listPartitionNamesPs(String db_name, String tbl_name, List<String> part_vals,
 +                                           short max_parts)
 +      throws MetaException, NoSuchObjectException {
 +    List<Partition> parts =
 +        listPartitionsPsWithAuth(db_name, tbl_name, part_vals, max_parts, null, null);
 +    List<String> partNames = new ArrayList<String>(parts.size());
 +    for (Partition part : parts) {
 +      partNames.add(buildExternalPartName(db_name, tbl_name, part.getValues()));
 +    }
 +    return partNames;
 +  }
 +
 +
 +  @Override
 +  public List<Partition> listPartitionsPsWithAuth(String db_name, String tbl_name,
 +                                                  List<String> part_vals, short max_parts,
 +                                                  String userName, List<String> groupNames)
 +      throws MetaException, NoSuchObjectException {
 +    // We don't handle auth info with partitions
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Partition> parts = getHBase().scanPartitions(db_name, tbl_name, part_vals, max_parts);
 +      commit = true;
 +      return parts;
 +    } catch (IOException e) {
 +      LOG.error("Unable to list partition names", e);
 +      throw new MetaException("Failed to list part names, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean updateTableColumnStatistics(ColumnStatistics colStats) throws
 +      NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().updateStatistics(colStats.getStatsDesc().getDbName(),
 +          colStats.getStatsDesc().getTableName(), null, colStats);
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to update column statistics", e);
 +      throw new MetaException("Failed to update column statistics, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj,
 +                                                 List<String> partVals) throws
 +      NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().updateStatistics(statsObj.getStatsDesc().getDbName(),
 +          statsObj.getStatsDesc().getTableName(), partVals, statsObj);
 +      // We need to invalidate aggregates that include this partition
 +      getHBase().getStatsCache().invalidate(statsObj.getStatsDesc().getDbName(),
 +          statsObj.getStatsDesc().getTableName(), statsObj.getStatsDesc().getPartName());
 +      commit = true;
 +      return true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to update column statistics", e);
 +      throw new MetaException("Failed to update column statistics, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public ColumnStatistics getTableColumnStatistics(String dbName, String tableName,
 +                                                   List<String> colName) throws MetaException,
 +      NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      ColumnStatistics cs = getHBase().getTableStatistics(dbName, tableName, colName);
 +      commit = true;
 +      return cs;
 +    } catch (IOException e) {
 +      LOG.error("Unable to fetch column statistics", e);
 +      throw new MetaException("Failed to fetch column statistics, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<ColumnStatistics> getPartitionColumnStatistics(String dbName, String tblName,
 +      List<String> partNames, List<String> colNames) throws MetaException, NoSuchObjectException {
 +    List<List<String>> partVals = new ArrayList<List<String>>(partNames.size());
 +    for (String partName : partNames) {
 +      partVals.add(partNameToVals(partName));
 +    }
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<ColumnStatistics> cs =
 +          getHBase().getPartitionStatistics(dbName, tblName, partNames,  partVals, colNames);
 +      commit = true;
 +      return cs;
 +    } catch (IOException e) {
 +      LOG.error("Unable to fetch column statistics", e);
 +      throw new MetaException("Failed fetching column statistics, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean deletePartitionColumnStatistics(String dbName, String tableName, String partName,
 +      List<String> partVals, String colName) throws NoSuchObjectException, MetaException,
 +      InvalidObjectException, InvalidInputException {
 +    // NOP, stats will be deleted along with the partition when it is dropped.
 +    return true;
 +  }
 +
 +  @Override
 +  public boolean deleteTableColumnStatistics(String dbName, String tableName, String colName) throws
 +      NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
 +    // NOP, stats will be deleted along with the table when it is dropped.
 +    return true;
 +  }
 +
 +  /**
 +   * Return aggregated statistics for each column in the colNames list aggregated over partitions in
 +   * the partNames list
 +   *
 +   */
 +  @Override
 +  public AggrStats get_aggr_stats_for(String dbName, String tblName, List<String> partNames,
 +      List<String> colNames) throws MetaException, NoSuchObjectException {
 +    List<List<String>> partVals = new ArrayList<List<String>>(partNames.size());
 +    for (String partName : partNames) {
 +      partVals.add(partNameToVals(partName));
 +    }
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      AggrStats aggrStats = new AggrStats();
 +      for (String colName : colNames) {
 +        try {
 +          AggrStats oneCol =
 +              getHBase().getStatsCache().get(dbName, tblName, partNames, colName);
 +          if (oneCol.getColStatsSize() > 0) {
 +            assert oneCol.getColStatsSize() == 1;
 +            aggrStats.setPartsFound(aggrStats.getPartsFound() + oneCol.getPartsFound());
 +            aggrStats.addToColStats(oneCol.getColStats().get(0));
 +          }
 +        } catch (CacheLoader.InvalidCacheLoadException e) {
 +          LOG.debug("Found no stats for column " + colName);
 +          // This means we have no stats at all for this column for these partitions, so just
 +          // move on.
 +        }
 +      }
 +      commit = true;
 +      return aggrStats;
 +    } catch (IOException e) {
 +      LOG.error("Unable to fetch aggregate column statistics", e);
 +      throw new MetaException("Failed fetching aggregate column statistics, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public long cleanupEvents() {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public boolean addToken(String tokenIdentifier, String delegationToken) {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().putDelegationToken(tokenIdentifier, delegationToken);
 +      commit = true;
 +      return commit; // See HIVE-11302, for now always returning true
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean removeToken(String tokenIdentifier) {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().deleteDelegationToken(tokenIdentifier);
 +      commit = true;
 +      return commit; // See HIVE-11302, for now always returning true
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public String getToken(String tokenIdentifier) {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      String token = getHBase().getDelegationToken(tokenIdentifier);
 +      commit = true;
 +      return token;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<String> getAllTokenIdentifiers() {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<String> ids = getHBase().scanDelegationTokenIdentifiers();
 +      commit = true;
 +      return ids;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public int addMasterKey(String key) throws MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      long seq = getHBase().getNextSequence(HBaseReadWrite.MASTER_KEY_SEQUENCE);
 +      getHBase().putMasterKey((int) seq, key);
 +      commit = true;
 +      return (int)seq;
 +    } catch (IOException e) {
 +      LOG.error("Unable to add master key", e);
 +      throw new MetaException("Failed adding master key, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public void updateMasterKey(Integer seqNo, String key) throws NoSuchObjectException,
 +      MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      if (getHBase().getMasterKey(seqNo) == null) {
 +        throw new NoSuchObjectException("No key found with keyId: " + seqNo);
 +      }
 +      getHBase().putMasterKey(seqNo, key);
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to update master key", e);
 +      throw new MetaException("Failed updating master key, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public boolean removeMasterKey(Integer keySeq) {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().deleteMasterKey(keySeq);
 +      commit = true;
 +      return true;  // See HIVE-11302, for now always returning true
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public String[] getMasterKeys() {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<String> keys = getHBase().scanMasterKeys();
 +      commit = true;
 +      return keys.toArray(new String[keys.size()]);
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public void verifySchema() throws MetaException {
 +
 +  }
 +
 +  @Override
 +  public String getMetaStoreSchemaVersion() throws MetaException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void setMetaStoreSchemaVersion(String version, String comment) throws MetaException {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void dropPartitions(String dbName, String tblName, List<String> partNames) throws
 +      MetaException, NoSuchObjectException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      for (String partName : partNames) {
 +        dropPartition(dbName, tblName, partNameToVals(partName));
 +      }
 +      commit = true;
 +    } catch (Exception e) {
 +      LOG.error("Unable to drop partitions", e);
 +      throw new NoSuchObjectException("Failure dropping partitions, " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalDBGrantsAll(String principalName,
 +                                                            PrincipalType principalType) {
 +    List<HiveObjectPrivilege> privileges = new ArrayList<HiveObjectPrivilege>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Database> dbs = getHBase().scanDatabases(null);
 +      for (Database db : dbs) {
 +        List<PrivilegeGrantInfo> grants;
 +
 +        PrincipalPrivilegeSet pps = db.getPrivileges();
 +        if (pps == null) continue;
 +        Map<String, List<PrivilegeGrantInfo>> map;
 +        switch (principalType) {
 +          case USER:
 +            map = pps.getUserPrivileges();
 +            break;
 +
 +          case ROLE:
 +            map = pps.getRolePrivileges();
 +            break;
 +
 +          default:
 +            throw new RuntimeException("Unknown or unsupported principal type " +
 +                principalType.toString());
 +        }
 +
 +        if (map == null) continue;
 +        grants = map.get(principalName);
 +        if (grants == null || grants.size() == 0) continue;
 +        for (PrivilegeGrantInfo pgi : grants) {
 +          privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.DATABASE,
 +              db.getName(), null, null, null), principalName, principalType, pgi));
 +        }
 +      }
 +      commit = true;
 +      return privileges;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalTableGrantsAll(String principalName,
 +                                                               PrincipalType principalType) {
 +    List<HiveObjectPrivilege> privileges = new ArrayList<HiveObjectPrivilege>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Table> tables = getHBase().scanTables(null, null);
 +      for (Table table : tables) {
 +        List<PrivilegeGrantInfo> grants;
 +
 +        PrincipalPrivilegeSet pps = table.getPrivileges();
 +        if (pps == null) continue;
 +        Map<String, List<PrivilegeGrantInfo>> map;
 +        switch (principalType) {
 +          case USER:
 +            map = pps.getUserPrivileges();
 +            break;
 +
 +          case ROLE:
 +            map = pps.getRolePrivileges();
 +            break;
 +
 +          default:
 +            throw new RuntimeException("Unknown or unsupported principal type " +
 +                principalType.toString());
 +        }
 +
 +        if (map == null) continue;
 +        grants = map.get(principalName);
 +        if (grants == null || grants.size() == 0) continue;
 +        for (PrivilegeGrantInfo pgi : grants) {
 +          privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.TABLE,
 +              table.getDbName(), table.getTableName(), null, null), principalName, principalType,
 +              pgi));
 +        }
 +      }
 +      commit = true;
 +      return privileges;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalPartitionGrantsAll(String principalName,
 +                                                                   PrincipalType principalType) {
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalTableColumnGrantsAll(String principalName,
 +                                                                     PrincipalType principalType) {
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(String principalName,
 +                                                                         PrincipalType principalType) {
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listGlobalGrantsAll() {
 +    List<HiveObjectPrivilege> privileges = new ArrayList<HiveObjectPrivilege>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      PrincipalPrivilegeSet pps = getHBase().getGlobalPrivs();
 +      if (pps != null) {
 +        for (Map.Entry<String, List<PrivilegeGrantInfo>> e : pps.getUserPrivileges().entrySet()) {
 +          for (PrivilegeGrantInfo pgi : e.getValue()) {
 +            privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.GLOBAL, null,
 +                null, null, null), e.getKey(), PrincipalType.USER, pgi));
 +          }
 +        }
 +        for (Map.Entry<String, List<PrivilegeGrantInfo>> e : pps.getRolePrivileges().entrySet()) {
 +          for (PrivilegeGrantInfo pgi : e.getValue()) {
 +            privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.GLOBAL, null,
 +                null, null, null), e.getKey(), PrincipalType.ROLE, pgi));
 +          }
 +        }
 +      }
 +      commit = true;
 +      return privileges;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listDBGrantsAll(String dbName) {
 +    List<HiveObjectPrivilege> privileges = new ArrayList<HiveObjectPrivilege>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Database db = getHBase().getDb(dbName);
 +      PrincipalPrivilegeSet pps = db.getPrivileges();
 +      if (pps != null) {
 +        for (Map.Entry<String, List<PrivilegeGrantInfo>> e : pps.getUserPrivileges().entrySet()) {
 +          for (PrivilegeGrantInfo pgi : e.getValue()) {
 +            privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.DATABASE, dbName,
 +                null, null, null), e.getKey(), PrincipalType.USER, pgi));
 +          }
 +        }
 +        for (Map.Entry<String, List<PrivilegeGrantInfo>> e : pps.getRolePrivileges().entrySet()) {
 +          for (PrivilegeGrantInfo pgi : e.getValue()) {
 +            privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.DATABASE, dbName,
 +                null, null, null), e.getKey(), PrincipalType.ROLE, pgi));
 +          }
 +        }
 +      }
 +      commit = true;
 +      return privileges;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String dbName, String tableName,
 +                                                                String partitionName,
 +                                                                String columnName) {
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listTableGrantsAll(String dbName, String tableName) {
 +    List<HiveObjectPrivilege> privileges = new ArrayList<HiveObjectPrivilege>();
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Table table = getHBase().getTable(dbName, tableName);
 +      PrincipalPrivilegeSet pps = table.getPrivileges();
 +      if (pps != null) {
 +        for (Map.Entry<String, List<PrivilegeGrantInfo>> e : pps.getUserPrivileges().entrySet()) {
 +          for (PrivilegeGrantInfo pgi : e.getValue()) {
 +            privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.TABLE, dbName,
 +                tableName, null, null), e.getKey(), PrincipalType.USER, pgi));
 +          }
 +        }
 +        for (Map.Entry<String, List<PrivilegeGrantInfo>> e : pps.getRolePrivileges().entrySet()) {
 +          for (PrivilegeGrantInfo pgi : e.getValue()) {
 +            privileges.add(new HiveObjectPrivilege(new HiveObjectRef(HiveObjectType.TABLE, dbName,
 +                tableName, null, null), e.getKey(), PrincipalType.ROLE, pgi));
 +          }
 +        }
 +      }
 +      commit = true;
 +      return privileges;
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPartitionGrantsAll(String dbName, String tableName,
 +                                                          String partitionName) {
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String dbName, String tableName,
 +                                                            String columnName) {
 +    return new ArrayList<HiveObjectPrivilege>();
 +  }
 +
 +  @Override
 +  public void createFunction(Function func) throws InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().putFunction(func);
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to create function", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public void alterFunction(String dbName, String funcName, Function newFunction) throws
 +      InvalidObjectException, MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().putFunction(newFunction);
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to alter function ", e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public void dropFunction(String dbName, String funcName) throws MetaException,
 +      NoSuchObjectException, InvalidObjectException, InvalidInputException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      getHBase().deleteFunction(dbName, funcName);
 +      commit = true;
 +    } catch (IOException e) {
 +      LOG.error("Unable to delete function" + e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public Function getFunction(String dbName, String funcName) throws MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      Function func = getHBase().getFunction(dbName, funcName);
 +      commit = true;
 +      return func;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get function" + e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
++  public List<Function> getAllFunctions() throws MetaException {
++    boolean commit = false;
++    openTransaction();
++    try {
++      List<Function> funcs = getHBase().scanFunctions(null, ".*");
++      commit = true;
++      return funcs;
++    } catch (IOException e) {
++      LOG.error("Unable to get functions" + e);
++      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
++    } finally {
++      commitOrRoleBack(commit);
++    }
++  }
++
++  @Override
 +  public List<String> getFunctions(String dbName, String pattern) throws MetaException {
 +    boolean commit = false;
 +    openTransaction();
 +    try {
 +      List<Function> funcs = getHBase().scanFunctions(dbName, likeToRegex(pattern));
 +      List<String> funcNames = new ArrayList<String>(funcs.size());
 +      for (Function func : funcs) funcNames.add(func.getFunctionName());
 +      commit = true;
 +      return funcNames;
 +    } catch (IOException e) {
 +      LOG.error("Unable to get functions" + e);
 +      throw new MetaException("Unable to read from or write to hbase " + e.getMessage());
 +    } finally {
 +      commitOrRoleBack(commit);
 +    }
 +  }
 +
 +  @Override
 +  public NotificationEventResponse getNextNotification(NotificationEventRequest rqst) {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void addNotificationEvent(NotificationEvent event) {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void cleanNotificationEvents(int olderThan) {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public CurrentNotificationEventId getCurrentNotificationEventId() {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void flushCache() {
 +    getHBase().flushCatalogCache();
 +  }
 +
 +  @Override
 +  public void setConf(Configuration configuration) {
 +    // initialize expressionProxy. Also re-initialize it if
 +    // setConf is being called with new configuration object (though that
 +    // is not expected to happen, doing it just for safety)
 +    if(expressionProxy == null || conf != configuration) {
 +      expressionProxy = PartFilterExprUtil.createExpressionProxy(configuration);
 +    }
 +    conf = configuration;
 +  }
 +
 +  @Override
 +  public Configuration getConf() {
 +    return conf;
 +
 +  }
 +
 +  private HBaseReadWrite getHBase() {
 +    if (hbase == null) hbase = HBaseReadWrite.getInstance(conf);
 +    return hbase;
 +  }
 +
 +  // This is for building error messages only.  It does not look up anything in the metastore.
 +  private String tableNameForErrorMsg(String dbName, String tableName) {
 +    return dbName + "." + tableName;
 +  }
 +
 +  // This is for building error messages only.  It does not look up anything in the metastore as
 +  // they may just throw another error.
 +  private String partNameForErrorMsg(String dbName, String tableName, List<String> partVals) {
 +    return tableNameForErrorMsg(dbName, tableName) + "." + StringUtils.join(partVals, ':');
 +  }
 +
 +  private String buildExternalPartName(Table table, Partition part) {
 +    return buildExternalPartName(table, part.getValues());
 +  }
 +
 +  private String buildExternalPartName(String dbName, String tableName, List<String> partVals)
 +      throws MetaException {
 +    return buildExternalPartName(getTable(dbName, tableName), partVals);
 +  }
 +
 +  private Set<String> findUsersToRemapRolesFor(Role role, String principalName, PrincipalType type)
 +      throws IOException, NoSuchObjectException {
 +    Set<String> usersToRemap;
 +    switch (type) {
 +      case USER:
 +        // In this case it's just the user being added to the role that we need to remap for.
 +        usersToRemap = new HashSet<String>();
 +        usersToRemap.add(principalName);
 +        break;
 +
 +      case ROLE:
 +        // In this case we need to remap for all users in the containing role (not the role being
 +        // granted into the containing role).
 +        usersToRemap = getHBase().findAllUsersInRole(role.getRoleName());
 +        break;
 +
 +      default:
 +        throw new RuntimeException("Unknown principal type " + type);
 +
 +    }
 +    return usersToRemap;
 +  }
 +
 +  /**
 +   * Build a partition name for external use.  Necessary since HBase itself doesn't store
 +   * partition names.
 +   * @param table  table object
 +   * @param partVals partition values.
 +   * @return
 +   */
 +  static String buildExternalPartName(Table table, List<String> partVals) {
 +    List<String> partCols = new ArrayList<String>();
 +    for (FieldSchema pc : table.getPartitionKeys()) partCols.add(pc.getName());
 +    return FileUtils.makePartName(partCols, partVals);
 +  }
 +
 +  private static List<String> partNameToVals(String name) {
 +    if (name == null) return null;
 +    List<String> vals = new ArrayList<String>();
 +    String[] kvp = name.split("/");
 +    for (String kv : kvp) {
 +      vals.add(kv.substring(kv.indexOf('=') + 1));
 +    }
 +    return vals;
 +  }
 +
 +  static List<List<String>> partNameListToValsList(List<String> partNames) {
 +    List<List<String>> valLists = new ArrayList<List<String>>(partNames.size());
 +    for (String partName : partNames) {
 +      valLists.add(partNameToVals(partName));
 +    }
 +    return valLists;
 +  }
 +
 +  private String likeToRegex(String like) {
 +    if (like == null) return null;
 +    // Convert Hive's strange like syntax to Java regex.  Per
 +    // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Show
 +    // the supported syntax is that * means Java .* and | means 'or'
 +    // This implementation leaves other regular expression syntax alone, which means people can
 +    // use it, even though it wouldn't work on RDBMS backed metastores.
 +    return like.replace("*", ".*");
 +  }
 +
 +  private void commitOrRoleBack(boolean commit) {
 +    if (commit) {
 +      LOG.debug("Committing transaction");
 +      commitTransaction();
 +    } else {
 +      LOG.debug("Rolling back transaction");
 +      rollbackTransaction();
 +    }
 +  }
 +
 +  @VisibleForTesting HBaseReadWrite backdoor() {
 +    return getHBase();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --cc metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 9831eb9,f184c56..6efadba
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@@ -19,8 -19,10 +19,9 @@@
  package org.apache.hadoop.hive.metastore;
  
  import java.util.ArrayList;
+ import java.util.Collections;
  import java.util.List;
  import java.util.Map;
 -import java.util.SortedSet;
  
  import org.apache.hadoop.conf.Configurable;
  import org.apache.hadoop.conf.Configuration;

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java
----------------------------------------------------------------------
diff --cc ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java
index 22ca225,5ea9b6e..b26ab96
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Adjacency implements org.apache.thrift.TBase<Adjacency, Adjacency._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Adjacency implements org.apache.thrift.TBase<Adjacency, Adjacency._Fields>, java.io.Serializable, Cloneable, Comparable<Adjacency> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Adjacency");
  
    private static final org.apache.thrift.protocol.TField NODE_FIELD_DESC = new org.apache.thrift.protocol.TField("node", org.apache.thrift.protocol.TType.STRING, (short)1);


[11/52] [abbrv] hive git commit: HIVE-11568 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java
----------------------------------------------------------------------
diff --cc ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java
index 35aa6cb,e621cfa..0a13175
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Graph implements org.apache.thrift.TBase<Graph, Graph._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Graph implements org.apache.thrift.TBase<Graph, Graph._Fields>, java.io.Serializable, Cloneable, Comparable<Graph> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Graph");
  
    private static final org.apache.thrift.protocol.TField NODE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("nodeType", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java
----------------------------------------------------------------------
diff --cc ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java
index f1c9e2d,1b18aab..991974c
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Operator implements org.apache.thrift.TBase<Operator, Operator._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Operator implements org.apache.thrift.TBase<Operator, Operator._Fields>, java.io.Serializable, Cloneable, Comparable<Operator> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Operator");
  
    private static final org.apache.thrift.protocol.TField OPERATOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("operatorId", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java
----------------------------------------------------------------------
diff --cc ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java
index e0d77e8,5c5e0f8..f98a7e1
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Query implements org.apache.thrift.TBase<Query, Query._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Query implements org.apache.thrift.TBase<Query, Query._Fields>, java.io.Serializable, Cloneable, Comparable<Query> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Query");
  
    private static final org.apache.thrift.protocol.TField QUERY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("queryId", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java
----------------------------------------------------------------------
diff --cc ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java
index e8566a5,d340d58..0994fda
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class QueryPlan implements org.apache.thrift.TBase<QueryPlan, QueryPlan._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class QueryPlan implements org.apache.thrift.TBase<QueryPlan, QueryPlan._Fields>, java.io.Serializable, Cloneable, Comparable<QueryPlan> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("QueryPlan");
  
    private static final org.apache.thrift.protocol.TField QUERIES_FIELD_DESC = new org.apache.thrift.protocol.TField("queries", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java
----------------------------------------------------------------------
diff --cc ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java
index c341db2,7353933..e0cd86c
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Stage implements org.apache.thrift.TBase<Stage, Stage._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Stage implements org.apache.thrift.TBase<Stage, Stage._Fields>, java.io.Serializable, Cloneable, Comparable<Stage> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Stage");
  
    private static final org.apache.thrift.protocol.TField STAGE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("stageId", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java
----------------------------------------------------------------------
diff --cc ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java
index fc4313f,09a2e7f..66e5e30
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Task implements org.apache.thrift.TBase<Task, Task._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Task implements org.apache.thrift.TBase<Task, Task._Fields>, java.io.Serializable, Cloneable, Comparable<Task> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Task");
  
    private static final org.apache.thrift.protocol.TField TASK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("taskId", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 7f918f8,c0c1b2e..ff3e6c8
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@@ -413,14 -409,8 +409,14 @@@ public class Driver implements CommandP
            getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK,
                HiveSemanticAnalyzerHook.class);
  
 +      // Flush the metastore cache.  This assures that we don't pick up objects from a previous
 +      // query running in this same thread.  This has to be done after we get our semantic
 +      // analyzer (this is when the connection to the metastore is made) but before we analyze,
 +      // because at that point we need access to the objects.
 +      Hive.get().getMSC().flushCache();
 +
        // Do semantic analysis and plan generation
-       if (saHooks != null) {
+       if (saHooks != null && !saHooks.isEmpty()) {
          HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
          hookCtx.setConf(conf);
          hookCtx.setUserName(userName);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java
----------------------------------------------------------------------
diff --cc serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java
index 72a28ae,db325af..68bb885
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class InnerStruct implements org.apache.thrift.TBase<InnerStruct, InnerStruct._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class InnerStruct implements org.apache.thrift.TBase<InnerStruct, InnerStruct._Fields>, java.io.Serializable, Cloneable, Comparable<InnerStruct> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InnerStruct");
  
    private static final org.apache.thrift.protocol.TField FIELD0_FIELD_DESC = new org.apache.thrift.protocol.TField("field0", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java
----------------------------------------------------------------------
diff --cc serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java
index 1b708dd,1232ff9..208fa82
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ThriftTestObj implements org.apache.thrift.TBase<ThriftTestObj, ThriftTestObj._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ThriftTestObj implements org.apache.thrift.TBase<ThriftTestObj, ThriftTestObj._Fields>, java.io.Serializable, Cloneable, Comparable<ThriftTestObj> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ThriftTestObj");
  
    private static final org.apache.thrift.protocol.TField FIELD1_FIELD_DESC = new org.apache.thrift.protocol.TField("field1", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java
----------------------------------------------------------------------
diff --cc serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java
index 07ea8b9,4d2f5bf..6d32947
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Complex implements org.apache.thrift.TBase<Complex, Complex._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Complex implements org.apache.thrift.TBase<Complex, Complex._Fields>, java.io.Serializable, Cloneable, Comparable<Complex> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Complex");
  
    private static final org.apache.thrift.protocol.TField AINT_FIELD_DESC = new org.apache.thrift.protocol.TField("aint", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java
----------------------------------------------------------------------
diff --cc serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java
index bd580ad,23d7363..26e7b38
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class IntString implements org.apache.thrift.TBase<IntString, IntString._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class IntString implements org.apache.thrift.TBase<IntString, IntString._Fields>, java.io.Serializable, Cloneable, Comparable<IntString> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IntString");
  
    private static final org.apache.thrift.protocol.TField MYINT_FIELD_DESC = new org.apache.thrift.protocol.TField("myint", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java
----------------------------------------------------------------------
diff --cc serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java
index 386fef9,9447708..d937a9c
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class MegaStruct implements org.apache.thrift.TBase<MegaStruct, MegaStruct._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class MegaStruct implements org.apache.thrift.TBase<MegaStruct, MegaStruct._Fields>, java.io.Serializable, Cloneable, Comparable<MegaStruct> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MegaStruct");
  
    private static final org.apache.thrift.protocol.TField MY_BOOL_FIELD_DESC = new org.apache.thrift.protocol.TField("my_bool", org.apache.thrift.protocol.TType.BOOL, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java
----------------------------------------------------------------------
diff --cc serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java
index b1d3946,d8c46f4..c25156a
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class MiniStruct implements org.apache.thrift.TBase<MiniStruct, MiniStruct._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class MiniStruct implements org.apache.thrift.TBase<MiniStruct, MiniStruct._Fields>, java.io.Serializable, Cloneable, Comparable<MiniStruct> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MiniStruct");
  
    private static final org.apache.thrift.protocol.TField MY_STRING_FIELD_DESC = new org.apache.thrift.protocol.TField("my_string", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java
----------------------------------------------------------------------
diff --cc serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java
index 676f2b2,58498b0..d1bd61d
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class SetIntString implements org.apache.thrift.TBase<SetIntString, SetIntString._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class SetIntString implements org.apache.thrift.TBase<SetIntString, SetIntString._Fields>, java.io.Serializable, Cloneable, Comparable<SetIntString> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SetIntString");
  
    private static final org.apache.thrift.protocol.TField S_INT_STRING_FIELD_DESC = new org.apache.thrift.protocol.TField("sIntString", org.apache.thrift.protocol.TType.SET, (short)1);


[16/52] [abbrv] hive git commit: HIVE-11568 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index 939c15a,e9088e0..c6ad69e
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequest, CompactionRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequest, CompactionRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CompactionRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CompactionRequest");
  
    private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java
index bdcf21d,000670a..99e7a83
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ConfigValSecurityException extends TException implements org.apache.thrift.TBase<ConfigValSecurityException, ConfigValSecurityException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ConfigValSecurityException extends TException implements org.apache.thrift.TBase<ConfigValSecurityException, ConfigValSecurityException._Fields>, java.io.Serializable, Cloneable, Comparable<ConfigValSecurityException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ConfigValSecurityException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java
index 8b8e5c4,d55d874..89abc78
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class CurrentNotificationEventId implements org.apache.thrift.TBase<CurrentNotificationEventId, CurrentNotificationEventId._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class CurrentNotificationEventId implements org.apache.thrift.TBase<CurrentNotificationEventId, CurrentNotificationEventId._Fields>, java.io.Serializable, Cloneable, Comparable<CurrentNotificationEventId> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CurrentNotificationEventId");
  
    private static final org.apache.thrift.protocol.TField EVENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("eventId", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
index c3c531d,56b7281..759eec9
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Database implements org.apache.thrift.TBase<Database, Database._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Database implements org.apache.thrift.TBase<Database, Database._Fields>, java.io.Serializable, Cloneable, Comparable<Database> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Database");
  
    private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java
index 3e02db5,0b406d6..b4a44a4
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Date implements org.apache.thrift.TBase<Date, Date._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Date implements org.apache.thrift.TBase<Date, Date._Fields>, java.io.Serializable, Cloneable, Comparable<Date> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Date");
  
    private static final org.apache.thrift.protocol.TField DAYS_SINCE_EPOCH_FIELD_DESC = new org.apache.thrift.protocol.TField("daysSinceEpoch", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java
index e9a577d,7a3d4ed..7050334
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class DateColumnStatsData implements org.apache.thrift.TBase<DateColumnStatsData, DateColumnStatsData._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class DateColumnStatsData implements org.apache.thrift.TBase<DateColumnStatsData, DateColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<DateColumnStatsData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DateColumnStatsData");
  
    private static final org.apache.thrift.protocol.TField LOW_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lowValue", org.apache.thrift.protocol.TType.STRUCT, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java
index ed8bb18,9215ce9..1f82543
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Decimal implements org.apache.thrift.TBase<Decimal, Decimal._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Decimal implements org.apache.thrift.TBase<Decimal, Decimal._Fields>, java.io.Serializable, Cloneable, Comparable<Decimal> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Decimal");
  
    private static final org.apache.thrift.protocol.TField UNSCALED_FIELD_DESC = new org.apache.thrift.protocol.TField("unscaled", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java
index 951d479,e64ca36..02092dc
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class DecimalColumnStatsData implements org.apache.thrift.TBase<DecimalColumnStatsData, DecimalColumnStatsData._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class DecimalColumnStatsData implements org.apache.thrift.TBase<DecimalColumnStatsData, DecimalColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<DecimalColumnStatsData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DecimalColumnStatsData");
  
    private static final org.apache.thrift.protocol.TField LOW_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lowValue", org.apache.thrift.protocol.TType.STRUCT, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java
index 4203fd8,2509ed5..52288e5
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class DoubleColumnStatsData implements org.apache.thrift.TBase<DoubleColumnStatsData, DoubleColumnStatsData._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class DoubleColumnStatsData implements org.apache.thrift.TBase<DoubleColumnStatsData, DoubleColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<DoubleColumnStatsData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DoubleColumnStatsData");
  
    private static final org.apache.thrift.protocol.TField LOW_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lowValue", org.apache.thrift.protocol.TType.DOUBLE, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java
index 59a66f3,5e3a2d1..0d1e50d
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class DropPartitionsExpr implements org.apache.thrift.TBase<DropPartitionsExpr, DropPartitionsExpr._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class DropPartitionsExpr implements org.apache.thrift.TBase<DropPartitionsExpr, DropPartitionsExpr._Fields>, java.io.Serializable, Cloneable, Comparable<DropPartitionsExpr> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DropPartitionsExpr");
  
    private static final org.apache.thrift.protocol.TField EXPR_FIELD_DESC = new org.apache.thrift.protocol.TField("expr", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
index 1923f38,24536ba..46cc9a7
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class DropPartitionsRequest implements org.apache.thrift.TBase<DropPartitionsRequest, DropPartitionsRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class DropPartitionsRequest implements org.apache.thrift.TBase<DropPartitionsRequest, DropPartitionsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<DropPartitionsRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DropPartitionsRequest");
  
    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
index b7f69f2,c139e65..09da136
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartitionsResult, DropPartitionsResult._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartitionsResult, DropPartitionsResult._Fields>, java.io.Serializable, Cloneable, Comparable<DropPartitionsResult> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DropPartitionsResult");
  
    private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
index ab0b399,6accb8d..3eabc86
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class EnvironmentContext implements org.apache.thrift.TBase<EnvironmentContext, EnvironmentContext._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class EnvironmentContext implements org.apache.thrift.TBase<EnvironmentContext, EnvironmentContext._Fields>, java.io.Serializable, Cloneable, Comparable<EnvironmentContext> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("EnvironmentContext");
  
    private static final org.apache.thrift.protocol.TField PROPERTIES_FIELD_DESC = new org.apache.thrift.protocol.TField("properties", org.apache.thrift.protocol.TType.MAP, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
index a993810,ba69622..e73edd4
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class FieldSchema implements org.apache.thrift.TBase<FieldSchema, FieldSchema._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class FieldSchema implements org.apache.thrift.TBase<FieldSchema, FieldSchema._Fields>, java.io.Serializable, Cloneable, Comparable<FieldSchema> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FieldSchema");
  
    private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 44b83da,f3d439c..25f9d54
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class FireEventRequest implements org.apache.thrift.TBase<FireEventRequest, FireEventRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class FireEventRequest implements org.apache.thrift.TBase<FireEventRequest, FireEventRequest._Fields>, java.io.Serializable, Cloneable, Comparable<FireEventRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FireEventRequest");
  
    private static final org.apache.thrift.protocol.TField SUCCESSFUL_FIELD_DESC = new org.apache.thrift.protocol.TField("successful", org.apache.thrift.protocol.TType.BOOL, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java
index 051f411,d95ae06..6f277aa
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class FireEventResponse implements org.apache.thrift.TBase<FireEventResponse, FireEventResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class FireEventResponse implements org.apache.thrift.TBase<FireEventResponse, FireEventResponse._Fields>, java.io.Serializable, Cloneable, Comparable<FireEventResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FireEventResponse");
  
  

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
index c40e33d,50eff73..33c617e
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Function implements org.apache.thrift.TBase<Function, Function._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Function implements org.apache.thrift.TBase<Function, Function._Fields>, java.io.Serializable, Cloneable, Comparable<Function> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Function");
  
    private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index 0000000,0a9e27b..170d8e7
mode 000000,100644..100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@@ -1,0 -1,447 +1,447 @@@
+ /**
+  * Autogenerated by Thrift Compiler (0.9.2)
+  *
+  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+  *  @generated
+  */
+ package org.apache.hadoop.hive.metastore.api;
+ 
+ import org.apache.thrift.scheme.IScheme;
+ import org.apache.thrift.scheme.SchemeFactory;
+ import org.apache.thrift.scheme.StandardScheme;
+ 
+ import org.apache.thrift.scheme.TupleScheme;
+ import org.apache.thrift.protocol.TTupleProtocol;
+ import org.apache.thrift.protocol.TProtocolException;
+ import org.apache.thrift.EncodingUtils;
+ import org.apache.thrift.TException;
+ import org.apache.thrift.async.AsyncMethodCallback;
+ import org.apache.thrift.server.AbstractNonblockingServer.*;
+ import java.util.List;
+ import java.util.ArrayList;
+ import java.util.Map;
+ import java.util.HashMap;
+ import java.util.EnumMap;
+ import java.util.Set;
+ import java.util.HashSet;
+ import java.util.EnumSet;
+ import java.util.Collections;
+ import java.util.BitSet;
+ import java.nio.ByteBuffer;
+ import java.util.Arrays;
+ import javax.annotation.Generated;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFunctionsResponse, GetAllFunctionsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetAllFunctionsResponse> {
+   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetAllFunctionsResponse");
+ 
+   private static final org.apache.thrift.protocol.TField FUNCTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("functions", org.apache.thrift.protocol.TType.LIST, (short)1);
+ 
+   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+   static {
+     schemes.put(StandardScheme.class, new GetAllFunctionsResponseStandardSchemeFactory());
+     schemes.put(TupleScheme.class, new GetAllFunctionsResponseTupleSchemeFactory());
+   }
+ 
+   private List<Function> functions; // optional
+ 
+   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+     FUNCTIONS((short)1, "functions");
+ 
+     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+ 
+     static {
+       for (_Fields field : EnumSet.allOf(_Fields.class)) {
+         byName.put(field.getFieldName(), field);
+       }
+     }
+ 
+     /**
+      * Find the _Fields constant that matches fieldId, or null if its not found.
+      */
+     public static _Fields findByThriftId(int fieldId) {
+       switch(fieldId) {
+         case 1: // FUNCTIONS
+           return FUNCTIONS;
+         default:
+           return null;
+       }
+     }
+ 
+     /**
+      * Find the _Fields constant that matches fieldId, throwing an exception
+      * if it is not found.
+      */
+     public static _Fields findByThriftIdOrThrow(int fieldId) {
+       _Fields fields = findByThriftId(fieldId);
+       if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+       return fields;
+     }
+ 
+     /**
+      * Find the _Fields constant that matches name, or null if its not found.
+      */
+     public static _Fields findByName(String name) {
+       return byName.get(name);
+     }
+ 
+     private final short _thriftId;
+     private final String _fieldName;
+ 
+     _Fields(short thriftId, String fieldName) {
+       _thriftId = thriftId;
+       _fieldName = fieldName;
+     }
+ 
+     public short getThriftFieldId() {
+       return _thriftId;
+     }
+ 
+     public String getFieldName() {
+       return _fieldName;
+     }
+   }
+ 
+   // isset id assignments
+   private static final _Fields optionals[] = {_Fields.FUNCTIONS};
+   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+   static {
+     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+     tmpMap.put(_Fields.FUNCTIONS, new org.apache.thrift.meta_data.FieldMetaData("functions", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Function.class))));
+     metaDataMap = Collections.unmodifiableMap(tmpMap);
+     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetAllFunctionsResponse.class, metaDataMap);
+   }
+ 
+   public GetAllFunctionsResponse() {
+   }
+ 
+   /**
+    * Performs a deep copy on <i>other</i>.
+    */
+   public GetAllFunctionsResponse(GetAllFunctionsResponse other) {
+     if (other.isSetFunctions()) {
+       List<Function> __this__functions = new ArrayList<Function>(other.functions.size());
+       for (Function other_element : other.functions) {
+         __this__functions.add(new Function(other_element));
+       }
+       this.functions = __this__functions;
+     }
+   }
+ 
+   public GetAllFunctionsResponse deepCopy() {
+     return new GetAllFunctionsResponse(this);
+   }
+ 
+   @Override
+   public void clear() {
+     this.functions = null;
+   }
+ 
+   public int getFunctionsSize() {
+     return (this.functions == null) ? 0 : this.functions.size();
+   }
+ 
+   public java.util.Iterator<Function> getFunctionsIterator() {
+     return (this.functions == null) ? null : this.functions.iterator();
+   }
+ 
+   public void addToFunctions(Function elem) {
+     if (this.functions == null) {
+       this.functions = new ArrayList<Function>();
+     }
+     this.functions.add(elem);
+   }
+ 
+   public List<Function> getFunctions() {
+     return this.functions;
+   }
+ 
+   public void setFunctions(List<Function> functions) {
+     this.functions = functions;
+   }
+ 
+   public void unsetFunctions() {
+     this.functions = null;
+   }
+ 
+   /** Returns true if field functions is set (has been assigned a value) and false otherwise */
+   public boolean isSetFunctions() {
+     return this.functions != null;
+   }
+ 
+   public void setFunctionsIsSet(boolean value) {
+     if (!value) {
+       this.functions = null;
+     }
+   }
+ 
+   public void setFieldValue(_Fields field, Object value) {
+     switch (field) {
+     case FUNCTIONS:
+       if (value == null) {
+         unsetFunctions();
+       } else {
+         setFunctions((List<Function>)value);
+       }
+       break;
+ 
+     }
+   }
+ 
+   public Object getFieldValue(_Fields field) {
+     switch (field) {
+     case FUNCTIONS:
+       return getFunctions();
+ 
+     }
+     throw new IllegalStateException();
+   }
+ 
+   /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+   public boolean isSet(_Fields field) {
+     if (field == null) {
+       throw new IllegalArgumentException();
+     }
+ 
+     switch (field) {
+     case FUNCTIONS:
+       return isSetFunctions();
+     }
+     throw new IllegalStateException();
+   }
+ 
+   @Override
+   public boolean equals(Object that) {
+     if (that == null)
+       return false;
+     if (that instanceof GetAllFunctionsResponse)
+       return this.equals((GetAllFunctionsResponse)that);
+     return false;
+   }
+ 
+   public boolean equals(GetAllFunctionsResponse that) {
+     if (that == null)
+       return false;
+ 
+     boolean this_present_functions = true && this.isSetFunctions();
+     boolean that_present_functions = true && that.isSetFunctions();
+     if (this_present_functions || that_present_functions) {
+       if (!(this_present_functions && that_present_functions))
+         return false;
+       if (!this.functions.equals(that.functions))
+         return false;
+     }
+ 
+     return true;
+   }
+ 
+   @Override
+   public int hashCode() {
+     List<Object> list = new ArrayList<Object>();
+ 
+     boolean present_functions = true && (isSetFunctions());
+     list.add(present_functions);
+     if (present_functions)
+       list.add(functions);
+ 
+     return list.hashCode();
+   }
+ 
+   @Override
+   public int compareTo(GetAllFunctionsResponse other) {
+     if (!getClass().equals(other.getClass())) {
+       return getClass().getName().compareTo(other.getClass().getName());
+     }
+ 
+     int lastComparison = 0;
+ 
+     lastComparison = Boolean.valueOf(isSetFunctions()).compareTo(other.isSetFunctions());
+     if (lastComparison != 0) {
+       return lastComparison;
+     }
+     if (isSetFunctions()) {
+       lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functions, other.functions);
+       if (lastComparison != 0) {
+         return lastComparison;
+       }
+     }
+     return 0;
+   }
+ 
+   public _Fields fieldForId(int fieldId) {
+     return _Fields.findByThriftId(fieldId);
+   }
+ 
+   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+     schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+   }
+ 
+   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+     schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+   }
+ 
+   @Override
+   public String toString() {
+     StringBuilder sb = new StringBuilder("GetAllFunctionsResponse(");
+     boolean first = true;
+ 
+     if (isSetFunctions()) {
+       sb.append("functions:");
+       if (this.functions == null) {
+         sb.append("null");
+       } else {
+         sb.append(this.functions);
+       }
+       first = false;
+     }
+     sb.append(")");
+     return sb.toString();
+   }
+ 
+   public void validate() throws org.apache.thrift.TException {
+     // check for required fields
+     // check for sub-struct validity
+   }
+ 
+   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+     try {
+       write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+     } catch (org.apache.thrift.TException te) {
+       throw new java.io.IOException(te);
+     }
+   }
+ 
+   private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+     try {
+       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+     } catch (org.apache.thrift.TException te) {
+       throw new java.io.IOException(te);
+     }
+   }
+ 
+   private static class GetAllFunctionsResponseStandardSchemeFactory implements SchemeFactory {
+     public GetAllFunctionsResponseStandardScheme getScheme() {
+       return new GetAllFunctionsResponseStandardScheme();
+     }
+   }
+ 
+   private static class GetAllFunctionsResponseStandardScheme extends StandardScheme<GetAllFunctionsResponse> {
+ 
+     public void read(org.apache.thrift.protocol.TProtocol iprot, GetAllFunctionsResponse struct) throws org.apache.thrift.TException {
+       org.apache.thrift.protocol.TField schemeField;
+       iprot.readStructBegin();
+       while (true)
+       {
+         schemeField = iprot.readFieldBegin();
+         if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+           break;
+         }
+         switch (schemeField.id) {
+           case 1: // FUNCTIONS
+             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+               {
+                 org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
+                 struct.functions = new ArrayList<Function>(_list524.size);
+                 Function _elem525;
+                 for (int _i526 = 0; _i526 < _list524.size; ++_i526)
+                 {
+                   _elem525 = new Function();
+                   _elem525.read(iprot);
+                   struct.functions.add(_elem525);
+                 }
+                 iprot.readListEnd();
+               }
+               struct.setFunctionsIsSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           default:
+             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+         }
+         iprot.readFieldEnd();
+       }
+       iprot.readStructEnd();
+       struct.validate();
+     }
+ 
+     public void write(org.apache.thrift.protocol.TProtocol oprot, GetAllFunctionsResponse struct) throws org.apache.thrift.TException {
+       struct.validate();
+ 
+       oprot.writeStructBegin(STRUCT_DESC);
+       if (struct.functions != null) {
+         if (struct.isSetFunctions()) {
+           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
+           {
+             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
+             for (Function _iter527 : struct.functions)
+             {
+               _iter527.write(oprot);
+             }
+             oprot.writeListEnd();
+           }
+           oprot.writeFieldEnd();
+         }
+       }
+       oprot.writeFieldStop();
+       oprot.writeStructEnd();
+     }
+ 
+   }
+ 
+   private static class GetAllFunctionsResponseTupleSchemeFactory implements SchemeFactory {
+     public GetAllFunctionsResponseTupleScheme getScheme() {
+       return new GetAllFunctionsResponseTupleScheme();
+     }
+   }
+ 
+   private static class GetAllFunctionsResponseTupleScheme extends TupleScheme<GetAllFunctionsResponse> {
+ 
+     @Override
+     public void write(org.apache.thrift.protocol.TProtocol prot, GetAllFunctionsResponse struct) throws org.apache.thrift.TException {
+       TTupleProtocol oprot = (TTupleProtocol) prot;
+       BitSet optionals = new BitSet();
+       if (struct.isSetFunctions()) {
+         optionals.set(0);
+       }
+       oprot.writeBitSet(optionals, 1);
+       if (struct.isSetFunctions()) {
+         {
+           oprot.writeI32(struct.functions.size());
+           for (Function _iter528 : struct.functions)
+           {
+             _iter528.write(oprot);
+           }
+         }
+       }
+     }
+ 
+     @Override
+     public void read(org.apache.thrift.protocol.TProtocol prot, GetAllFunctionsResponse struct) throws org.apache.thrift.TException {
+       TTupleProtocol iprot = (TTupleProtocol) prot;
+       BitSet incoming = iprot.readBitSet(1);
+       if (incoming.get(0)) {
+         {
+           org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+           struct.functions = new ArrayList<Function>(_list529.size);
+           Function _elem530;
+           for (int _i531 = 0; _i531 < _list529.size; ++_i531)
+           {
+             _elem530 = new Function();
+             _elem530.read(iprot);
+             struct.functions.add(_elem530);
+           }
+         }
+         struct.setFunctionsIsSet(true);
+       }
+     }
+   }
+ 
+ }
+ 

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
index 9fe8d82,61dea16..8d2b62e
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenTxnsInfoResponse, GetOpenTxnsInfoResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenTxnsInfoResponse, GetOpenTxnsInfoResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetOpenTxnsInfoResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOpenTxnsInfoResponse");
  
    private static final org.apache.thrift.protocol.TField TXN_HIGH_WATER_MARK_FIELD_DESC = new org.apache.thrift.protocol.TField("txn_high_water_mark", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
index 917ecea,bd21de5..3d70d93
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsResponse, GetOpenTxnsResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsResponse, GetOpenTxnsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetOpenTxnsResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOpenTxnsResponse");
  
    private static final org.apache.thrift.protocol.TField TXN_HIGH_WATER_MARK_FIELD_DESC = new org.apache.thrift.protocol.TField("txn_high_water_mark", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java
index 565eba9,45e0cc1..1d34a40
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GetPrincipalsInRoleRequest implements org.apache.thrift.TBase<GetPrincipalsInRoleRequest, GetPrincipalsInRoleRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GetPrincipalsInRoleRequest implements org.apache.thrift.TBase<GetPrincipalsInRoleRequest, GetPrincipalsInRoleRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetPrincipalsInRoleRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPrincipalsInRoleRequest");
  
    private static final org.apache.thrift.protocol.TField ROLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("roleName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java
index 3ef6224,805b898..3db7c94
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GetPrincipalsInRoleResponse implements org.apache.thrift.TBase<GetPrincipalsInRoleResponse, GetPrincipalsInRoleResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GetPrincipalsInRoleResponse implements org.apache.thrift.TBase<GetPrincipalsInRoleResponse, GetPrincipalsInRoleResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetPrincipalsInRoleResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPrincipalsInRoleResponse");
  
    private static final org.apache.thrift.protocol.TField PRINCIPAL_GRANTS_FIELD_DESC = new org.apache.thrift.protocol.TField("principalGrants", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java
index 84b73c3,c2c7259..fb9eb88
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GetRoleGrantsForPrincipalRequest implements org.apache.thrift.TBase<GetRoleGrantsForPrincipalRequest, GetRoleGrantsForPrincipalRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GetRoleGrantsForPrincipalRequest implements org.apache.thrift.TBase<GetRoleGrantsForPrincipalRequest, GetRoleGrantsForPrincipalRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetRoleGrantsForPrincipalRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetRoleGrantsForPrincipalRequest");
  
    private static final org.apache.thrift.protocol.TField PRINCIPAL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("principal_name", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java
index 3ddc1ac,7156665..42dfa01
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GetRoleGrantsForPrincipalResponse implements org.apache.thrift.TBase<GetRoleGrantsForPrincipalResponse, GetRoleGrantsForPrincipalResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GetRoleGrantsForPrincipalResponse implements org.apache.thrift.TBase<GetRoleGrantsForPrincipalResponse, GetRoleGrantsForPrincipalResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetRoleGrantsForPrincipalResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetRoleGrantsForPrincipalResponse");
  
    private static final org.apache.thrift.protocol.TField PRINCIPAL_GRANTS_FIELD_DESC = new org.apache.thrift.protocol.TField("principalGrants", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java
index 29ce977,94e5a5e..8a042f7
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GrantRevokePrivilegeRequest implements org.apache.thrift.TBase<GrantRevokePrivilegeRequest, GrantRevokePrivilegeRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GrantRevokePrivilegeRequest implements org.apache.thrift.TBase<GrantRevokePrivilegeRequest, GrantRevokePrivilegeRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GrantRevokePrivilegeRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GrantRevokePrivilegeRequest");
  
    private static final org.apache.thrift.protocol.TField REQUEST_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("requestType", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java
index f86bf9f,1949aed..113a07f
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GrantRevokePrivilegeResponse implements org.apache.thrift.TBase<GrantRevokePrivilegeResponse, GrantRevokePrivilegeResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GrantRevokePrivilegeResponse implements org.apache.thrift.TBase<GrantRevokePrivilegeResponse, GrantRevokePrivilegeResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GrantRevokePrivilegeResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GrantRevokePrivilegeResponse");
  
    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java
index db17aef,ac3527a..c538b72
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GrantRevokeRoleRequest implements org.apache.thrift.TBase<GrantRevokeRoleRequest, GrantRevokeRoleRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GrantRevokeRoleRequest implements org.apache.thrift.TBase<GrantRevokeRoleRequest, GrantRevokeRoleRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GrantRevokeRoleRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GrantRevokeRoleRequest");
  
    private static final org.apache.thrift.protocol.TField REQUEST_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("requestType", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java
index b9acbb5,81ae31f..dd7ee80
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class GrantRevokeRoleResponse implements org.apache.thrift.TBase<GrantRevokeRoleResponse, GrantRevokeRoleResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class GrantRevokeRoleResponse implements org.apache.thrift.TBase<GrantRevokeRoleResponse, GrantRevokeRoleResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GrantRevokeRoleResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GrantRevokeRoleResponse");
  
    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java
index 04439b9,4a9ba7f..f0e2a60
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class HeartbeatRequest implements org.apache.thrift.TBase<HeartbeatRequest, HeartbeatRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class HeartbeatRequest implements org.apache.thrift.TBase<HeartbeatRequest, HeartbeatRequest._Fields>, java.io.Serializable, Cloneable, Comparable<HeartbeatRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HeartbeatRequest");
  
    private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java
index 062449a,dd6ed17..de4fe40
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class HeartbeatTxnRangeRequest implements org.apache.thrift.TBase<HeartbeatTxnRangeRequest, HeartbeatTxnRangeRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class HeartbeatTxnRangeRequest implements org.apache.thrift.TBase<HeartbeatTxnRangeRequest, HeartbeatTxnRangeRequest._Fields>, java.io.Serializable, Cloneable, Comparable<HeartbeatTxnRangeRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HeartbeatTxnRangeRequest");
  
    private static final org.apache.thrift.protocol.TField MIN_FIELD_DESC = new org.apache.thrift.protocol.TField("min", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index caf07d6,11bd82b..c8d2314
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<HeartbeatTxnRangeResponse, HeartbeatTxnRangeResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<HeartbeatTxnRangeResponse, HeartbeatTxnRangeResponse._Fields>, java.io.Serializable, Cloneable, Comparable<HeartbeatTxnRangeResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HeartbeatTxnRangeResponse");
  
    private static final org.apache.thrift.protocol.TField ABORTED_FIELD_DESC = new org.apache.thrift.protocol.TField("aborted", org.apache.thrift.protocol.TType.SET, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java
index bd7641d,931b4e1..58fb744
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class HiveObjectPrivilege implements org.apache.thrift.TBase<HiveObjectPrivilege, HiveObjectPrivilege._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class HiveObjectPrivilege implements org.apache.thrift.TBase<HiveObjectPrivilege, HiveObjectPrivilege._Fields>, java.io.Serializable, Cloneable, Comparable<HiveObjectPrivilege> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HiveObjectPrivilege");
  
    private static final org.apache.thrift.protocol.TField HIVE_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("hiveObject", org.apache.thrift.protocol.TType.STRUCT, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
index b22b211,2573cea..13a97ab
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class HiveObjectRef implements org.apache.thrift.TBase<HiveObjectRef, HiveObjectRef._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class HiveObjectRef implements org.apache.thrift.TBase<HiveObjectRef, HiveObjectRef._Fields>, java.io.Serializable, Cloneable, Comparable<HiveObjectRef> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HiveObjectRef");
  
    private static final org.apache.thrift.protocol.TField OBJECT_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("objectType", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
index cb3a435,81227c3..ea997ce
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Index implements org.apache.thrift.TBase<Index, Index._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Index implements org.apache.thrift.TBase<Index, Index._Fields>, java.io.Serializable, Cloneable, Comparable<Index> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Index");
  
    private static final org.apache.thrift.protocol.TField INDEX_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("indexName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java
index f90a65d,50517ed..a4bed6c
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class IndexAlreadyExistsException extends TException implements org.apache.thrift.TBase<IndexAlreadyExistsException, IndexAlreadyExistsException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class IndexAlreadyExistsException extends TException implements org.apache.thrift.TBase<IndexAlreadyExistsException, IndexAlreadyExistsException._Fields>, java.io.Serializable, Cloneable, Comparable<IndexAlreadyExistsException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IndexAlreadyExistsException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index c1992c0,2e14cf9..5c07c0b
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEventRequestData, InsertEventRequestData._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEventRequestData, InsertEventRequestData._Fields>, java.io.Serializable, Cloneable, Comparable<InsertEventRequestData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InsertEventRequestData");
  
    private static final org.apache.thrift.protocol.TField FILES_ADDED_FIELD_DESC = new org.apache.thrift.protocol.TField("filesAdded", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java
index 4b63e3f,87e2f6b..5992099
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class InvalidInputException extends TException implements org.apache.thrift.TBase<InvalidInputException, InvalidInputException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class InvalidInputException extends TException implements org.apache.thrift.TBase<InvalidInputException, InvalidInputException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidInputException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidInputException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java
index 2f1414e,14dd7d8..60e9b39
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class InvalidObjectException extends TException implements org.apache.thrift.TBase<InvalidObjectException, InvalidObjectException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class InvalidObjectException extends TException implements org.apache.thrift.TBase<InvalidObjectException, InvalidObjectException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidObjectException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidObjectException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java
index 9b6acb8,2e3f1e3..e565e1e
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class InvalidOperationException extends TException implements org.apache.thrift.TBase<InvalidOperationException, InvalidOperationException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class InvalidOperationException extends TException implements org.apache.thrift.TBase<InvalidOperationException, InvalidOperationException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidOperationException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidOperationException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java
index 99ffddb,1d9b565..00ff5ee
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class InvalidPartitionException extends TException implements org.apache.thrift.TBase<InvalidPartitionException, InvalidPartitionException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class InvalidPartitionException extends TException implements org.apache.thrift.TBase<InvalidPartitionException, InvalidPartitionException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidPartitionException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidPartitionException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
index 59c8199,319f8bb..106adaf
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class LockComponent implements org.apache.thrift.TBase<LockComponent, LockComponent._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class LockComponent implements org.apache.thrift.TBase<LockComponent, LockComponent._Fields>, java.io.Serializable, Cloneable, Comparable<LockComponent> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockComponent");
  
    private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index a5d8f9b,6894bfa..d319d0c
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockRequest._Fields>, java.io.Serializable, Cloneable, Comparable<LockRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockRequest");
  
    private static final org.apache.thrift.protocol.TField COMPONENT_FIELD_DESC = new org.apache.thrift.protocol.TField("component", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java
index c9ab465,08acaeb..0f4e115
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class LockResponse implements org.apache.thrift.TBase<LockResponse, LockResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class LockResponse implements org.apache.thrift.TBase<LockResponse, LockResponse._Fields>, java.io.Serializable, Cloneable, Comparable<LockResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockResponse");
  
    private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java
index d817d46,93d2386..3df92ab
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class LongColumnStatsData implements org.apache.thrift.TBase<LongColumnStatsData, LongColumnStatsData._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class LongColumnStatsData implements org.apache.thrift.TBase<LongColumnStatsData, LongColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<LongColumnStatsData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LongColumnStatsData");
  
    private static final org.apache.thrift.protocol.TField LOW_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lowValue", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
index 108bca9,883a1d4..ab00b5d
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class MetaException extends TException implements org.apache.thrift.TBase<MetaException, MetaException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class MetaException extends TException implements org.apache.thrift.TBase<MetaException, MetaException._Fields>, java.io.Serializable, Cloneable, Comparable<MetaException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MetaException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java
index 9c8bd0b,e214b36..501e857
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class NoSuchLockException extends TException implements org.apache.thrift.TBase<NoSuchLockException, NoSuchLockException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class NoSuchLockException extends TException implements org.apache.thrift.TBase<NoSuchLockException, NoSuchLockException._Fields>, java.io.Serializable, Cloneable, Comparable<NoSuchLockException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NoSuchLockException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);


[03/52] [abbrv] hive git commit: HIVE-11300 HBase metastore: Support token and master key methods (gates)

Posted by se...@apache.org.
HIVE-11300 HBase metastore: Support token and master key methods (gates)


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

Branch: refs/heads/llap
Commit: a310524c48f54cb3071395e11fd97538816261d8
Parents: c53c6f4
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jul 22 11:57:32 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jul 22 11:57:32 2015 -0700

----------------------------------------------------------------------
 .../hbase/TestHBaseStoreIntegration.java        |   44 +
 .../metastore/hbase/HbaseMetastoreProto.java    | 3754 +++++++++++-------
 .../hive/metastore/hbase/HBaseReadWrite.java    |  221 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |  101 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |  111 +-
 .../metastore/hbase/hbase_metastore_proto.proto |   13 +-
 .../hive/metastore/hbase/TestHBaseStore.java    |   51 -
 7 files changed, 2769 insertions(+), 1526 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a310524c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java
index 4ff01a4..8b0b431 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java
@@ -1747,4 +1747,48 @@ public class TestHBaseStoreIntegration extends HBaseIntegrationTests {
           statsList.get(i).getStatsObj().get(1).getStatsData().getStringStats().getNumDVs());
     }
   }
+
+  @Test
+  public void delegationToken() throws Exception {
+    store.addToken("abc", "def");
+    store.addToken("ghi", "jkl");
+
+    Assert.assertEquals("def", store.getToken("abc"));
+    Assert.assertEquals("jkl", store.getToken("ghi"));
+    Assert.assertNull(store.getToken("wabawaba"));
+    String[] allToks = store.getAllTokenIdentifiers().toArray(new String[2]);
+    Arrays.sort(allToks);
+    Assert.assertArrayEquals(new String[]{"abc", "ghi"}, allToks);
+
+    store.removeToken("abc");
+    store.removeToken("wabawaba");
+
+    Assert.assertNull(store.getToken("abc"));
+    Assert.assertEquals("jkl", store.getToken("ghi"));
+    allToks = store.getAllTokenIdentifiers().toArray(new String[1]);
+    Assert.assertArrayEquals(new String[]{"ghi"}, allToks);
+  }
+
+  @Test
+  public void masterKey() throws Exception {
+    Assert.assertEquals(0, store.addMasterKey("k1"));
+    Assert.assertEquals(1, store.addMasterKey("k2"));
+
+    String[] keys = store.getMasterKeys();
+    Arrays.sort(keys);
+    Assert.assertArrayEquals(new String[]{"k1", "k2"}, keys);
+
+    store.updateMasterKey(0, "k3");
+    keys = store.getMasterKeys();
+    Arrays.sort(keys);
+    Assert.assertArrayEquals(new String[]{"k2", "k3"}, keys);
+
+    store.removeMasterKey(1);
+    keys = store.getMasterKeys();
+    Assert.assertArrayEquals(new String[]{"k3"}, keys);
+
+    thrown.expect(NoSuchObjectException.class);
+    store.updateMasterKey(72, "whatever");
+  }
+
 }


[13/52] [abbrv] hive git commit: HIVE-11568 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java
index ae39507,a877338..fe60838
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class TxnAbortedException extends TException implements org.apache.thrift.TBase<TxnAbortedException, TxnAbortedException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class TxnAbortedException extends TException implements org.apache.thrift.TBase<TxnAbortedException, TxnAbortedException._Fields>, java.io.Serializable, Cloneable, Comparable<TxnAbortedException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnAbortedException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java
index 18cbe53,8b255b9..266fbe1
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class TxnInfo implements org.apache.thrift.TBase<TxnInfo, TxnInfo._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class TxnInfo implements org.apache.thrift.TBase<TxnInfo, TxnInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TxnInfo> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnInfo");
  
    private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java
index 4f5d02d,05af505..18db1b8
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class TxnOpenException extends TException implements org.apache.thrift.TBase<TxnOpenException, TxnOpenException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class TxnOpenException extends TException implements org.apache.thrift.TBase<TxnOpenException, TxnOpenException._Fields>, java.io.Serializable, Cloneable, Comparable<TxnOpenException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnOpenException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
index 1882b57,61e7ceb..b330ce2
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Type implements org.apache.thrift.TBase<Type, Type._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Type implements org.apache.thrift.TBase<Type, Type._Fields>, java.io.Serializable, Cloneable, Comparable<Type> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Type");
  
    private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
index ab91419,e05e79d..b7623ca
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class UnknownDBException extends TException implements org.apache.thrift.TBase<UnknownDBException, UnknownDBException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class UnknownDBException extends TException implements org.apache.thrift.TBase<UnknownDBException, UnknownDBException._Fields>, java.io.Serializable, Cloneable, Comparable<UnknownDBException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnknownDBException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java
index 7e28591,c626bf6..bdd674b
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class UnknownPartitionException extends TException implements org.apache.thrift.TBase<UnknownPartitionException, UnknownPartitionException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class UnknownPartitionException extends TException implements org.apache.thrift.TBase<UnknownPartitionException, UnknownPartitionException._Fields>, java.io.Serializable, Cloneable, Comparable<UnknownPartitionException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnknownPartitionException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
index 7aa8012,2856121..768eb65
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class UnknownTableException extends TException implements org.apache.thrift.TBase<UnknownTableException, UnknownTableException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class UnknownTableException extends TException implements org.apache.thrift.TBase<UnknownTableException, UnknownTableException._Fields>, java.io.Serializable, Cloneable, Comparable<UnknownTableException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnknownTableException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java
index 54b949d,cf248e0..395c15f
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class UnlockRequest implements org.apache.thrift.TBase<UnlockRequest, UnlockRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class UnlockRequest implements org.apache.thrift.TBase<UnlockRequest, UnlockRequest._Fields>, java.io.Serializable, Cloneable, Comparable<UnlockRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnlockRequest");
  
    private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
index aa83fd7,cc8d5f5..f3cbb74
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Version implements org.apache.thrift.TBase<Version, Version._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Version implements org.apache.thrift.TBase<Version, Version._Fields>, java.io.Serializable, Cloneable, Comparable<Version> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Version");
  
    private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 1684674,ae47cb5..9c73767
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@@ -15,131 -16,997 +16,1000 @@@ use Thrift\Protocol\TBinaryProtocolAcce
  use Thrift\Exception\TApplicationException;
  
  
+ /**
+  * This interface is live.
+  */
  interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
+   /**
+    * @param string $key
+    * @return string
+    * @throws \metastore\MetaException
+    */
    public function getMetaConf($key);
+   /**
+    * @param string $key
+    * @param string $value
+    * @throws \metastore\MetaException
+    */
    public function setMetaConf($key, $value);
+   /**
+    * @param \metastore\Database $database
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\MetaException
+    */
    public function create_database(\metastore\Database $database);
+   /**
+    * @param string $name
+    * @return \metastore\Database
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function get_database($name);
+   /**
+    * @param string $name
+    * @param bool $deleteData
+    * @param bool $cascade
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
    public function drop_database($name, $deleteData, $cascade);
+   /**
+    * @param string $pattern
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function get_databases($pattern);
+   /**
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function get_all_databases();
+   /**
+    * @param string $dbname
+    * @param \metastore\Database $db
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function alter_database($dbname, \metastore\Database $db);
+   /**
+    * @param string $name
+    * @return \metastore\Type
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function get_type($name);
+   /**
+    * @param \metastore\Type $type
+    * @return bool
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\MetaException
+    */
    public function create_type(\metastore\Type $type);
+   /**
+    * @param string $type
+    * @return bool
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function drop_type($type);
+   /**
+    * @param string $name
+    * @return array
+    * @throws \metastore\MetaException
+    */
    public function get_type_all($name);
+   /**
+    * @param string $db_name
+    * @param string $table_name
+    * @return \metastore\FieldSchema[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\UnknownTableException
+    * @throws \metastore\UnknownDBException
+    */
    public function get_fields($db_name, $table_name);
+   /**
+    * @param string $db_name
+    * @param string $table_name
+    * @param \metastore\EnvironmentContext $environment_context
+    * @return \metastore\FieldSchema[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\UnknownTableException
+    * @throws \metastore\UnknownDBException
+    */
    public function get_fields_with_environment_context($db_name, $table_name, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param string $db_name
+    * @param string $table_name
+    * @return \metastore\FieldSchema[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\UnknownTableException
+    * @throws \metastore\UnknownDBException
+    */
    public function get_schema($db_name, $table_name);
+   /**
+    * @param string $db_name
+    * @param string $table_name
+    * @param \metastore\EnvironmentContext $environment_context
+    * @return \metastore\FieldSchema[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\UnknownTableException
+    * @throws \metastore\UnknownDBException
+    */
    public function get_schema_with_environment_context($db_name, $table_name, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param \metastore\Table $tbl
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function create_table(\metastore\Table $tbl);
+   /**
+    * @param \metastore\Table $tbl
+    * @param \metastore\EnvironmentContext $environment_context
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function create_table_with_environment_context(\metastore\Table $tbl, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param string $dbname
+    * @param string $name
+    * @param bool $deleteData
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function drop_table($dbname, $name, $deleteData);
+   /**
+    * @param string $dbname
+    * @param string $name
+    * @param bool $deleteData
+    * @param \metastore\EnvironmentContext $environment_context
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function drop_table_with_environment_context($dbname, $name, $deleteData, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param string $db_name
+    * @param string $pattern
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function get_tables($db_name, $pattern);
+   /**
+    * @param string $db_name
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function get_all_tables($db_name);
+   /**
+    * @param string $dbname
+    * @param string $tbl_name
+    * @return \metastore\Table
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function get_table($dbname, $tbl_name);
-   public function get_table_objects_by_name($dbname, $tbl_names);
+   /**
+    * @param string $dbname
+    * @param string[] $tbl_names
+    * @return \metastore\Table[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\UnknownDBException
+    */
+   public function get_table_objects_by_name($dbname, array $tbl_names);
+   /**
+    * @param string $dbname
+    * @param string $filter
+    * @param int $max_tables
+    * @return string[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\UnknownDBException
+    */
    public function get_table_names_by_filter($dbname, $filter, $max_tables);
+   /**
+    * @param string $dbname
+    * @param string $tbl_name
+    * @param \metastore\Table $new_tbl
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
    public function alter_table($dbname, $tbl_name, \metastore\Table $new_tbl);
+   /**
+    * @param string $dbname
+    * @param string $tbl_name
+    * @param \metastore\Table $new_tbl
+    * @param \metastore\EnvironmentContext $environment_context
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
    public function alter_table_with_environment_context($dbname, $tbl_name, \metastore\Table $new_tbl, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param string $dbname
+    * @param string $tbl_name
+    * @param \metastore\Table $new_tbl
+    * @param bool $cascade
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
    public function alter_table_with_cascade($dbname, $tbl_name, \metastore\Table $new_tbl, $cascade);
+   /**
+    * @param \metastore\Partition $new_part
+    * @return \metastore\Partition
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
    public function add_partition(\metastore\Partition $new_part);
+   /**
+    * @param \metastore\Partition $new_part
+    * @param \metastore\EnvironmentContext $environment_context
+    * @return \metastore\Partition
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
    public function add_partition_with_environment_context(\metastore\Partition $new_part, \metastore\EnvironmentContext $environment_context);
-   public function add_partitions($new_parts);
-   public function add_partitions_pspec($new_parts);
-   public function append_partition($db_name, $tbl_name, $part_vals);
+   /**
+    * @param \metastore\Partition[] $new_parts
+    * @return int
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
+   public function add_partitions(array $new_parts);
+   /**
+    * @param \metastore\PartitionSpec[] $new_parts
+    * @return int
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
+   public function add_partitions_pspec(array $new_parts);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @return \metastore\Partition
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
+   public function append_partition($db_name, $tbl_name, array $part_vals);
+   /**
+    * @param \metastore\AddPartitionsRequest $request
+    * @return \metastore\AddPartitionsResult
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
    public function add_partitions_req(\metastore\AddPartitionsRequest $request);
-   public function append_partition_with_environment_context($db_name, $tbl_name, $part_vals, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @param \metastore\EnvironmentContext $environment_context
+    * @return \metastore\Partition
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
+   public function append_partition_with_environment_context($db_name, $tbl_name, array $part_vals, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $part_name
+    * @return \metastore\Partition
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
    public function append_partition_by_name($db_name, $tbl_name, $part_name);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $part_name
+    * @param \metastore\EnvironmentContext $environment_context
+    * @return \metastore\Partition
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
    public function append_partition_by_name_with_environment_context($db_name, $tbl_name, $part_name, \metastore\EnvironmentContext $environment_context);
-   public function drop_partition($db_name, $tbl_name, $part_vals, $deleteData);
-   public function drop_partition_with_environment_context($db_name, $tbl_name, $part_vals, $deleteData, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @param bool $deleteData
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
+   public function drop_partition($db_name, $tbl_name, array $part_vals, $deleteData);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @param bool $deleteData
+    * @param \metastore\EnvironmentContext $environment_context
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
+   public function drop_partition_with_environment_context($db_name, $tbl_name, array $part_vals, $deleteData, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $part_name
+    * @param bool $deleteData
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function drop_partition_by_name($db_name, $tbl_name, $part_name, $deleteData);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $part_name
+    * @param bool $deleteData
+    * @param \metastore\EnvironmentContext $environment_context
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function drop_partition_by_name_with_environment_context($db_name, $tbl_name, $part_name, $deleteData, \metastore\EnvironmentContext $environment_context);
+   /**
+    * @param \metastore\DropPartitionsRequest $req
+    * @return \metastore\DropPartitionsResult
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function drop_partitions_req(\metastore\DropPartitionsRequest $req);
-   public function get_partition($db_name, $tbl_name, $part_vals);
-   public function exchange_partition($partitionSpecs, $source_db, $source_table_name, $dest_db, $dest_table_name);
-   public function get_partition_with_auth($db_name, $tbl_name, $part_vals, $user_name, $group_names);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @return \metastore\Partition
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
+   public function get_partition($db_name, $tbl_name, array $part_vals);
+   /**
+    * @param array $partitionSpecs
+    * @param string $source_db
+    * @param string $source_table_name
+    * @param string $dest_db
+    * @param string $dest_table_name
+    * @return \metastore\Partition
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\InvalidInputException
+    */
+   public function exchange_partition(array $partitionSpecs, $source_db, $source_table_name, $dest_db, $dest_table_name);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @param string $user_name
+    * @param string[] $group_names
+    * @return \metastore\Partition
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
+   public function get_partition_with_auth($db_name, $tbl_name, array $part_vals, $user_name, array $group_names);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $part_name
+    * @return \metastore\Partition
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function get_partition_by_name($db_name, $tbl_name, $part_name);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param int $max_parts
+    * @return \metastore\Partition[]
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function get_partitions($db_name, $tbl_name, $max_parts);
-   public function get_partitions_with_auth($db_name, $tbl_name, $max_parts, $user_name, $group_names);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param int $max_parts
+    * @param string $user_name
+    * @param string[] $group_names
+    * @return \metastore\Partition[]
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
+   public function get_partitions_with_auth($db_name, $tbl_name, $max_parts, $user_name, array $group_names);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param int $max_parts
+    * @return \metastore\PartitionSpec[]
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function get_partitions_pspec($db_name, $tbl_name, $max_parts);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param int $max_parts
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function get_partition_names($db_name, $tbl_name, $max_parts);
-   public function get_partitions_ps($db_name, $tbl_name, $part_vals, $max_parts);
-   public function get_partitions_ps_with_auth($db_name, $tbl_name, $part_vals, $max_parts, $user_name, $group_names);
-   public function get_partition_names_ps($db_name, $tbl_name, $part_vals, $max_parts);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @param int $max_parts
+    * @return \metastore\Partition[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
+   public function get_partitions_ps($db_name, $tbl_name, array $part_vals, $max_parts);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @param int $max_parts
+    * @param string $user_name
+    * @param string[] $group_names
+    * @return \metastore\Partition[]
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
+   public function get_partitions_ps_with_auth($db_name, $tbl_name, array $part_vals, $max_parts, $user_name, array $group_names);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @param int $max_parts
+    * @return string[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
+   public function get_partition_names_ps($db_name, $tbl_name, array $part_vals, $max_parts);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $filter
+    * @param int $max_parts
+    * @return \metastore\Partition[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function get_partitions_by_filter($db_name, $tbl_name, $filter, $max_parts);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $filter
+    * @param int $max_parts
+    * @return \metastore\PartitionSpec[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function get_part_specs_by_filter($db_name, $tbl_name, $filter, $max_parts);
+   /**
+    * @param \metastore\PartitionsByExprRequest $req
+    * @return \metastore\PartitionsByExprResult
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function get_partitions_by_expr(\metastore\PartitionsByExprRequest $req);
-   public function get_partitions_by_names($db_name, $tbl_name, $names);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $names
+    * @return \metastore\Partition[]
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
+   public function get_partitions_by_names($db_name, $tbl_name, array $names);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param \metastore\Partition $new_part
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
    public function alter_partition($db_name, $tbl_name, \metastore\Partition $new_part);
-   public function alter_partitions($db_name, $tbl_name, $new_parts);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param \metastore\Partition[] $new_parts
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
+   public function alter_partitions($db_name, $tbl_name, array $new_parts);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param \metastore\Partition $new_part
+    * @param \metastore\EnvironmentContext $environment_context
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
    public function alter_partition_with_environment_context($db_name, $tbl_name, \metastore\Partition $new_part, \metastore\EnvironmentContext $environment_context);
-   public function rename_partition($db_name, $tbl_name, $part_vals, \metastore\Partition $new_part);
-   public function partition_name_has_valid_characters($part_vals, $throw_exception);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string[] $part_vals
+    * @param \metastore\Partition $new_part
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
+   public function rename_partition($db_name, $tbl_name, array $part_vals, \metastore\Partition $new_part);
+   /**
+    * @param string[] $part_vals
+    * @param bool $throw_exception
+    * @return bool
+    * @throws \metastore\MetaException
+    */
+   public function partition_name_has_valid_characters(array $part_vals, $throw_exception);
+   /**
+    * @param string $name
+    * @param string $defaultValue
+    * @return string
+    * @throws \metastore\ConfigValSecurityException
+    */
    public function get_config_value($name, $defaultValue);
+   /**
+    * @param string $part_name
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function partition_name_to_vals($part_name);
+   /**
+    * @param string $part_name
+    * @return array
+    * @throws \metastore\MetaException
+    */
    public function partition_name_to_spec($part_name);
-   public function markPartitionForEvent($db_name, $tbl_name, $part_vals, $eventType);
-   public function isPartitionMarkedForEvent($db_name, $tbl_name, $part_vals, $eventType);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param array $part_vals
+    * @param int $eventType
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\UnknownDBException
+    * @throws \metastore\UnknownTableException
+    * @throws \metastore\UnknownPartitionException
+    * @throws \metastore\InvalidPartitionException
+    */
+   public function markPartitionForEvent($db_name, $tbl_name, array $part_vals, $eventType);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param array $part_vals
+    * @param int $eventType
+    * @return bool
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\UnknownDBException
+    * @throws \metastore\UnknownTableException
+    * @throws \metastore\UnknownPartitionException
+    * @throws \metastore\InvalidPartitionException
+    */
+   public function isPartitionMarkedForEvent($db_name, $tbl_name, array $part_vals, $eventType);
+   /**
+    * @param \metastore\Index $new_index
+    * @param \metastore\Table $index_table
+    * @return \metastore\Index
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\MetaException
+    */
    public function add_index(\metastore\Index $new_index, \metastore\Table $index_table);
+   /**
+    * @param string $dbname
+    * @param string $base_tbl_name
+    * @param string $idx_name
+    * @param \metastore\Index $new_idx
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
    public function alter_index($dbname, $base_tbl_name, $idx_name, \metastore\Index $new_idx);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $index_name
+    * @param bool $deleteData
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function drop_index_by_name($db_name, $tbl_name, $index_name, $deleteData);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $index_name
+    * @return \metastore\Index
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function get_index_by_name($db_name, $tbl_name, $index_name);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param int $max_indexes
+    * @return \metastore\Index[]
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function get_indexes($db_name, $tbl_name, $max_indexes);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param int $max_indexes
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function get_index_names($db_name, $tbl_name, $max_indexes);
+   /**
+    * @param \metastore\ColumnStatistics $stats_obj
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidInputException
+    */
    public function update_table_column_statistics(\metastore\ColumnStatistics $stats_obj);
+   /**
+    * @param \metastore\ColumnStatistics $stats_obj
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidInputException
+    */
    public function update_partition_column_statistics(\metastore\ColumnStatistics $stats_obj);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $col_name
+    * @return \metastore\ColumnStatistics
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidInputException
+    * @throws \metastore\InvalidObjectException
+    */
    public function get_table_column_statistics($db_name, $tbl_name, $col_name);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $part_name
+    * @param string $col_name
+    * @return \metastore\ColumnStatistics
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidInputException
+    * @throws \metastore\InvalidObjectException
+    */
    public function get_partition_column_statistics($db_name, $tbl_name, $part_name, $col_name);
+   /**
+    * @param \metastore\TableStatsRequest $request
+    * @return \metastore\TableStatsResult
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function get_table_statistics_req(\metastore\TableStatsRequest $request);
+   /**
+    * @param \metastore\PartitionsStatsRequest $request
+    * @return \metastore\PartitionsStatsResult
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function get_partitions_statistics_req(\metastore\PartitionsStatsRequest $request);
+   /**
+    * @param \metastore\PartitionsStatsRequest $request
+    * @return \metastore\AggrStats
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function get_aggr_stats_for(\metastore\PartitionsStatsRequest $request);
+   /**
+    * @param \metastore\SetPartitionsStatsRequest $request
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidInputException
+    */
    public function set_aggr_stats_for(\metastore\SetPartitionsStatsRequest $request);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $part_name
+    * @param string $col_name
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\InvalidInputException
+    */
    public function delete_partition_column_statistics($db_name, $tbl_name, $part_name, $col_name);
+   /**
+    * @param string $db_name
+    * @param string $tbl_name
+    * @param string $col_name
+    * @return bool
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\InvalidInputException
+    */
    public function delete_table_column_statistics($db_name, $tbl_name, $col_name);
+   /**
+    * @param \metastore\Function $func
+    * @throws \metastore\AlreadyExistsException
+    * @throws \metastore\InvalidObjectException
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function create_function(\metastore\Function $func);
+   /**
+    * @param string $dbName
+    * @param string $funcName
+    * @throws \metastore\NoSuchObjectException
+    * @throws \metastore\MetaException
+    */
    public function drop_function($dbName, $funcName);
+   /**
+    * @param string $dbName
+    * @param string $funcName
+    * @param \metastore\Function $newFunc
+    * @throws \metastore\InvalidOperationException
+    * @throws \metastore\MetaException
+    */
    public function alter_function($dbName, $funcName, \metastore\Function $newFunc);
+   /**
+    * @param string $dbName
+    * @param string $pattern
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function get_functions($dbName, $pattern);
+   /**
+    * @param string $dbName
+    * @param string $funcName
+    * @return \metastore\Function
+    * @throws \metastore\MetaException
+    * @throws \metastore\NoSuchObjectException
+    */
    public function get_function($dbName, $funcName);
+   /**
+    * @return \metastore\GetAllFunctionsResponse
+    * @throws \metastore\MetaException
+    */
+   public function get_all_functions();
+   /**
+    * @param \metastore\Role $role
+    * @return bool
+    * @throws \metastore\MetaException
+    */
    public function create_role(\metastore\Role $role);
+   /**
+    * @param string $role_name
+    * @return bool
+    * @throws \metastore\MetaException
+    */
    public function drop_role($role_name);
+   /**
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
    public function get_role_names();
+   /**
+    * @param string $role_name
+    * @param string $principal_name
+    * @param int $principal_type
+    * @param string $grantor
+    * @param int $grantorType
+    * @param bool $grant_option
+    * @return bool
+    * @throws \metastore\MetaException
+    */
    public function grant_role($role_name, $principal_name, $principal_type, $grantor, $grantorType, $grant_option);
+   /**
+    * @param string $role_name
+    * @param string $principal_name
+    * @param int $principal_type
+    * @return bool
+    * @throws \metastore\MetaException
+    */
    public function revoke_role($role_name, $principal_name, $principal_type);
+   /**
+    * @param string $principal_name
+    * @param int $principal_type
+    * @return \metastore\Role[]
+    * @throws \metastore\MetaException
+    */
    public function list_roles($principal_name, $principal_type);
+   /**
+    * @param \metastore\GrantRevokeRoleRequest $request
+    * @return \metastore\GrantRevokeRoleResponse
+    * @throws \metastore\MetaException
+    */
    public function grant_revoke_role(\metastore\GrantRevokeRoleRequest $request);
+   /**
+    * @param \metastore\GetPrincipalsInRoleRequest $request
+    * @return \metastore\GetPrincipalsInRoleResponse
+    * @throws \metastore\MetaException
+    */
    public function get_principals_in_role(\metastore\GetPrincipalsInRoleRequest $request);
+   /**
+    * @param \metastore\GetRoleGrantsForPrincipalRequest $request
+    * @return \metastore\GetRoleGrantsForPrincipalResponse
+    * @throws \metastore\MetaException
+    */
    public function get_role_grants_for_principal(\metastore\GetRoleGrantsForPrincipalRequest $request);
-   public function get_privilege_set(\metastore\HiveObjectRef $hiveObject, $user_name, $group_names);
+   /**
+    * @param \metastore\HiveObjectRef $hiveObject
+    * @param string $user_name
+    * @param string[] $group_names
+    * @return \metastore\PrincipalPrivilegeSet
+    * @throws \metastore\MetaException
+    */
+   public function get_privilege_set(\metastore\HiveObjectRef $hiveObject, $user_name, array $group_names);
+   /**
+    * @param string $principal_name
+    * @param int $principal_type
+    * @param \metastore\HiveObjectRef $hiveObject
+    * @return \metastore\HiveObjectPrivilege[]
+    * @throws \metastore\MetaException
+    */
    public function list_privileges($principal_name, $principal_type, \metastore\HiveObjectRef $hiveObject);
+   /**
+    * @param \metastore\PrivilegeBag $privileges
+    * @return bool
+    * @throws \metastore\MetaException
+    */
    public function grant_privileges(\metastore\PrivilegeBag $privileges);
+   /**
+    * @param \metastore\PrivilegeBag $privileges
+    * @return bool
+    * @throws \metastore\MetaException
+    */
    public function revoke_privileges(\metastore\PrivilegeBag $privileges);
+   /**
+    * @param \metastore\GrantRevokePrivilegeRequest $request
+    * @return \metastore\GrantRevokePrivilegeResponse
+    * @throws \metastore\MetaException
+    */
    public function grant_revoke_privileges(\metastore\GrantRevokePrivilegeRequest $request);
-   public function set_ugi($user_name, $group_names);
+   /**
+    * @param string $user_name
+    * @param string[] $group_names
+    * @return string[]
+    * @throws \metastore\MetaException
+    */
+   public function set_ugi($user_name, array $group_names);
+   /**
+    * @param string $token_owner
+    * @param string $renewer_kerberos_principal_name
+    * @return string
+    * @throws \metastore\MetaException
+    */
    public function get_delegation_token($token_owner, $renewer_kerberos_principal_name);
+   /**
+    * @param string $token_str_form
+    * @return int
+    * @throws \metastore\MetaException
+    */
    public function renew_delegation_token($token_str_form);
+   /**
+    * @param string $token_str_form
+    * @throws \metastore\MetaException
+    */
    public function cancel_delegation_token($token_str_form);
+   /**
+    * @return \metastore\GetOpenTxnsResponse
+    */
    public function get_open_txns();
+   /**
+    * @return \metastore\GetOpenTxnsInfoResponse
+    */
    public function get_open_txns_info();
+   /**
+    * @param \metastore\OpenTxnRequest $rqst
+    * @return \metastore\OpenTxnsResponse
+    */
    public function open_txns(\metastore\OpenTxnRequest $rqst);
+   /**
+    * @param \metastore\AbortTxnRequest $rqst
+    * @throws \metastore\NoSuchTxnException
+    */
    public function abort_txn(\metastore\AbortTxnRequest $rqst);
+   /**
+    * @param \metastore\CommitTxnRequest $rqst
+    * @throws \metastore\NoSuchTxnException
+    * @throws \metastore\TxnAbortedException
+    */
    public function commit_txn(\metastore\CommitTxnRequest $rqst);
+   /**
+    * @param \metastore\LockRequest $rqst
+    * @return \metastore\LockResponse
+    * @throws \metastore\NoSuchTxnException
+    * @throws \metastore\TxnAbortedException
+    */
    public function lock(\metastore\LockRequest $rqst);
+   /**
+    * @param \metastore\CheckLockRequest $rqst
+    * @return \metastore\LockResponse
+    * @throws \metastore\NoSuchTxnException
+    * @throws \metastore\TxnAbortedException
+    * @throws \metastore\NoSuchLockException
+    */
    public function check_lock(\metastore\CheckLockRequest $rqst);
+   /**
+    * @param \metastore\UnlockRequest $rqst
+    * @throws \metastore\NoSuchLockException
+    * @throws \metastore\TxnOpenException
+    */
    public function unlock(\metastore\UnlockRequest $rqst);
+   /**
+    * @param \metastore\ShowLocksRequest $rqst
+    * @return \metastore\ShowLocksResponse
+    */
    public function show_locks(\metastore\ShowLocksRequest $rqst);
+   /**
+    * @param \metastore\HeartbeatRequest $ids
+    * @throws \metastore\NoSuchLockException
+    * @throws \metastore\NoSuchTxnException
+    * @throws \metastore\TxnAbortedException
+    */
    public function heartbeat(\metastore\HeartbeatRequest $ids);
+   /**
+    * @param \metastore\HeartbeatTxnRangeRequest $txns
+    * @return \metastore\HeartbeatTxnRangeResponse
+    */
    public function heartbeat_txn_range(\metastore\HeartbeatTxnRangeRequest $txns);
+   /**
+    * @param \metastore\CompactionRequest $rqst
+    */
    public function compact(\metastore\CompactionRequest $rqst);
+   /**
+    * @param \metastore\ShowCompactRequest $rqst
+    * @return \metastore\ShowCompactResponse
+    */
    public function show_compact(\metastore\ShowCompactRequest $rqst);
+   /**
+    * @param \metastore\AddDynamicPartitions $rqst
+    * @throws \metastore\NoSuchTxnException
+    * @throws \metastore\TxnAbortedException
+    */
    public function add_dynamic_partitions(\metastore\AddDynamicPartitions $rqst);
+   /**
+    * @param \metastore\NotificationEventRequest $rqst
+    * @return \metastore\NotificationEventResponse
+    */
    public function get_next_notification(\metastore\NotificationEventRequest $rqst);
+   /**
+    * @return \metastore\CurrentNotificationEventId
+    */
    public function get_current_notificationEventId();
+   /**
+    * @param \metastore\FireEventRequest $rqst
+    * @return \metastore\FireEventResponse
+    */
    public function fire_listener_event(\metastore\FireEventRequest $rqst);
++  /**
++   */
 +  public function flushCache();
  }
  
  class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@@ -7221,53 -8141,6 +8144,53 @@@
      throw new \Exception("fire_listener_event failed: unknown result");
    }
  
 +  public function flushCache()
 +  {
 +    $this->send_flushCache();
 +    $this->recv_flushCache();
 +  }
 +
 +  public function send_flushCache()
 +  {
 +    $args = new \metastore\ThriftHiveMetastore_flushCache_args();
-     $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary');
++    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
 +    if ($bin_accel)
 +    {
 +      thrift_protocol_write_binary($this->output_, 'flushCache', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
 +    }
 +    else
 +    {
 +      $this->output_->writeMessageBegin('flushCache', TMessageType::CALL, $this->seqid_);
 +      $args->write($this->output_);
 +      $this->output_->writeMessageEnd();
 +      $this->output_->getTransport()->flush();
 +    }
 +  }
 +
 +  public function recv_flushCache()
 +  {
-     $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary');
++    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
 +    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_flushCache_result', $this->input_->isStrictRead());
 +    else
 +    {
 +      $rseqid = 0;
 +      $fname = null;
 +      $mtype = 0;
 +
 +      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
 +      if ($mtype == TMessageType::EXCEPTION) {
 +        $x = new TApplicationException();
 +        $x->read($this->input_);
 +        $this->input_->readMessageEnd();
 +        throw $x;
 +      }
 +      $result = new \metastore\ThriftHiveMetastore_flushCache_result();
 +      $result->read($this->input_);
 +      $this->input_->readMessageEnd();
 +    }
 +    return;
 +  }
 +
  }
  
  // HELPER FUNCTIONS AND STRUCTURES

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 531a475,dc348ef..6bd2728
mode 100644,100755..100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@@ -15,139 -16,152 +16,153 @@@ from thrift.transport import TSSLSocke
  from thrift.transport import THttpClient
  from thrift.protocol import TBinaryProtocol
  
- import ThriftHiveMetastore
- from ttypes import *
+ from hive_metastore import ThriftHiveMetastore
+ from hive_metastore.ttypes import *
  
  if len(sys.argv) <= 1 or sys.argv[1] == '--help':
-   print ''
-   print 'Usage: ' + sys.argv[0] + ' [-h host[:port]] [-u url] [-f[ramed]] function [arg1 [arg2...]]'
-   print ''
-   print 'Functions:'
-   print '  string getMetaConf(string key)'
-   print '  void setMetaConf(string key, string value)'
-   print '  void create_database(Database database)'
-   print '  Database get_database(string name)'
-   print '  void drop_database(string name, bool deleteData, bool cascade)'
-   print '   get_databases(string pattern)'
-   print '   get_all_databases()'
-   print '  void alter_database(string dbname, Database db)'
-   print '  Type get_type(string name)'
-   print '  bool create_type(Type type)'
-   print '  bool drop_type(string type)'
-   print '   get_type_all(string name)'
-   print '   get_fields(string db_name, string table_name)'
-   print '   get_fields_with_environment_context(string db_name, string table_name, EnvironmentContext environment_context)'
-   print '   get_schema(string db_name, string table_name)'
-   print '   get_schema_with_environment_context(string db_name, string table_name, EnvironmentContext environment_context)'
-   print '  void create_table(Table tbl)'
-   print '  void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context)'
-   print '  void drop_table(string dbname, string name, bool deleteData)'
-   print '  void drop_table_with_environment_context(string dbname, string name, bool deleteData, EnvironmentContext environment_context)'
-   print '   get_tables(string db_name, string pattern)'
-   print '   get_all_tables(string db_name)'
-   print '  Table get_table(string dbname, string tbl_name)'
-   print '   get_table_objects_by_name(string dbname,  tbl_names)'
-   print '   get_table_names_by_filter(string dbname, string filter, i16 max_tables)'
-   print '  void alter_table(string dbname, string tbl_name, Table new_tbl)'
-   print '  void alter_table_with_environment_context(string dbname, string tbl_name, Table new_tbl, EnvironmentContext environment_context)'
-   print '  void alter_table_with_cascade(string dbname, string tbl_name, Table new_tbl, bool cascade)'
-   print '  Partition add_partition(Partition new_part)'
-   print '  Partition add_partition_with_environment_context(Partition new_part, EnvironmentContext environment_context)'
-   print '  i32 add_partitions( new_parts)'
-   print '  i32 add_partitions_pspec( new_parts)'
-   print '  Partition append_partition(string db_name, string tbl_name,  part_vals)'
-   print '  AddPartitionsResult add_partitions_req(AddPartitionsRequest request)'
-   print '  Partition append_partition_with_environment_context(string db_name, string tbl_name,  part_vals, EnvironmentContext environment_context)'
-   print '  Partition append_partition_by_name(string db_name, string tbl_name, string part_name)'
-   print '  Partition append_partition_by_name_with_environment_context(string db_name, string tbl_name, string part_name, EnvironmentContext environment_context)'
-   print '  bool drop_partition(string db_name, string tbl_name,  part_vals, bool deleteData)'
-   print '  bool drop_partition_with_environment_context(string db_name, string tbl_name,  part_vals, bool deleteData, EnvironmentContext environment_context)'
-   print '  bool drop_partition_by_name(string db_name, string tbl_name, string part_name, bool deleteData)'
-   print '  bool drop_partition_by_name_with_environment_context(string db_name, string tbl_name, string part_name, bool deleteData, EnvironmentContext environment_context)'
-   print '  DropPartitionsResult drop_partitions_req(DropPartitionsRequest req)'
-   print '  Partition get_partition(string db_name, string tbl_name,  part_vals)'
-   print '  Partition exchange_partition( partitionSpecs, string source_db, string source_table_name, string dest_db, string dest_table_name)'
-   print '  Partition get_partition_with_auth(string db_name, string tbl_name,  part_vals, string user_name,  group_names)'
-   print '  Partition get_partition_by_name(string db_name, string tbl_name, string part_name)'
-   print '   get_partitions(string db_name, string tbl_name, i16 max_parts)'
-   print '   get_partitions_with_auth(string db_name, string tbl_name, i16 max_parts, string user_name,  group_names)'
-   print '   get_partitions_pspec(string db_name, string tbl_name, i32 max_parts)'
-   print '   get_partition_names(string db_name, string tbl_name, i16 max_parts)'
-   print '   get_partitions_ps(string db_name, string tbl_name,  part_vals, i16 max_parts)'
-   print '   get_partitions_ps_with_auth(string db_name, string tbl_name,  part_vals, i16 max_parts, string user_name,  group_names)'
-   print '   get_partition_names_ps(string db_name, string tbl_name,  part_vals, i16 max_parts)'
-   print '   get_partitions_by_filter(string db_name, string tbl_name, string filter, i16 max_parts)'
-   print '   get_part_specs_by_filter(string db_name, string tbl_name, string filter, i32 max_parts)'
-   print '  PartitionsByExprResult get_partitions_by_expr(PartitionsByExprRequest req)'
-   print '   get_partitions_by_names(string db_name, string tbl_name,  names)'
-   print '  void alter_partition(string db_name, string tbl_name, Partition new_part)'
-   print '  void alter_partitions(string db_name, string tbl_name,  new_parts)'
-   print '  void alter_partition_with_environment_context(string db_name, string tbl_name, Partition new_part, EnvironmentContext environment_context)'
-   print '  void rename_partition(string db_name, string tbl_name,  part_vals, Partition new_part)'
-   print '  bool partition_name_has_valid_characters( part_vals, bool throw_exception)'
-   print '  string get_config_value(string name, string defaultValue)'
-   print '   partition_name_to_vals(string part_name)'
-   print '   partition_name_to_spec(string part_name)'
-   print '  void markPartitionForEvent(string db_name, string tbl_name,  part_vals, PartitionEventType eventType)'
-   print '  bool isPartitionMarkedForEvent(string db_name, string tbl_name,  part_vals, PartitionEventType eventType)'
-   print '  Index add_index(Index new_index, Table index_table)'
-   print '  void alter_index(string dbname, string base_tbl_name, string idx_name, Index new_idx)'
-   print '  bool drop_index_by_name(string db_name, string tbl_name, string index_name, bool deleteData)'
-   print '  Index get_index_by_name(string db_name, string tbl_name, string index_name)'
-   print '   get_indexes(string db_name, string tbl_name, i16 max_indexes)'
-   print '   get_index_names(string db_name, string tbl_name, i16 max_indexes)'
-   print '  bool update_table_column_statistics(ColumnStatistics stats_obj)'
-   print '  bool update_partition_column_statistics(ColumnStatistics stats_obj)'
-   print '  ColumnStatistics get_table_column_statistics(string db_name, string tbl_name, string col_name)'
-   print '  ColumnStatistics get_partition_column_statistics(string db_name, string tbl_name, string part_name, string col_name)'
-   print '  TableStatsResult get_table_statistics_req(TableStatsRequest request)'
-   print '  PartitionsStatsResult get_partitions_statistics_req(PartitionsStatsRequest request)'
-   print '  AggrStats get_aggr_stats_for(PartitionsStatsRequest request)'
-   print '  bool set_aggr_stats_for(SetPartitionsStatsRequest request)'
-   print '  bool delete_partition_column_statistics(string db_name, string tbl_name, string part_name, string col_name)'
-   print '  bool delete_table_column_statistics(string db_name, string tbl_name, string col_name)'
-   print '  void create_function(Function func)'
-   print '  void drop_function(string dbName, string funcName)'
-   print '  void alter_function(string dbName, string funcName, Function newFunc)'
-   print '   get_functions(string dbName, string pattern)'
-   print '  Function get_function(string dbName, string funcName)'
-   print '  bool create_role(Role role)'
-   print '  bool drop_role(string role_name)'
-   print '   get_role_names()'
-   print '  bool grant_role(string role_name, string principal_name, PrincipalType principal_type, string grantor, PrincipalType grantorType, bool grant_option)'
-   print '  bool revoke_role(string role_name, string principal_name, PrincipalType principal_type)'
-   print '   list_roles(string principal_name, PrincipalType principal_type)'
-   print '  GrantRevokeRoleResponse grant_revoke_role(GrantRevokeRoleRequest request)'
-   print '  GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest request)'
-   print '  GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(GetRoleGrantsForPrincipalRequest request)'
-   print '  PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject, string user_name,  group_names)'
-   print '   list_privileges(string principal_name, PrincipalType principal_type, HiveObjectRef hiveObject)'
-   print '  bool grant_privileges(PrivilegeBag privileges)'
-   print '  bool revoke_privileges(PrivilegeBag privileges)'
-   print '  GrantRevokePrivilegeResponse grant_revoke_privileges(GrantRevokePrivilegeRequest request)'
-   print '   set_ugi(string user_name,  group_names)'
-   print '  string get_delegation_token(string token_owner, string renewer_kerberos_principal_name)'
-   print '  i64 renew_delegation_token(string token_str_form)'
-   print '  void cancel_delegation_token(string token_str_form)'
-   print '  GetOpenTxnsResponse get_open_txns()'
-   print '  GetOpenTxnsInfoResponse get_open_txns_info()'
-   print '  OpenTxnsResponse open_txns(OpenTxnRequest rqst)'
-   print '  void abort_txn(AbortTxnRequest rqst)'
-   print '  void commit_txn(CommitTxnRequest rqst)'
-   print '  LockResponse lock(LockRequest rqst)'
-   print '  LockResponse check_lock(CheckLockRequest rqst)'
-   print '  void unlock(UnlockRequest rqst)'
-   print '  ShowLocksResponse show_locks(ShowLocksRequest rqst)'
-   print '  void heartbeat(HeartbeatRequest ids)'
-   print '  HeartbeatTxnRangeResponse heartbeat_txn_range(HeartbeatTxnRangeRequest txns)'
-   print '  void compact(CompactionRequest rqst)'
-   print '  ShowCompactResponse show_compact(ShowCompactRequest rqst)'
-   print '  void add_dynamic_partitions(AddDynamicPartitions rqst)'
-   print '  NotificationEventResponse get_next_notification(NotificationEventRequest rqst)'
-   print '  CurrentNotificationEventId get_current_notificationEventId()'
-   print '  FireEventResponse fire_listener_event(FireEventRequest rqst)'
-   print '  void flushCache()'
-   print ''
+   print('')
+   print('Usage: ' + sys.argv[0] + ' [-h host[:port]] [-u url] [-f[ramed]] [-s[sl]] function [arg1 [arg2...]]')
+   print('')
+   print('Functions:')
+   print('  string getMetaConf(string key)')
+   print('  void setMetaConf(string key, string value)')
+   print('  void create_database(Database database)')
+   print('  Database get_database(string name)')
+   print('  void drop_database(string name, bool deleteData, bool cascade)')
+   print('   get_databases(string pattern)')
+   print('   get_all_databases()')
+   print('  void alter_database(string dbname, Database db)')
+   print('  Type get_type(string name)')
+   print('  bool create_type(Type type)')
+   print('  bool drop_type(string type)')
+   print('   get_type_all(string name)')
+   print('   get_fields(string db_name, string table_name)')
+   print('   get_fields_with_environment_context(string db_name, string table_name, EnvironmentContext environment_context)')
+   print('   get_schema(string db_name, string table_name)')
+   print('   get_schema_with_environment_context(string db_name, string table_name, EnvironmentContext environment_context)')
+   print('  void create_table(Table tbl)')
+   print('  void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context)')
+   print('  void drop_table(string dbname, string name, bool deleteData)')
+   print('  void drop_table_with_environment_context(string dbname, string name, bool deleteData, EnvironmentContext environment_context)')
+   print('   get_tables(string db_name, string pattern)')
+   print('   get_all_tables(string db_name)')
+   print('  Table get_table(string dbname, string tbl_name)')
+   print('   get_table_objects_by_name(string dbname,  tbl_names)')
+   print('   get_table_names_by_filter(string dbname, string filter, i16 max_tables)')
+   print('  void alter_table(string dbname, string tbl_name, Table new_tbl)')
+   print('  void alter_table_with_environment_context(string dbname, string tbl_name, Table new_tbl, EnvironmentContext environment_context)')
+   print('  void alter_table_with_cascade(string dbname, string tbl_name, Table new_tbl, bool cascade)')
+   print('  Partition add_partition(Partition new_part)')
+   print('  Partition add_partition_with_environment_context(Partition new_part, EnvironmentContext environment_context)')
+   print('  i32 add_partitions( new_parts)')
+   print('  i32 add_partitions_pspec( new_parts)')
+   print('  Partition append_partition(string db_name, string tbl_name,  part_vals)')
+   print('  AddPartitionsResult add_partitions_req(AddPartitionsRequest request)')
+   print('  Partition append_partition_with_environment_context(string db_name, string tbl_name,  part_vals, EnvironmentContext environment_context)')
+   print('  Partition append_partition_by_name(string db_name, string tbl_name, string part_name)')
+   print('  Partition append_partition_by_name_with_environment_context(string db_name, string tbl_name, string part_name, EnvironmentContext environment_context)')
+   print('  bool drop_partition(string db_name, string tbl_name,  part_vals, bool deleteData)')
+   print('  bool drop_partition_with_environment_context(string db_name, string tbl_name,  part_vals, bool deleteData, EnvironmentContext environment_context)')
+   print('  bool drop_partition_by_name(string db_name, string tbl_name, string part_name, bool deleteData)')
+   print('  bool drop_partition_by_name_with_environment_context(string db_name, string tbl_name, string part_name, bool deleteData, EnvironmentContext environment_context)')
+   print('  DropPartitionsResult drop_partitions_req(DropPartitionsRequest req)')
+   print('  Partition get_partition(string db_name, string tbl_name,  part_vals)')
+   print('  Partition exchange_partition( partitionSpecs, string source_db, string source_table_name, string dest_db, string dest_table_name)')
+   print('  Partition get_partition_with_auth(string db_name, string tbl_name,  part_vals, string user_name,  group_names)')
+   print('  Partition get_partition_by_name(string db_name, string tbl_name, string part_name)')
+   print('   get_partitions(string db_name, string tbl_name, i16 max_parts)')
+   print('   get_partitions_with_auth(string db_name, string tbl_name, i16 max_parts, string user_name,  group_names)')
+   print('   get_partitions_pspec(string db_name, string tbl_name, i32 max_parts)')
+   print('   get_partition_names(string db_name, string tbl_name, i16 max_parts)')
+   print('   get_partitions_ps(string db_name, string tbl_name,  part_vals, i16 max_parts)')
+   print('   get_partitions_ps_with_auth(string db_name, string tbl_name,  part_vals, i16 max_parts, string user_name,  group_names)')
+   print('   get_partition_names_ps(string db_name, string tbl_name,  part_vals, i16 max_parts)')
+   print('   get_partitions_by_filter(string db_name, string tbl_name, string filter, i16 max_parts)')
+   print('   get_part_specs_by_filter(string db_name, string tbl_name, string filter, i32 max_parts)')
+   print('  PartitionsByExprResult get_partitions_by_expr(PartitionsByExprRequest req)')
+   print('   get_partitions_by_names(string db_name, string tbl_name,  names)')
+   print('  void alter_partition(string db_name, string tbl_name, Partition new_part)')
+   print('  void alter_partitions(string db_name, string tbl_name,  new_parts)')
+   print('  void alter_partition_with_environment_context(string db_name, string tbl_name, Partition new_part, EnvironmentContext environment_context)')
+   print('  void rename_partition(string db_name, string tbl_name,  part_vals, Partition new_part)')
+   print('  bool partition_name_has_valid_characters( part_vals, bool throw_exception)')
+   print('  string get_config_value(string name, string defaultValue)')
+   print('   partition_name_to_vals(string part_name)')
+   print('   partition_name_to_spec(string part_name)')
+   print('  void markPartitionForEvent(string db_name, string tbl_name,  part_vals, PartitionEventType eventType)')
+   print('  bool isPartitionMarkedForEvent(string db_name, string tbl_name,  part_vals, PartitionEventType eventType)')
+   print('  Index add_index(Index new_index, Table index_table)')
+   print('  void alter_index(string dbname, string base_tbl_name, string idx_name, Index new_idx)')
+   print('  bool drop_index_by_name(string db_name, string tbl_name, string index_name, bool deleteData)')
+   print('  Index get_index_by_name(string db_name, string tbl_name, string index_name)')
+   print('   get_indexes(string db_name, string tbl_name, i16 max_indexes)')
+   print('   get_index_names(string db_name, string tbl_name, i16 max_indexes)')
+   print('  bool update_table_column_statistics(ColumnStatistics stats_obj)')
+   print('  bool update_partition_column_statistics(ColumnStatistics stats_obj)')
+   print('  ColumnStatistics get_table_column_statistics(string db_name, string tbl_name, string col_name)')
+   print('  ColumnStatistics get_partition_column_statistics(string db_name, string tbl_name, string part_name, string col_name)')
+   print('  TableStatsResult get_table_statistics_req(TableStatsRequest request)')
+   print('  PartitionsStatsResult get_partitions_statistics_req(PartitionsStatsRequest request)')
+   print('  AggrStats get_aggr_stats_for(PartitionsStatsRequest request)')
+   print('  bool set_aggr_stats_for(SetPartitionsStatsRequest request)')
+   print('  bool delete_partition_column_statistics(string db_name, string tbl_name, string part_name, string col_name)')
+   print('  bool delete_table_column_statistics(string db_name, string tbl_name, string col_name)')
+   print('  void create_function(Function func)')
+   print('  void drop_function(string dbName, string funcName)')
+   print('  void alter_function(string dbName, string funcName, Function newFunc)')
+   print('   get_functions(string dbName, string pattern)')
+   print('  Function get_function(string dbName, string funcName)')
+   print('  GetAllFunctionsResponse get_all_functions()')
+   print('  bool create_role(Role role)')
+   print('  bool drop_role(string role_name)')
+   print('   get_role_names()')
+   print('  bool grant_role(string role_name, string principal_name, PrincipalType principal_type, string grantor, PrincipalType grantorType, bool grant_option)')
+   print('  bool revoke_role(string role_name, string principal_name, PrincipalType principal_type)')
+   print('   list_roles(string principal_name, PrincipalType principal_type)')
+   print('  GrantRevokeRoleResponse grant_revoke_role(GrantRevokeRoleRequest request)')
+   print('  GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest request)')
+   print('  GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(GetRoleGrantsForPrincipalRequest request)')
+   print('  PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject, string user_name,  group_names)')
+   print('   list_privileges(string principal_name, PrincipalType principal_type, HiveObjectRef hiveObject)')
+   print('  bool grant_privileges(PrivilegeBag privileges)')
+   print('  bool revoke_privileges(PrivilegeBag privileges)')
+   print('  GrantRevokePrivilegeResponse grant_revoke_privileges(GrantRevokePrivilegeRequest request)')
+   print('   set_ugi(string user_name,  group_names)')
+   print('  string get_delegation_token(string token_owner, string renewer_kerberos_principal_name)')
+   print('  i64 renew_delegation_token(string token_str_form)')
+   print('  void cancel_delegation_token(string token_str_form)')
+   print('  GetOpenTxnsResponse get_open_txns()')
+   print('  GetOpenTxnsInfoResponse get_open_txns_info()')
+   print('  OpenTxnsResponse open_txns(OpenTxnRequest rqst)')
+   print('  void abort_txn(AbortTxnRequest rqst)')
+   print('  void commit_txn(CommitTxnRequest rqst)')
+   print('  LockResponse lock(LockRequest rqst)')
+   print('  LockResponse check_lock(CheckLockRequest rqst)')
+   print('  void unlock(UnlockRequest rqst)')
+   print('  ShowLocksResponse show_locks(ShowLocksRequest rqst)')
+   print('  void heartbeat(HeartbeatRequest ids)')
+   print('  HeartbeatTxnRangeResponse heartbeat_txn_range(HeartbeatTxnRangeRequest txns)')
+   print('  void compact(CompactionRequest rqst)')
+   print('  ShowCompactResponse show_compact(ShowCompactRequest rqst)')
+   print('  void add_dynamic_partitions(AddDynamicPartitions rqst)')
+   print('  NotificationEventResponse get_next_notification(NotificationEventRequest rqst)')
+   print('  CurrentNotificationEventId get_current_notificationEventId()')
+   print('  FireEventResponse fire_listener_event(FireEventRequest rqst)')
++  print('  void flushCache()')
+   print('  string getName()')
+   print('  string getVersion()')
+   print('  fb_status getStatus()')
+   print('  string getStatusDetails()')
+   print('   getCounters()')
+   print('  i64 getCounter(string key)')
+   print('  void setOption(string key, string value)')
+   print('  string getOption(string key)')
+   print('   getOptions()')
++  print('  string getCpuProfile(i32 profileDurationInSec)')
+   print('  i64 aliveSince()')
 -  print('  reflection_limited.Service getLimitedReflection()')
+   print('  void reinitialize()')
+   print('  void shutdown()')
+   print('')
    sys.exit(0)
  
  pp = pprint.PrettyPrinter(indent = 2)
@@@ -936,14 -961,86 +962,92 @@@ elif cmd == 'fire_listener_event'
      sys.exit(1)
    pp.pprint(client.fire_listener_event(eval(args[0]),))
  
 +elif cmd == 'flushCache':
 +  if len(args) != 0:
-     print 'flushCache requires 0 args'
++    print('flushCache requires 0 args')
 +    sys.exit(1)
 +  pp.pprint(client.flushCache())
 +
+ elif cmd == 'getName':
+   if len(args) != 0:
+     print('getName requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.getName())
+ 
+ elif cmd == 'getVersion':
+   if len(args) != 0:
+     print('getVersion requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.getVersion())
+ 
+ elif cmd == 'getStatus':
+   if len(args) != 0:
+     print('getStatus requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.getStatus())
+ 
+ elif cmd == 'getStatusDetails':
+   if len(args) != 0:
+     print('getStatusDetails requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.getStatusDetails())
+ 
+ elif cmd == 'getCounters':
+   if len(args) != 0:
+     print('getCounters requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.getCounters())
+ 
+ elif cmd == 'getCounter':
+   if len(args) != 1:
+     print('getCounter requires 1 args')
+     sys.exit(1)
+   pp.pprint(client.getCounter(args[0],))
+ 
+ elif cmd == 'setOption':
+   if len(args) != 2:
+     print('setOption requires 2 args')
+     sys.exit(1)
+   pp.pprint(client.setOption(args[0],args[1],))
+ 
+ elif cmd == 'getOption':
+   if len(args) != 1:
+     print('getOption requires 1 args')
+     sys.exit(1)
+   pp.pprint(client.getOption(args[0],))
+ 
+ elif cmd == 'getOptions':
+   if len(args) != 0:
+     print('getOptions requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.getOptions())
+ 
++elif cmd == 'getCpuProfile':
++  if len(args) != 1:
++    print('getCpuProfile requires 1 args')
++    sys.exit(1)
++  pp.pprint(client.getCpuProfile(eval(args[0]),))
++
+ elif cmd == 'aliveSince':
+   if len(args) != 0:
+     print('aliveSince requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.aliveSince())
+ 
 -elif cmd == 'getLimitedReflection':
 -  if len(args) != 0:
 -    print('getLimitedReflection requires 0 args')
 -    sys.exit(1)
 -  pp.pprint(client.getLimitedReflection())
 -
+ elif cmd == 'reinitialize':
+   if len(args) != 0:
+     print('reinitialize requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.reinitialize())
+ 
+ elif cmd == 'shutdown':
+   if len(args) != 0:
+     print('shutdown requires 0 args')
+     sys.exit(1)
+   pp.pprint(client.shutdown())
+ 
  else:
-   print 'Unrecognized method %s' % cmd
+   print('Unrecognized method %s' % cmd)
    sys.exit(1)
  
  transport.close()

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 4aad3aa,9e460f0..dd75b01
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@@ -1018,9 -1021,6 +1021,9 @@@ class Iface(fb303.FacebookService.Iface
      """
      pass
  
-   def flushCache(self, ):
++  def flushCache(self):
 +    pass
 +
  
  class Client(fb303.FacebookService.Client, Iface):
    """
@@@ -5438,29 -5589,6 +5592,30 @@@
        return result.success
      raise TApplicationException(TApplicationException.MISSING_RESULT, "fire_listener_event failed: unknown result");
  
-   def flushCache(self, ):
++  def flushCache(self):
 +    self.send_flushCache()
 +    self.recv_flushCache()
 +
-   def send_flushCache(self, ):
++  def send_flushCache(self):
 +    self._oprot.writeMessageBegin('flushCache', TMessageType.CALL, self._seqid)
 +    args = flushCache_args()
 +    args.write(self._oprot)
 +    self._oprot.writeMessageEnd()
 +    self._oprot.trans.flush()
 +
-   def recv_flushCache(self, ):
-     (fname, mtype, rseqid) = self._iprot.readMessageBegin()
++  def recv_flushCache(self):
++    iprot = self._iprot
++    (fname, mtype, rseqid) = iprot.readMessageBegin()
 +    if mtype == TMessageType.EXCEPTION:
 +      x = TApplicationException()
-       x.read(self._iprot)
-       self._iprot.readMessageEnd()
++      x.read(iprot)
++      iprot.readMessageEnd()
 +      raise x
 +    result = flushCache_result()
-     result.read(self._iprot)
-     self._iprot.readMessageEnd()
++    result.read(iprot)
++    iprot.readMessageEnd()
 +    return
 +
  
  class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
    def __init__(self, handler):
@@@ -27411,87 -29264,3 +29303,95 @@@ class fire_listener_event_result
  
    def __ne__(self, other):
      return not (self == other)
 +
 +class flushCache_args:
 +
 +  thrift_spec = (
 +  )
 +
 +  def read(self, iprot):
 +    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
 +      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
 +      return
 +    iprot.readStructBegin()
 +    while True:
 +      (fname, ftype, fid) = iprot.readFieldBegin()
 +      if ftype == TType.STOP:
 +        break
 +      else:
 +        iprot.skip(ftype)
 +      iprot.readFieldEnd()
 +    iprot.readStructEnd()
 +
 +  def write(self, oprot):
 +    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
 +      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
 +      return
 +    oprot.writeStructBegin('flushCache_args')
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    return
 +
 +
++  def __hash__(self):
++    value = 17
++    return value
++
 +  def __repr__(self):
 +    L = ['%s=%r' % (key, value)
 +      for key, value in self.__dict__.iteritems()]
 +    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
 +
 +  def __eq__(self, other):
 +    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
 +
 +  def __ne__(self, other):
 +    return not (self == other)
 +
 +class flushCache_result:
 +
 +  thrift_spec = (
 +  )
 +
 +  def read(self, iprot):
 +    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
 +      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
 +      return
 +    iprot.readStructBegin()
 +    while True:
 +      (fname, ftype, fid) = iprot.readFieldBegin()
 +      if ftype == TType.STOP:
 +        break
 +      else:
 +        iprot.skip(ftype)
 +      iprot.readFieldEnd()
 +    iprot.readStructEnd()
 +
 +  def write(self, oprot):
 +    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
 +      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
 +      return
 +    oprot.writeStructBegin('flushCache_result')
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    return
 +
 +
++  def __hash__(self):
++    value = 17
++    return value
++
 +  def __repr__(self):
 +    L = ['%s=%r' % (key, value)
 +      for key, value in self.__dict__.iteritems()]
 +    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
 +
 +  def __eq__(self, other):
 +    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
 +
 +  def __ne__(self, other):
 +    return not (self == other)

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------


[40/52] [abbrv] hive git commit: HIVE-11636 NPE in stats conversion with HBase metastore (Sergey Shelukhin via gates)

Posted by se...@apache.org.
HIVE-11636 NPE in stats conversion with HBase metastore (Sergey Shelukhin via gates)


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

Branch: refs/heads/llap
Commit: fbbb7cf1fa5691037243a6db3993f294ffb00eeb
Parents: e150af9
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Aug 28 11:03:26 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Aug 28 11:03:26 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/metastore/HiveMetaStore.java | 4 ++--
 .../org/apache/hadoop/hive/metastore/hbase/HBaseStore.java   | 8 ++++++++
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fbbb7cf1/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index a06efc6..df64124 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -4281,8 +4281,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
       try {
         ColumnStatistics cs = getMS().getTableColumnStatistics(dbName, tblName, lowerCaseColNames);
-        result = new TableStatsResult(
-            cs == null ? Lists.<ColumnStatisticsObj>newArrayList() : cs.getStatsObj());
+        result = new TableStatsResult((cs == null || cs.getStatsObj() == null)
+            ? Lists.<ColumnStatisticsObj>newArrayList() : cs.getStatsObj());
       } finally {
         endFunction("get_table_statistics_req: ", result == null, null, tblName);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/fbbb7cf1/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index 4cda9cc..df0fac3 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.RawStore;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -1693,9 +1694,11 @@ public class HBaseStore implements RawStore {
       partVals.add(partNameToVals(partName));
     }
     boolean commit = false;
+    boolean hasAnyStats = false;
     openTransaction();
     try {
       AggrStats aggrStats = new AggrStats();
+      aggrStats.setPartsFound(0);
       for (String colName : colNames) {
         try {
           AggrStats oneCol =
@@ -1704,6 +1707,7 @@ public class HBaseStore implements RawStore {
             assert oneCol.getColStatsSize() == 1;
             aggrStats.setPartsFound(oneCol.getPartsFound());
             aggrStats.addToColStats(oneCol.getColStats().get(0));
+            hasAnyStats = true;
           }
         } catch (CacheLoader.InvalidCacheLoadException e) {
           LOG.debug("Found no stats for column " + colName);
@@ -1712,6 +1716,10 @@ public class HBaseStore implements RawStore {
         }
       }
       commit = true;
+      if (!hasAnyStats) {
+        // Set the required field.
+        aggrStats.setColStats(new ArrayList<ColumnStatisticsObj>());
+      }
       return aggrStats;
     } catch (IOException e) {
       LOG.error("Unable to fetch aggregate column statistics", e);


[49/52] [abbrv] hive git commit: HIVE-11783: Extending HPL/SQL parser (Dmitry Tolpeko reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/offline/select_db2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/select_db2.out.txt b/hplsql/src/test/results/offline/select_db2.out.txt
new file mode 100644
index 0000000..1d64e8a
--- /dev/null
+++ b/hplsql/src/test/results/offline/select_db2.out.txt
@@ -0,0 +1,6 @@
+Ln:1 SELECT
+Ln:1 select coalesce(max(info_id) + 1, 0) from sproc_info
+Ln:1 Not executed - offline mode set
+Ln:3 SELECT
+Ln:3 select cd, cd + inc days, cd - inc days + coalesce(inc, 0) days from (select date '2015-09-02' as cd, 3 as inc from sysibm.sysdummy1)
+Ln:3 Not executed - offline mode set
\ No newline at end of file


[38/52] [abbrv] hive git commit: HIVE-11621 Fix TestMiniTezCliDriver test failures when HBase Metastore is used (Daniel Dai via gates)

Posted by se...@apache.org.
HIVE-11621 Fix TestMiniTezCliDriver test failures when HBase Metastore is used (Daniel Dai via gates)


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

Branch: refs/heads/llap
Commit: f014f0da457d1e959cb7da2824f6cf2c5ee0c971
Parents: 4d66206
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Aug 28 10:38:49 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Aug 28 10:38:49 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ant/QTestGenTask.java    |  11 ++
 data/conf/tez/hive-site.xml                     |   9 ++
 itests/qtest/pom.xml                            |   3 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |  80 +++++-------
 .../hadoop/hive/metastore/HiveAlterHandler.java |  22 ++--
 .../hive/metastore/hbase/HBaseReadWrite.java    |  17 ++-
 .../hadoop/hive/metastore/hbase/HBaseStore.java | 127 +++++++++++++++----
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |  60 ++++++---
 .../hadoop/hive/metastore/hbase/StatsCache.java |   2 +-
 .../stats/ColumnStatsAggregatorFactory.java     |   9 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |   3 +
 .../dynpart_sort_opt_vectorization.q            |   2 +
 .../clientpositive/dynpart_sort_optimization.q  |   2 +
 .../tez/dynpart_sort_opt_vectorization.q.out    |  12 +-
 .../tez/dynpart_sort_optimization.q.out         |  12 +-
 ql/src/test/templates/TestCliDriver.vm          |   3 +-
 16 files changed, 248 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
----------------------------------------------------------------------
diff --git a/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java b/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
index 7fffe13..8b1c4fe 100644
--- a/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
+++ b/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
@@ -149,6 +149,8 @@ public class QTestGenTask extends Task {
 
   private String cleanupScript;
 
+  private String useHBaseMetastore;
+
   public void setHadoopVersion(String ver) {
     this.hadoopVersion = ver;
   }
@@ -221,6 +223,14 @@ public class QTestGenTask extends Task {
     this.cleanupScript = cleanupScript;
   }
 
+  public String getUseHBaseMetastore() {
+    return useHBaseMetastore;
+  }
+
+  public void setUseHBaseMetastore(String useHBaseMetastore) {
+    this.useHBaseMetastore = useHBaseMetastore;
+  }
+
   public void setHiveRootDirectory(File hiveRootDirectory) {
     try {
       this.hiveRootDirectory = hiveRootDirectory.getCanonicalPath();
@@ -530,6 +540,7 @@ public class QTestGenTask extends Task {
       ctx.put("hadoopVersion", hadoopVersion);
       ctx.put("initScript", initScript);
       ctx.put("cleanupScript", cleanupScript);
+      ctx.put("useHBaseMetastore", useHBaseMetastore);
 
       File outFile = new File(outDir, className + ".java");
       FileWriter writer = new FileWriter(outFile);

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/data/conf/tez/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/tez/hive-site.xml b/data/conf/tez/hive-site.xml
index e0238aa..bcda3ea 100644
--- a/data/conf/tez/hive-site.xml
+++ b/data/conf/tez/hive-site.xml
@@ -253,5 +253,14 @@
   </description>
 </property>
 
+<property>
+  <name>hive.metastore.fastpath</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>hive.metastore.rawstore.impl</name>
+  <value>org.apache.hadoop.hive.metastore.hbase.HBaseStore</value>
+</property>
 
 </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index c2cb2f6..664068b 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -509,7 +509,8 @@
                               logDirectory="${project.build.directory}/qfile-results/clientpositive/"
                               hadoopVersion="${active.hadoop.version}"
                               initScript="${initScript}"
-                              cleanupScript="q_test_cleanup.sql"/>
+                              cleanupScript="q_test_cleanup.sql"
+                              useHBaseMetastore="true"/>
 
                     <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
                               outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hadoop/hive/cli/"

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 358fdbd..9f112ad 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -84,7 +84,6 @@ import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite;
-import org.apache.hadoop.hive.metastore.hbase.TephraHBaseConnection;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -109,6 +108,8 @@ import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.util.StreamPrinter;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
 import org.apache.tools.ant.BuildException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -166,12 +167,12 @@ public class QTestUtil {
 
   private final String initScript;
   private final String cleanupScript;
+  private boolean useHBaseMetastore = false;
 
   public interface SuiteAddTestFunctor {
     public void addTestToSuite(TestSuite suite, Object setup, String tName);
   }
   private HBaseTestingUtility utility;
-  private boolean snapshotTaken = false;
 
   static {
     for (String srcTable : System.getProperty("test.src.tables", "").trim().split(",")) {
@@ -348,61 +349,46 @@ public class QTestUtil {
     return "jceks://file" + new Path(keyDir, "test.jks").toUri();
   }
 
-  private void rebuildHBase() throws Exception {
-    HBaseAdmin admin = utility.getHBaseAdmin();
-    if (!snapshotTaken) {
-      for (String tableName : HBaseReadWrite.tableNames) {
-        List<byte[]> families = HBaseReadWrite.columnFamilies.get(tableName);
-        HTableDescriptor desc = new HTableDescriptor(
-            TableName.valueOf(tableName));
-        for (byte[] family : families) {
-          HColumnDescriptor columnDesc = new HColumnDescriptor(family);
-          desc.addFamily(columnDesc);
-        }
-        try {
-          admin.disableTable(tableName);
-          admin.deleteTable(tableName);
-        } catch (IOException e) {
-          System.out.println(e.getMessage());
-        }
-        admin.createTable(desc);
-      }
-    } else {
-      for (String tableName : HBaseReadWrite.tableNames) {
-        admin.disableTable(tableName);
-        admin.restoreSnapshot("snapshot_" + tableName);
-        admin.enableTable(tableName);
-      }
-      try {
-        db.createDatabase(new org.apache.hadoop.hive.metastore.api.Database(
-            DEFAULT_DATABASE_NAME, DEFAULT_DATABASE_COMMENT, new Warehouse(conf)
-                .getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString(), null));
-      } catch (Exception e) {
-        // Ignore if default database already exist
-      }
-      SessionState.get().setCurrentDatabase(DEFAULT_DATABASE_NAME);
-    }
-    admin.close();
-  }
-
   private void startMiniHBaseCluster() throws Exception {
     utility = new HBaseTestingUtility();
     utility.startMiniCluster();
     conf = new HiveConf(utility.getConfiguration(), Driver.class);
-    rebuildHBase();
+    conf = new HiveConf(utility.getConfiguration(), Driver.class);
+    HBaseAdmin admin = utility.getHBaseAdmin();
+    for (String tableName : HBaseReadWrite.tableNames) {
+      List<byte[]> families = HBaseReadWrite.columnFamilies.get(tableName);
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+      for (byte[] family : families) {
+        HColumnDescriptor columnDesc = new HColumnDescriptor(family);
+        desc.addFamily(columnDesc);
+      }
+      admin.createTable(desc);
+    }
+    admin.close();
     HBaseReadWrite.getInstance(conf);
   }
 
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
       String confDir, String hadoopVer, String initScript, String cleanupScript)
     throws Exception {
+    this(outDir, logDir, clusterType, confDir, hadoopVer, initScript, cleanupScript, false);
+  }
+  public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
+      String confDir, String hadoopVer, String initScript, String cleanupScript, boolean useHBaseMetastore)
+    throws Exception {
     this.outDir = outDir;
     this.logDir = logDir;
+    this.useHBaseMetastore = useHBaseMetastore;
+
+    Logger hadoopLog = Logger.getLogger("org.apache.hadoop");
+    hadoopLog.setLevel(Level.INFO);
     if (confDir != null && !confDir.isEmpty()) {
       HiveConf.setHiveSiteLocation(new URL("file://"+ new File(confDir).toURI().getPath() + "/hive-site.xml"));
       System.out.println("Setting hive-site: "+HiveConf.getHiveSiteLocation());
     }
-    startMiniHBaseCluster();
+    if (useHBaseMetastore) {
+      startMiniHBaseCluster();
+    }
     conf = new HiveConf(Driver.class);
     this.hadoopVer = getHadoopMainVersion(hadoopVer);
     qMap = new TreeMap<String, String>();
@@ -491,7 +477,9 @@ public class QTestUtil {
         sparkSession = null;
       }
     }
-    utility.shutdownMiniCluster();
+    if (useHBaseMetastore) {
+      utility.shutdownMiniCluster();
+    }
     if (mr != null) {
       mr.shutdown();
       mr = null;
@@ -779,8 +767,6 @@ public class QTestUtil {
       return;
     }
 
-    rebuildHBase();
-
     clearTablesCreatedDuringTests();
     clearKeysCreatedInTests();
 
@@ -880,12 +866,6 @@ public class QTestUtil {
     cliDriver.processLine(initCommands);
 
     conf.setBoolean("hive.test.init.phase", false);
-
-    HBaseAdmin admin = utility.getHBaseAdmin();
-    for (String tableName : HBaseReadWrite.tableNames) {
-      admin.snapshot("snapshot_" + tableName, tableName);
-    }
-    snapshotTaken = true;
   }
 
   public void init() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index f402f73..d9382ff 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -668,17 +668,19 @@ public class HiveAlterHandler implements AlterHandler {
           }
 
           List<ColumnStatisticsObj> statsObjs = cs.getStatsObj();
-          for (ColumnStatisticsObj statsObj : statsObjs) {
-            boolean found = false;
-            for (FieldSchema newCol : newCols) {
-              if (statsObj.getColName().equalsIgnoreCase(newCol.getName())
-                  && statsObj.getColType().equals(newCol.getType())) {
-                found = true;
-                break;
+          if (statsObjs != null) {
+            for (ColumnStatisticsObj statsObj : statsObjs) {
+              boolean found = false;
+              for (FieldSchema newCol : newCols) {
+                if (statsObj.getColName().equalsIgnoreCase(newCol.getName())
+                    && statsObj.getColType().equals(newCol.getType())) {
+                  found = true;
+                  break;
+                }
+              }
+              if (!found) {
+                msdb.deleteTableColumnStatistics(dbName, tableName, statsObj.getColName());
               }
-            }
-            if (!found) {
-              msdb.deleteTableColumnStatistics(dbName, tableName, statsObj.getColName());
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index f1336dc..8a1448c 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -680,7 +680,13 @@ public class HBaseReadWrite {
         firstStar = i;
         break;
       } else {
-        keyElements.add(partVals.get(i));
+        // empty string equals to null partition,
+        // means star
+        if (partVals.get(i).equals("")) {
+          break;
+        } else {
+          keyElements.add(partVals.get(i));
+        }
       }
     }
 
@@ -693,7 +699,7 @@ public class HBaseReadWrite {
     }
     keyPrefix = HBaseUtils.buildPartitionKey(dbName, tableName,
         HBaseUtils.getPartitionKeyTypes(table.getPartitionKeys().subList(0, keyElements.size()-2)),
-          keyElements.subList(0, keyElements.size()-2));
+          keyElements.subList(2, keyElements.size()));
 
     // Now, build a filter out of the remaining keys
     List<PartitionKeyComparator.Range> ranges = new ArrayList<PartitionKeyComparator.Range>();
@@ -809,7 +815,7 @@ public class HBaseReadWrite {
     for (int i = 0; i < numToFetch && iter.hasNext(); i++) {
       Result result = iter.next();
       HBaseUtils.StorageDescriptorParts sdParts = HBaseUtils.deserializePartition(dbName, tableName,
-          tablePartitions, result.getRow(), result.getValue(CATALOG_CF, CATALOG_COL));
+          tablePartitions, result.getRow(), result.getValue(CATALOG_CF, CATALOG_COL), staticConf);
       StorageDescriptor sd = getStorageDescriptor(sdParts.sdHash);
       HBaseUtils.assembleStorageDescriptor(sd, sdParts);
       parts.add(sdParts.containingPartition);
@@ -1604,9 +1610,10 @@ public class HBaseReadWrite {
             // recontruct the key.  We have to pull the dbName and tableName out of the key to
             // find the partition values.
             byte[] key = results[i].getRow();
-            String[] reconstructedKey = HBaseUtils.parseKey(key);
+            List<String> reconstructedKey = HBaseUtils.parseKey(key, HBaseUtils.getPartitionNames(getTable(dbName, tblName).getPartitionKeys()),
+                HBaseUtils.getPartitionKeyTypes(getTable(dbName, tblName).getPartitionKeys()));
             List<String> reconstructedPartVals =
-                Arrays.asList(reconstructedKey).subList(2, reconstructedKey.length);
+                reconstructedKey.subList(2, reconstructedKey.size());
             String partName = valToPartMap.get(reconstructedPartVals);
             assert partName != null;
             csd.setIsTblLevel(false);

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index f30fcab..568a347 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -20,15 +20,18 @@ package org.apache.hadoop.hive.metastore.hbase;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.CacheLoader;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
 import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
 import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -64,6 +67,9 @@ import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.PlanResult;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.ScanPlan;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hive.common.util.HiveStringUtils;
 import org.apache.thrift.TException;
 
 import java.io.IOException;
@@ -71,6 +77,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -150,7 +157,7 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      Database db = getHBase().getDb(name);
+      Database db = getHBase().getDb(HiveStringUtils.normalizeIdentifier(name));
       if (db == null) {
         throw new NoSuchObjectException("Unable to find db " + name);
       }
@@ -169,7 +176,7 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().deleteDb(dbname);
+      getHBase().deleteDb(HiveStringUtils.normalizeIdentifier(dbname));
       commit = true;
       return true;
     } catch (IOException e) {
@@ -259,7 +266,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().deleteTable(dbName, tableName);
+      getHBase().deleteTable(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tableName));
       commit = true;
       return true;
     } catch (IOException e) {
@@ -275,7 +283,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      Table table = getHBase().getTable(dbName, tableName);
+      Table table = getHBase().getTable(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tableName));
       if (table == null) {
         LOG.debug("Unable to find table " + tableNameForErrorMsg(dbName, tableName));
       }
@@ -334,7 +343,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      Partition part = getHBase().getPartition(dbName, tableName, part_vals);
+      Partition part = getHBase().getPartition(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tableName), part_vals);
       if (part == null) {
         throw new NoSuchObjectException("Unable to find partition " +
             partNameForErrorMsg(dbName, tableName, part_vals));
@@ -355,7 +365,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      boolean exists = getHBase().getPartition(dbName, tableName, part_vals) != null;
+      boolean exists = getHBase().getPartition(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tableName), part_vals) != null;
       commit = true;
       return exists;
     } catch (IOException e) {
@@ -372,9 +383,11 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().deletePartition(dbName, tableName, part_vals);
+      getHBase().deletePartition(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tableName), part_vals);
       // Drop any cached stats that reference this partitions
-      getHBase().getStatsCache().invalidate(dbName, tableName,
+      getHBase().getStatsCache().invalidate(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tableName),
           buildExternalPartName(dbName, tableName, part_vals));
       commit = true;
       return true;
@@ -393,7 +406,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Partition> parts = getHBase().scanPartitionsInTable(dbName, tableName, max);
+      List<Partition> parts = getHBase().scanPartitionsInTable(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tableName), max);
       commit = true;
       return parts;
     } catch (IOException e) {
@@ -410,7 +424,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().replaceTable(getHBase().getTable(dbname, name), newTable);
+      getHBase().replaceTable(getHBase().getTable(HiveStringUtils.normalizeIdentifier(dbname),
+          HiveStringUtils.normalizeIdentifier(name)), newTable);
       if (newTable.getPartitionKeys() != null && newTable.getPartitionKeys().size() > 0
           && !name.equals(newTable.getTableName())) {
         // They renamed the table, so we need to change each partition as well, since it changes
@@ -443,7 +458,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Table> tables = getHBase().scanTables(dbName, likeToRegex(pattern));
+      List<Table> tables = getHBase().scanTables(HiveStringUtils.normalizeIdentifier(dbName),
+          likeToRegex(pattern));
       List<String> tableNames = new ArrayList<String>(tables.size());
       for (Table table : tables) tableNames.add(table.getTableName());
       commit = true;
@@ -462,7 +478,12 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Table> tables = getHBase().getTables(dbname, tableNames);
+      List<String> normalizedTableNames = new ArrayList<String>(tableNames.size());
+      for (String tableName : tableNames) {
+        normalizedTableNames.add(HiveStringUtils.normalizeIdentifier(tableName));
+      }
+      List<Table> tables = getHBase().getTables(HiveStringUtils.normalizeIdentifier(dbname),
+          normalizedTableNames);
       commit = true;
       return tables;
     } catch (IOException e) {
@@ -491,10 +512,12 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Partition> parts = getHBase().scanPartitionsInTable(db_name, tbl_name, max_parts);
+      List<Partition> parts = getHBase().scanPartitionsInTable(HiveStringUtils.normalizeIdentifier(db_name),
+          HiveStringUtils.normalizeIdentifier(tbl_name), max_parts);
       if (parts == null) return null;
       List<String> names = new ArrayList<String>(parts.size());
-      Table table = getHBase().getTable(db_name, tbl_name);
+      Table table = getHBase().getTable(HiveStringUtils.normalizeIdentifier(db_name),
+          HiveStringUtils.normalizeIdentifier(tbl_name));
       for (Partition p : parts) {
         names.add(buildExternalPartName(table, p));
       }
@@ -521,10 +544,12 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      Partition oldPart = getHBase().getPartition(db_name, tbl_name, part_vals);
+      Partition oldPart = getHBase().getPartition(HiveStringUtils.normalizeIdentifier(db_name),
+          HiveStringUtils.normalizeIdentifier(tbl_name), part_vals);
       getHBase().replacePartition(oldPart, new_part);
       // Drop any cached stats that reference this partitions
-      getHBase().getStatsCache().invalidate(db_name, tbl_name,
+      getHBase().getStatsCache().invalidate(HiveStringUtils.normalizeIdentifier(db_name),
+          HiveStringUtils.normalizeIdentifier(tbl_name),
           buildExternalPartName(db_name, tbl_name, part_vals));
       commit = true;
     } catch (IOException e) {
@@ -542,11 +567,14 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Partition> oldParts = getHBase().getPartitions(db_name, tbl_name,
-          HBaseUtils.getPartitionKeyTypes(getTable(db_name, tbl_name).getPartitionKeys()), part_vals_list);
+      List<Partition> oldParts = getHBase().getPartitions(HiveStringUtils.normalizeIdentifier(db_name),
+          HiveStringUtils.normalizeIdentifier(tbl_name),
+          HBaseUtils.getPartitionKeyTypes(getTable(HiveStringUtils.normalizeIdentifier(db_name),
+          HiveStringUtils.normalizeIdentifier(tbl_name)).getPartitionKeys()), part_vals_list);
       getHBase().replacePartitions(oldParts, new_parts);
       for (List<String> part_vals : part_vals_list) {
-        getHBase().getStatsCache().invalidate(db_name, tbl_name,
+        getHBase().getStatsCache().invalidate(HiveStringUtils.normalizeIdentifier(db_name),
+            HiveStringUtils.normalizeIdentifier(tbl_name),
             buildExternalPartName(db_name, tbl_name, part_vals));
       }
       commit = true;
@@ -604,7 +632,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getPartitionsByExprInternal(dbName, tblName, exprTree, maxParts, result);
+      getPartitionsByExprInternal(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tblName), exprTree, maxParts, result);
       return result;
     } finally {
       commitOrRoleBack(commit);
@@ -616,22 +645,62 @@ public class HBaseStore implements RawStore {
                                      String defaultPartitionName, short maxParts,
                                      List<Partition> result) throws TException {
     final ExpressionTree exprTree = PartFilterExprUtil.makeExpressionTree(expressionProxy, expr);
-    // TODO: investigate if there should be any role for defaultPartitionName in this
-    // implementation. direct sql code path in ObjectStore does not use it.
-
+    dbName = HiveStringUtils.normalizeIdentifier(dbName);
+    tblName = HiveStringUtils.normalizeIdentifier(tblName);
+    Table table = getTable(dbName, tblName);
     boolean commit = false;
     openTransaction();
     try {
-      return getPartitionsByExprInternal(dbName, tblName, exprTree, maxParts, result);
+      if (exprTree == null) {
+        List<String> partNames = new LinkedList<String>();
+        boolean hasUnknownPartitions = getPartitionNamesPrunedByExprNoTxn(
+            table, expr, defaultPartitionName, maxParts, partNames);
+        result.addAll(getPartitionsByNames(dbName, tblName, partNames));
+        return hasUnknownPartitions;
+      } else {
+        return getPartitionsByExprInternal(dbName, tblName, exprTree, maxParts, result);
+      }
     } finally {
       commitOrRoleBack(commit);
     }
   }
 
+  /**
+   * Gets the partition names from a table, pruned using an expression.
+   * @param table Table.
+   * @param expr Expression.
+   * @param defaultPartName Default partition name from job config, if any.
+   * @param maxParts Maximum number of partition names to return.
+   * @param result The resulting names.
+   * @return Whether the result contains any unknown partitions.
+   * @throws NoSuchObjectException
+   */
+  private boolean getPartitionNamesPrunedByExprNoTxn(Table table, byte[] expr,
+      String defaultPartName, short maxParts, List<String> result) throws MetaException, NoSuchObjectException {
+    List<Partition> parts = getPartitions(
+        table.getDbName(), table.getTableName(), maxParts);
+    for (Partition part : parts) {
+      result.add(Warehouse.makePartName(table.getPartitionKeys(), part.getValues()));
+    }
+    List<String> columnNames = new ArrayList<String>();
+    List<PrimitiveTypeInfo> typeInfos = new ArrayList<PrimitiveTypeInfo>();
+    for (FieldSchema fs : table.getPartitionKeys()) {
+      columnNames.add(fs.getName());
+      typeInfos.add(TypeInfoFactory.getPrimitiveTypeInfo(fs.getType()));
+    }
+    if (defaultPartName == null || defaultPartName.isEmpty()) {
+      defaultPartName = HiveConf.getVar(getConf(), HiveConf.ConfVars.DEFAULTPARTITIONNAME);
+    }
+    return expressionProxy.filterPartitionsByExpr(
+        columnNames, typeInfos, expr, defaultPartName, result);
+  }
+
   private boolean getPartitionsByExprInternal(String dbName, String tblName,
       ExpressionTree exprTree, short maxParts, List<Partition> result) throws MetaException,
       NoSuchObjectException {
 
+    dbName = HiveStringUtils.normalizeIdentifier(dbName);
+    tblName = HiveStringUtils.normalizeIdentifier(tblName);
     Table table = getTable(dbName, tblName);
     if (table == null) {
       throw new NoSuchObjectException("Unable to find table " + dbName + "." + tblName);
@@ -1453,7 +1522,8 @@ public class HBaseStore implements RawStore {
         listPartitionsPsWithAuth(db_name, tbl_name, part_vals, max_parts, null, null);
     List<String> partNames = new ArrayList<String>(parts.size());
     for (Partition part : parts) {
-      partNames.add(buildExternalPartName(db_name, tbl_name, part.getValues()));
+      partNames.add(buildExternalPartName(HiveStringUtils.normalizeIdentifier(db_name),
+          HiveStringUtils.normalizeIdentifier(tbl_name), part.getValues()));
     }
     return partNames;
   }
@@ -1468,7 +1538,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Partition> parts = getHBase().scanPartitions(db_name, tbl_name, part_vals, max_parts);
+      List<Partition> parts = getHBase().scanPartitions(HiveStringUtils.normalizeIdentifier(db_name),
+          HiveStringUtils.normalizeIdentifier(tbl_name), part_vals, max_parts);
       commit = true;
       return parts;
     } catch (IOException e) {
@@ -1596,7 +1667,7 @@ public class HBaseStore implements RawStore {
               getHBase().getStatsCache().get(dbName, tblName, partNames, colName);
           if (oneCol.getColStatsSize() > 0) {
             assert oneCol.getColStatsSize() == 1;
-            aggrStats.setPartsFound(aggrStats.getPartsFound() + oneCol.getPartsFound());
+            aggrStats.setPartsFound(oneCol.getPartsFound());
             aggrStats.addToColStats(oneCol.getColStats().get(0));
           }
         } catch (CacheLoader.InvalidCacheLoadException e) {
@@ -2204,7 +2275,7 @@ public class HBaseStore implements RawStore {
     List<String> vals = new ArrayList<String>();
     String[] kvp = name.split("/");
     for (String kv : kvp) {
-      vals.add(kv.substring(kv.indexOf('=') + 1));
+      vals.add(FileUtils.unescapePathName(kv.substring(kv.indexOf('=') + 1)));
     }
     return vals;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index 841afd4..cc90a76 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -26,6 +26,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
@@ -67,6 +68,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hive.common.util.BloomFilter;
+import org.apache.hive.common.util.HiveStringUtils;
 
 import java.io.IOException;
 import java.nio.charset.Charset;
@@ -115,9 +117,22 @@ class HBaseUtils {
     return protoKey.getBytes(ENCODING);
   }
 
-  static String[] parseKey(byte[] serialized) {
-    String munged = new String(serialized, ENCODING);
-    return munged.split(KEY_SEPARATOR_STR);
+  static List<String> parseKey(byte[] serialized, List<String> partNames, List<String> partTypes) {
+    BinarySortableSerDe serDe = new BinarySortableSerDe();
+    Properties props = new Properties();
+    props.setProperty(serdeConstants.LIST_COLUMNS, "dbName,tableName," + StringUtils.join(partNames, ","));
+    props.setProperty(serdeConstants.LIST_COLUMN_TYPES, "string,string," + StringUtils.join(partTypes, ","));
+    List<String> partVals = null;
+    try {
+      serDe.initialize(new Configuration(), props);
+      List deserializedkeys = ((List)serDe.deserialize(new BytesWritable(serialized)));
+      partVals = new ArrayList<String>();
+      for (Object deserializedkey : deserializedkeys) {
+        partVals.add(deserializedkey.toString());
+      }
+    } catch (SerDeException e) {
+    }
+    return partVals;
   }
 
   private static HbaseMetastoreProto.Parameters buildParameters(Map<String, String> params) {
@@ -240,9 +255,16 @@ class HBaseUtils {
 
   private static PrincipalPrivilegeSet buildPrincipalPrivilegeSet(
       HbaseMetastoreProto.PrincipalPrivilegeSet proto) throws InvalidProtocolBufferException {
-    PrincipalPrivilegeSet pps = new PrincipalPrivilegeSet();
-    pps.setUserPrivileges(convertPrincipalPrivilegeSetEntries(proto.getUsersList()));
-    pps.setRolePrivileges(convertPrincipalPrivilegeSetEntries(proto.getRolesList()));
+    PrincipalPrivilegeSet pps = null;
+    if (!proto.getUsersList().isEmpty() || !proto.getRolesList().isEmpty()) {
+      pps = new PrincipalPrivilegeSet();
+      if (!proto.getUsersList().isEmpty()) {
+        pps.setUserPrivileges(convertPrincipalPrivilegeSetEntries(proto.getUsersList()));
+      }
+      if (!proto.getRolesList().isEmpty()) {
+        pps.setRolePrivileges(convertPrincipalPrivilegeSetEntries(proto.getRolesList()));
+      }
+    }
     return pps;
   }
   /**
@@ -339,7 +361,7 @@ class HBaseUtils {
    */
   static byte[][] serializeDatabase(Database db) {
     byte[][] result = new byte[2][];
-    result[0] = buildKey(db.getName());
+    result[0] = buildKey(HiveStringUtils.normalizeIdentifier(db.getName()));
     HbaseMetastoreProto.Database.Builder builder = HbaseMetastoreProto.Database.newBuilder();
 
     if (db.getDescription() != null) builder.setDescription(db.getDescription());
@@ -696,8 +718,10 @@ class HBaseUtils {
     sd.setNumBuckets(proto.getNumBuckets());
     if (proto.hasSerdeInfo()) {
       SerDeInfo serde = new SerDeInfo();
-      serde.setName(proto.getSerdeInfo().getName());
-      serde.setSerializationLib(proto.getSerdeInfo().getSerializationLib());
+      serde.setName(proto.getSerdeInfo().hasName()?
+          proto.getSerdeInfo().getName():null);
+      serde.setSerializationLib(proto.getSerdeInfo().hasSerializationLib()?
+          proto.getSerdeInfo().getSerializationLib():null);
       serde.setParameters(buildParameters(proto.getSerdeInfo().getParameters()));
       sd.setSerdeInfo(serde);
     }
@@ -848,8 +872,8 @@ class HBaseUtils {
    * @return A struct that contains the partition plus parts of the storage descriptor
    */
   static StorageDescriptorParts deserializePartition(String dbName, String tableName, List<FieldSchema> partitions,
-      byte[] key, byte[] serialized) throws InvalidProtocolBufferException {
-    List keys = deserializePartitionKey(partitions, key);
+      byte[] key, byte[] serialized, Configuration conf) throws InvalidProtocolBufferException {
+    List keys = deserializePartitionKey(partitions, key, conf);
     return deserializePartition(dbName, tableName, keys, serialized);
   }
 
@@ -886,7 +910,8 @@ class HBaseUtils {
     return k.split(KEY_SEPARATOR_STR);
   }
 
-  private static List<String> deserializePartitionKey(List<FieldSchema> partitions, byte[] key) {
+  private static List<String> deserializePartitionKey(List<FieldSchema> partitions, byte[] key,
+      Configuration conf) {
     StringBuffer names = new StringBuffer();
     names.append("dbName,tableName,");
     StringBuffer types = new StringBuffer();
@@ -908,7 +933,8 @@ class HBaseUtils {
       List deserializedkeys = ((List)serDe.deserialize(new BytesWritable(key))).subList(2, partitions.size()+2);
       List<String> partitionKeys = new ArrayList<String>();
       for (Object deserializedKey : deserializedkeys) {
-        partitionKeys.add(deserializedKey.toString());
+        partitionKeys.add(deserializedKey!=null?deserializedKey.toString():
+          HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME));
       }
       return partitionKeys;
     } catch (SerDeException e) {
@@ -924,7 +950,8 @@ class HBaseUtils {
    */
   static byte[][] serializeTable(Table table, byte[] sdHash) {
     byte[][] result = new byte[2][];
-    result[0] = buildKey(table.getDbName(), table.getTableName());
+    result[0] = buildKey(HiveStringUtils.normalizeIdentifier(table.getDbName()),
+        HiveStringUtils.normalizeIdentifier(table.getTableName()));
     HbaseMetastoreProto.Table.Builder builder = HbaseMetastoreProto.Table.newBuilder();
     if (table.getOwner() != null) builder.setOwner(table.getOwner());
     builder
@@ -952,7 +979,10 @@ class HBaseUtils {
     if (table.getPrivileges() != null) {
       builder.setPrivileges(buildPrincipalPrivilegeSet(table.getPrivileges()));
     }
-    builder.setIsTemporary(table.isTemporary());
+    // Set only if table is temporary
+    if (table.isTemporary()) {
+      builder.setIsTemporary(table.isTemporary());
+    }
     result[1] = builder.build().toByteArray();
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java
index 0d3ed40..42efe94 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java
@@ -99,7 +99,7 @@ class StatsCache {
                   for (ColumnStatisticsObj cso : cs.getStatsObj()) {
                     if (statsObj == null) {
                       statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(key.colName,
-                          cso.getStatsData().getSetField());
+                          cso.getColType(), cso.getStatsData().getSetField());
                     }
                     if (aggregator == null) {
                       aggregator = ColumnStatsAggregatorFactory.getColumnStatsAggregator(

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java
index ebecfe3..a8dbc1f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java
@@ -53,39 +53,34 @@ public class ColumnStatsAggregatorFactory {
     }
   }
 
-  public static ColumnStatisticsObj newColumnStaticsObj(String colName, _Fields type) {
+  public static ColumnStatisticsObj newColumnStaticsObj(String colName, String colType, _Fields type) {
     ColumnStatisticsObj cso = new ColumnStatisticsObj();
     ColumnStatisticsData csd = new ColumnStatisticsData();
     cso.setColName(colName);
+    cso.setColType(colType);
     switch (type) {
     case BOOLEAN_STATS:
       csd.setBooleanStats(new BooleanColumnStatsData());
-      cso.setColType("boolean");
       break;
 
     case LONG_STATS:
       csd.setLongStats(new LongColumnStatsData());
-      cso.setColType("long");
       break;
 
     case DOUBLE_STATS:
       csd.setDoubleStats(new DoubleColumnStatsData());
-      cso.setColType("double");
       break;
 
     case STRING_STATS:
       csd.setStringStats(new StringColumnStatsData());
-      cso.setColType("string");
       break;
 
     case BINARY_STATS:
       csd.setBinaryStats(new BinaryColumnStatsData());
-      cso.setColType("binary");
       break;
 
     case DECIMAL_STATS:
       csd.setDecimalStats(new DecimalColumnStatsData());
-      cso.setColType("decimal");
       break;
 
     default:

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index 55aea0e..351cb2b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -737,6 +737,9 @@ public class StatsUtils {
   }
 
   private static List<ColStatistics> convertColStats(List<ColumnStatisticsObj> colStats, String tabName) {
+    if (colStats==null) {
+      return new ArrayList<ColStatistics>();
+    }
     List<ColStatistics> stats = new ArrayList<ColStatistics>(colStats.size());
     for (ColumnStatisticsObj statObj : colStats) {
       ColStatistics cs = getColStatistics(statObj, tabName, statObj.getColName());

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q b/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
index 8001081..7e94f23 100644
--- a/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
+++ b/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
@@ -123,6 +123,7 @@ insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t
 desc formatted over1k_part2_orc partition(ds="foo",t=27);
 desc formatted over1k_part2_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
 
+-- SORT_BEFORE_DIFF
 select * from over1k_part2_orc;
 select count(*) from over1k_part2_orc;
 
@@ -132,6 +133,7 @@ insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t
 desc formatted over1k_part2_orc partition(ds="foo",t=27);
 desc formatted over1k_part2_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
 
+-- SORT_BEFORE_DIFF
 select * from over1k_part2_orc;
 select count(*) from over1k_part2_orc;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q b/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
index f842efe..ea670e9 100644
--- a/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
+++ b/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
@@ -117,6 +117,7 @@ insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from
 desc formatted over1k_part2 partition(ds="foo",t=27);
 desc formatted over1k_part2 partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
 
+-- SORT_BEFORE_DIFF
 select * from over1k_part2;
 select count(*) from over1k_part2;
 
@@ -126,6 +127,7 @@ insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from
 desc formatted over1k_part2 partition(ds="foo",t=27);
 desc formatted over1k_part2 partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
 
+-- SORT_BEFORE_DIFF
 select * from over1k_part2;
 select count(*) from over1k_part2;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
index 4451046..f0fc221 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
@@ -1835,13 +1835,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2_orc
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2_orc
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2_orc
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2_orc
 POSTHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
@@ -1980,13 +1982,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2_orc
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2_orc
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2_orc
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2_orc
 POSTHOOK: Input: default@over1k_part2_orc@ds=foo/t=27

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
index cb001b9..8d4c1b7 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
@@ -1735,13 +1735,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2
 PREHOOK: Input: default@over1k_part2@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2
 POSTHOOK: Input: default@over1k_part2@ds=foo/t=27
@@ -1880,13 +1882,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2
 PREHOOK: Input: default@over1k_part2@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2
 POSTHOOK: Input: default@over1k_part2@ds=foo/t=27

http://git-wip-us.apache.org/repos/asf/hive/blob/f014f0da/ql/src/test/templates/TestCliDriver.vm
----------------------------------------------------------------------
diff --git a/ql/src/test/templates/TestCliDriver.vm b/ql/src/test/templates/TestCliDriver.vm
index ae449c5..01745da 100644
--- a/ql/src/test/templates/TestCliDriver.vm
+++ b/ql/src/test/templates/TestCliDriver.vm
@@ -45,13 +45,14 @@ public class $className extends TestCase {
     String hiveConfDir = "$hiveConfDir";
     String initScript = "$initScript";
     String cleanupScript = "$cleanupScript";
+    boolean useHBaseMetastore = Boolean.valueOf("$useHBaseMetastore");
     try {
       String hadoopVer = "$hadoopVersion";
       if (!hiveConfDir.isEmpty()) {
         hiveConfDir = HIVE_ROOT + hiveConfDir;
       }
       qt = new QTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
-      hiveConfDir, hadoopVer, initScript, cleanupScript);
+      hiveConfDir, hadoopVer, initScript, cleanupScript, useHBaseMetastore);
 
       // do a one time initialization
       qt.cleanUp();


[10/52] [abbrv] hive git commit: HIVE-11389 hbase import should allow partial imports and should work in parallel (gates)

Posted by se...@apache.org.
HIVE-11389 hbase import should allow partial imports and should work in parallel (gates)


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

Branch: refs/heads/llap
Commit: 0fa45e4a562fc2586b1ef06a88e9c186a0835316
Parents: 7e7f461
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Jul 31 11:07:00 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Jul 31 11:07:00 2015 -0700

----------------------------------------------------------------------
 .../hive/metastore/hbase/TestHBaseImport.java   | 557 +++++++++++++++++--
 .../hive/metastore/hbase/HBaseImport.java       | 435 +++++++++++++--
 2 files changed, 899 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0fa45e4a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
index 7bdff18..1ac10f0 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.FunctionType;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.ResourceType;
@@ -38,12 +41,16 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 /**
  * Test that import from an RDBMS based metastore works
@@ -52,6 +59,13 @@ public class TestHBaseImport extends HBaseIntegrationTests {
 
   private static final Log LOG = LogFactory.getLog(TestHBaseStoreIntegration.class.getName());
 
+  private static final String[] tableNames = new String[] {"allnonparttable", "allparttable"};
+  private static final String[] partVals = new String[] {"na", "emea", "latam", "apac"};
+  private static final String[] funcNames = new String[] {"allfunc1", "allfunc2"};
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
   @BeforeClass
   public static void startup() throws Exception {
     HBaseIntegrationTests.startMiniCluster();
@@ -69,25 +83,396 @@ public class TestHBaseImport extends HBaseIntegrationTests {
   }
 
   @Test
-  public void doImport() throws Exception {
-    RawStore rdbms = new ObjectStore();
+  public void importAll() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+
+    String[] dbNames = new String[] {"alldb1", "alldb2"};
+    String[] roles = new String[] {"allrole1", "allrole2"};
+    String[] tokenIds = new String[] {"alltokenid1", "alltokenid2"};
+    String[] tokens = new String[] {"alltoken1", "alltoken2"};
+    String[] masterKeys = new String[] {"allmk1", "allmk2"};
+    int now = (int)System.currentTimeMillis() / 1000;
+
+    setupObjectStore(rdbms, roles, dbNames, tokenIds, tokens, masterKeys, now);
+
+    int baseNumRoles = store.listRoleNames() == null ? 0 : store.listRoleNames().size();
+    int baseNumDbs = store.getAllDatabases() == null ? 0 : store.getAllDatabases().size();
+
+    HBaseImport importer = new HBaseImport("-a");
+    importer.setConnections(rdbms, store);
+    importer.run();
+
+    for (int i = 0; i < roles.length; i++) {
+      Role role = store.getRole(roles[i]);
+      Assert.assertNotNull(role);
+      Assert.assertEquals(roles[i], role.getRoleName());
+    }
+    // Make sure there aren't any extra roles
+    Assert.assertEquals(baseNumRoles + 2, store.listRoleNames().size());
+
+    for (int i = 0; i < dbNames.length; i++) {
+      Database db = store.getDatabase(dbNames[i]);
+      Assert.assertNotNull(db);
+      // check one random value in the db rather than every value
+      Assert.assertEquals("file:/tmp", db.getLocationUri());
+
+      Table table = store.getTable(db.getName(), tableNames[0]);
+      Assert.assertNotNull(table);
+      Assert.assertEquals(now, table.getLastAccessTime());
+      Assert.assertEquals("input", table.getSd().getInputFormat());
+
+      table = store.getTable(db.getName(), tableNames[1]);
+      Assert.assertNotNull(table);
+
+      for (int j = 0; j < partVals.length; j++) {
+        Partition part = store.getPartition(dbNames[i], tableNames[1], Arrays.asList(partVals[j]));
+        Assert.assertNotNull(part);
+        Assert.assertEquals("file:/tmp/region=" + partVals[j], part.getSd().getLocation());
+      }
+
+      Assert.assertEquals(4, store.getPartitions(dbNames[i], tableNames[1], -1).size());
+      Assert.assertEquals(2, store.getAllTables(dbNames[i]).size());
+
+      Assert.assertEquals(2, store.getFunctions(dbNames[i], "*").size());
+      for (int j = 0; j < funcNames.length; j++) {
+        Assert.assertNotNull(store.getFunction(dbNames[i], funcNames[j]));
+      }
+    }
+
+    Assert.assertEquals(baseNumDbs + 2, store.getAllDatabases().size());
+
+    // I can't test total number of tokens or master keys because the import grabs all and copies
+    // them, which means it grabs the ones imported by importSecurity test (if it's already run).
+    // Depending on it already running would make the tests order dependent, which junit doesn't
+    // guarantee.
+    for (int i = 0; i < tokenIds.length; i++) {
+      Assert.assertEquals(tokens[i], store.getToken(tokenIds[i]));
+    }
+    String[] hbaseKeys = store.getMasterKeys();
+    Set<String> keys = new HashSet<>(Arrays.asList(hbaseKeys));
+    for (int i = 0; i < masterKeys.length; i++) {
+      Assert.assertTrue(keys.contains(masterKeys[i]));
+    }
+  }
+
+  @Test
+  public void importOneDb() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+
+    String[] dbNames = new String[] {"onedbdb1", "onedbdb2"};
+    String[] roles = new String[] {"onedbrole1", "onedbrole2"};
+    String[] tokenIds = new String[] {"onedbtokenid1", "onedbtokenid2"};
+    String[] tokens = new String[] {"onedbtoken1", "onedbtoken2"};
+    String[] masterKeys = new String[] {"onedbmk1", "onedbmk2"};
+    int now = (int)System.currentTimeMillis() / 1000;
+
+    setupObjectStore(rdbms, roles, dbNames, tokenIds, tokens, masterKeys, now);
+
+    int baseNumRoles = store.listRoleNames() == null ? 0 : store.listRoleNames().size();
+    int baseNumDbs = store.getAllDatabases() == null ? 0 : store.getAllDatabases().size();
+    int baseNumToks = store.getAllTokenIdentifiers() == null ? 0 :
+        store.getAllTokenIdentifiers().size();
+    int baseNumKeys =  store.getMasterKeys() == null ? 0 : store.getMasterKeys().length;
+
+    HBaseImport importer = new HBaseImport("-d", dbNames[0]);
+    importer.setConnections(rdbms, store);
+    importer.run();
+
+    // Make sure there aren't any extra roles
+    Assert.assertEquals(baseNumRoles, store.listRoleNames().size());
+
+    Database db = store.getDatabase(dbNames[0]);
+    Assert.assertNotNull(db);
+    // check one random value in the db rather than every value
+    Assert.assertEquals("file:/tmp", db.getLocationUri());
+
+    Table table = store.getTable(db.getName(), tableNames[0]);
+    Assert.assertNotNull(table);
+    Assert.assertEquals(now, table.getLastAccessTime());
+    Assert.assertEquals("input", table.getSd().getInputFormat());
+
+    table = store.getTable(db.getName(), tableNames[1]);
+    Assert.assertNotNull(table);
+
+    for (int j = 0; j < partVals.length; j++) {
+      Partition part = store.getPartition(dbNames[0], tableNames[1], Arrays.asList(partVals[j]));
+      Assert.assertNotNull(part);
+      Assert.assertEquals("file:/tmp/region=" + partVals[j], part.getSd().getLocation());
+    }
+
+    Assert.assertEquals(4, store.getPartitions(dbNames[0], tableNames[1], -1).size());
+    Assert.assertEquals(2, store.getAllTables(dbNames[0]).size());
+
+    Assert.assertEquals(2, store.getFunctions(dbNames[0], "*").size());
+    for (int j = 0; j < funcNames.length; j++) {
+      Assert.assertNotNull(store.getFunction(dbNames[0], funcNames[j]));
+    }
+
+    Assert.assertEquals(baseNumDbs + 1, store.getAllDatabases().size());
+
+    Assert.assertEquals(baseNumToks, store.getAllTokenIdentifiers().size());
+    String[] hbaseKeys = store.getMasterKeys();
+    Assert.assertEquals(baseNumKeys, hbaseKeys.length);
+
+    // Have to do this last as it will throw an exception
+    thrown.expect(NoSuchObjectException.class);
+    store.getDatabase(dbNames[1]);
+  }
+
+  @Test
+  public void importOneFunc() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+
+    String[] dbNames = new String[] {"onefuncdb1", "onefuncdb2"};
+    String[] roles = new String[] {"onefuncrole1", "onefuncrole2"};
+    String[] tokenIds = new String[] {"onefunctokenid1", "onefunctokenid2"};
+    String[] tokens = new String[] {"onefunctoken1", "onefunctoken2"};
+    String[] masterKeys = new String[] {"onefuncmk1", "onefuncmk2"};
+    int now = (int)System.currentTimeMillis() / 1000;
+
+    setupObjectStore(rdbms, roles, dbNames, tokenIds, tokens, masterKeys, now);
+
+    int baseNumRoles = store.listRoleNames() == null ? 0 : store.listRoleNames().size();
+    int baseNumDbs = store.getAllDatabases() == null ? 0 : store.getAllDatabases().size();
+    int baseNumToks = store.getAllTokenIdentifiers() == null ? 0 :
+        store.getAllTokenIdentifiers().size();
+    int baseNumKeys =  store.getMasterKeys() == null ? 0 : store.getMasterKeys().length;
+
+    // Create the database so I can put the function in it.
+    store.createDatabase(
+        new Database(dbNames[0], "no description", "file:/tmp", emptyParameters));
+
+    HBaseImport importer = new HBaseImport("-f", dbNames[0] + "." + funcNames[0]);
+    importer.setConnections(rdbms, store);
+    importer.run();
+
+    // Make sure there aren't any extra roles
+    Assert.assertEquals(baseNumRoles, store.listRoleNames().size());
+
+    Database db = store.getDatabase(dbNames[0]);
+    Assert.assertNotNull(db);
+
+    Assert.assertEquals(0, store.getAllTables(dbNames[0]).size());
+    Assert.assertEquals(1, store.getFunctions(dbNames[0], "*").size());
+    Assert.assertNotNull(store.getFunction(dbNames[0], funcNames[0]));
+    Assert.assertNull(store.getFunction(dbNames[0], funcNames[1]));
+
+    Assert.assertEquals(baseNumDbs + 1, store.getAllDatabases().size());
+
+    Assert.assertEquals(baseNumToks, store.getAllTokenIdentifiers().size());
+    String[] hbaseKeys = store.getMasterKeys();
+    Assert.assertEquals(baseNumKeys, hbaseKeys.length);
+  }
+
+  @Test
+  public void importOneTableNonPartitioned() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+
+    String[] dbNames = new String[] {"onetabdb1", "onetabdb2"};
+    String[] roles = new String[] {"onetabrole1", "onetabrole2"};
+    String[] tokenIds = new String[] {"onetabtokenid1", "onetabtokenid2"};
+    String[] tokens = new String[] {"onetabtoken1", "onetabtoken2"};
+    String[] masterKeys = new String[] {"onetabmk1", "onetabmk2"};
+    int now = (int)System.currentTimeMillis() / 1000;
+
+    setupObjectStore(rdbms, roles, dbNames, tokenIds, tokens, masterKeys, now);
+
+    int baseNumRoles = store.listRoleNames() == null ? 0 : store.listRoleNames().size();
+    int baseNumDbs = store.getAllDatabases() == null ? 0 : store.getAllDatabases().size();
+    int baseNumToks = store.getAllTokenIdentifiers() == null ? 0 :
+        store.getAllTokenIdentifiers().size();
+    int baseNumKeys =  store.getMasterKeys() == null ? 0 : store.getMasterKeys().length;
+
+    // Create the database so I can put the table in it.
+    store.createDatabase(
+        new Database(dbNames[0], "no description", "file:/tmp", emptyParameters));
+
+    HBaseImport importer = new HBaseImport("-t", dbNames[0] + "." + tableNames[0]);
+    importer.setConnections(rdbms, store);
+    importer.run();
+
+    // Make sure there aren't any extra roles
+    Assert.assertEquals(baseNumRoles, store.listRoleNames().size());
+
+    Database db = store.getDatabase(dbNames[0]);
+    Assert.assertNotNull(db);
+
+    Table table = store.getTable(db.getName(), tableNames[0]);
+    Assert.assertNotNull(table);
+    Assert.assertEquals(1, store.getAllTables(db.getName()).size());
+    Assert.assertNull(store.getTable(db.getName(), tableNames[1]));
+
+    Assert.assertEquals(0, store.getFunctions(dbNames[0], "*").size());
+    Assert.assertEquals(baseNumDbs + 1, store.getAllDatabases().size());
+
+    Assert.assertEquals(baseNumToks, store.getAllTokenIdentifiers().size());
+    String[] hbaseKeys = store.getMasterKeys();
+    Assert.assertEquals(baseNumKeys, hbaseKeys.length);
+
+  }
+
+  @Test
+  public void importOneTablePartitioned() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+
+    String[] dbNames = new String[] {"onetabpartdb1", "onetabpartodb2"};
+    String[] roles = new String[] {"onetabpartorole1", "onetabpartorole2"};
+    String[] tokenIds = new String[] {"onetabpartotokenid1", "onetabpartotokenid2"};
+    String[] tokens = new String[] {"onetabpartotoken1", "onetabpartotoken2"};
+    String[] masterKeys = new String[] {"onetabpartomk1", "onetabpartomk2"};
+    int now = (int)System.currentTimeMillis() / 1000;
+
+    setupObjectStore(rdbms, roles, dbNames, tokenIds, tokens, masterKeys, now);
+
+    int baseNumRoles = store.listRoleNames() == null ? 0 : store.listRoleNames().size();
+    int baseNumDbs = store.getAllDatabases() == null ? 0 : store.getAllDatabases().size();
+    int baseNumToks = store.getAllTokenIdentifiers() == null ? 0 :
+        store.getAllTokenIdentifiers().size();
+    int baseNumKeys =  store.getMasterKeys() == null ? 0 : store.getMasterKeys().length;
+
+    // Create the database so I can put the table in it.
+    store.createDatabase(
+        new Database(dbNames[0], "no description", "file:/tmp", emptyParameters));
+
+    HBaseImport importer = new HBaseImport("-t", dbNames[0] + "." + tableNames[1]);
+    importer.setConnections(rdbms, store);
+    importer.run();
+
+    // Make sure there aren't any extra roles
+    Assert.assertEquals(baseNumRoles, store.listRoleNames().size());
+
+    Database db = store.getDatabase(dbNames[0]);
+    Assert.assertNotNull(db);
+
+    Table table = store.getTable(db.getName(), tableNames[1]);
+    Assert.assertNotNull(table);
+    Assert.assertEquals(1, store.getAllTables(db.getName()).size());
+
+    for (int j = 0; j < partVals.length; j++) {
+      Partition part = store.getPartition(dbNames[0], tableNames[1], Arrays.asList(partVals[j]));
+      Assert.assertNotNull(part);
+      Assert.assertEquals("file:/tmp/region=" + partVals[j], part.getSd().getLocation());
+    }
+    Assert.assertEquals(4, store.getPartitions(dbNames[0], tableNames[1], -1).size());
+
+    Assert.assertNull(store.getTable(db.getName(), tableNames[0]));
+
+    Assert.assertEquals(0, store.getFunctions(dbNames[0], "*").size());
+    Assert.assertEquals(baseNumDbs + 1, store.getAllDatabases().size());
+
+    Assert.assertEquals(baseNumToks, store.getAllTokenIdentifiers().size());
+    String[] hbaseKeys = store.getMasterKeys();
+    Assert.assertEquals(baseNumKeys, hbaseKeys.length);
+  }
+
+  @Test
+  public void importSecurity() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+
+    String[] dbNames = new String[] {"securitydb1", "securitydb2"};
+    String[] roles = new String[] {"securityrole1", "securityrole2"};
+    String[] tokenIds = new String[] {"securitytokenid1", "securitytokenid2"};
+    String[] tokens = new String[] {"securitytoken1", "securitytoken2"};
+    String[] masterKeys = new String[] {"securitymk1", "securitymk2"};
+    int now = (int)System.currentTimeMillis() / 1000;
+
+    setupObjectStore(rdbms, roles, dbNames, tokenIds, tokens, masterKeys, now);
+
+    int baseNumRoles = store.listRoleNames() == null ? 0 : store.listRoleNames().size();
+    int baseNumDbs = store.getAllDatabases() == null ? 0 : store.getAllDatabases().size();
+
+    HBaseImport importer = new HBaseImport("-k");
+    importer.setConnections(rdbms, store);
+    importer.run();
+
+    Assert.assertEquals(baseNumRoles, store.listRoleNames().size());
+
+    Assert.assertEquals(baseNumDbs, store.getAllDatabases().size());
+
+    // I can't test total number of tokens or master keys because the import grabs all and copies
+    // them, which means it grabs the ones imported by importAll test (if it's already run).
+    // Depending on it already running would make the tests order dependent, which junit doesn't
+    // guarantee.
+    for (int i = 0; i < tokenIds.length; i++) {
+      Assert.assertEquals(tokens[i], store.getToken(tokenIds[i]));
+    }
+    String[] hbaseKeys = store.getMasterKeys();
+    Set<String> keys = new HashSet<>(Arrays.asList(hbaseKeys));
+    for (int i = 0; i < masterKeys.length; i++) {
+      Assert.assertTrue(keys.contains(masterKeys[i]));
+    }
+  }
+
+  // TODO test for bogus function name
+  // TODO test for bogus table name
+  // TODO test for non-existent items
+
+  @Test
+  public void importOneRole() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
     rdbms.setConf(conf);
 
-    String[] dbNames = new String[] {"importdb1", "importdb2"};
-    String[] tableNames = new String[] {"nonparttable", "parttable"};
-    String[] partVals = new String[] {"na", "emea", "latam", "apac"};
-    String[] funcNames = new String[] {"func1", "func2"};
-    String[] roles = new String[] {"role1", "role2"};
+    String[] dbNames = new String[] {"oneroledb1", "oneroledb2"};
+    String[] roles = new String[] {"onerolerole1", "onerolerole2"};
+    String[] tokenIds = new String[] {"oneroletokenid1", "oneroletokenid2"};
+    String[] tokens = new String[] {"oneroletoken1", "oneroletoken2"};
+    String[] masterKeys = new String[] {"onerolemk1", "onerolemk2"};
     int now = (int)System.currentTimeMillis() / 1000;
 
+    setupObjectStore(rdbms, roles, dbNames, tokenIds, tokens, masterKeys, now);
+
+    int baseNumRoles = store.listRoleNames() == null ? 0 : store.listRoleNames().size();
+    int baseNumDbs = store.getAllDatabases() == null ? 0 : store.getAllDatabases().size();
+    int baseNumToks = store.getAllTokenIdentifiers() == null ? 0 :
+        store.getAllTokenIdentifiers().size();
+    int baseNumKeys =  store.getMasterKeys() == null ? 0 : store.getMasterKeys().length;
+
+    HBaseImport importer = new HBaseImport("-r", roles[0]);
+    importer.setConnections(rdbms, store);
+    importer.run();
+
+    Role role = store.getRole(roles[0]);
+    Assert.assertNotNull(role);
+    Assert.assertEquals(roles[0], role.getRoleName());
+
+    // Make sure there aren't any extra roles
+    Assert.assertEquals(baseNumRoles + 1, store.listRoleNames().size());
+    Assert.assertEquals(baseNumDbs, store.getAllDatabases().size());
+
+    Assert.assertEquals(baseNumToks, store.getAllTokenIdentifiers().size());
+    String[] hbaseKeys = store.getMasterKeys();
+    Assert.assertEquals(baseNumKeys, hbaseKeys.length);
+
+    // Have to do this last as it will throw an exception
+    thrown.expect(NoSuchObjectException.class);
+    store.getRole(roles[1]);
+  }
+
+  private void setupObjectStore(RawStore rdbms, String[] roles, String[] dbNames,
+                                String[] tokenIds, String[] tokens, String[] masterKeys, int now)
+      throws MetaException, InvalidObjectException, NoSuchObjectException {
     for (int i = 0; i < roles.length; i++) {
       rdbms.addRole(roles[i], "me");
     }
 
     for (int i = 0; i < dbNames.length; i++) {
-      rdbms.createDatabase(new Database(dbNames[i], "no description", "file:/tmp", emptyParameters));
+      rdbms.createDatabase(
+          new Database(dbNames[i], "no description", "file:/tmp", emptyParameters));
 
-      List<FieldSchema> cols = new ArrayList<FieldSchema>();
+      List<FieldSchema> cols = new ArrayList<>();
       cols.add(new FieldSchema("col1", "int", "nocomment"));
       SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
       StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
@@ -95,7 +480,7 @@ public class TestHBaseImport extends HBaseIntegrationTests {
       rdbms.createTable(new Table(tableNames[0], dbNames[i], "me", now, now, 0, sd, null,
           emptyParameters, null, null, null));
 
-      List<FieldSchema> partCols = new ArrayList<FieldSchema>();
+      List<FieldSchema> partCols = new ArrayList<>();
       partCols.add(new FieldSchema("region", "string", ""));
       rdbms.createTable(new Table(tableNames[1], dbNames[i], "me", now, now, 0, sd, partCols,
           emptyParameters, null, null, null));
@@ -105,57 +490,149 @@ public class TestHBaseImport extends HBaseIntegrationTests {
         psd.setLocation("file:/tmp/region=" + partVals[j]);
         Partition part = new Partition(Arrays.asList(partVals[j]), dbNames[i], tableNames[1],
             now, now, psd, emptyParameters);
-        store.addPartition(part);
+        rdbms.addPartition(part);
       }
 
       for (String funcName : funcNames) {
-        store.createFunction(new Function(funcName, dbNames[i], "classname", "ownername",
-            PrincipalType.USER, (int)System.currentTimeMillis()/1000, FunctionType.JAVA,
+        LOG.debug("Creating new function " + dbNames[i] + "." + funcName);
+        rdbms.createFunction(new Function(funcName, dbNames[i], "classname", "ownername",
+            PrincipalType.USER, (int) System.currentTimeMillis() / 1000, FunctionType.JAVA,
             Arrays.asList(new ResourceUri(ResourceType.JAR, "uri"))));
       }
     }
+    for (int i = 0; i < tokenIds.length; i++) rdbms.addToken(tokenIds[i], tokens[i]);
+    for (int i = 0; i < masterKeys.length; i++) {
+      rdbms.addMasterKey(masterKeys[i]);
+    }
+  }
 
-    HBaseImport importer = new HBaseImport();
-    importer.setConnections(rdbms, store);
-    importer.run();
+  @Test
+  public void parallel() throws Exception {
+    int parallelFactor = 10;
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
 
-    for (int i = 0; i < roles.length; i++) {
-      Role role = store.getRole(roles[i]);
-      Assert.assertNotNull(role);
-      Assert.assertEquals(roles[i], role.getRoleName());
+    String[] dbNames = new String[] {"paralleldb1"};
+    int now = (int)System.currentTimeMillis() / 1000;
+
+    for (int i = 0; i < dbNames.length; i++) {
+      rdbms.createDatabase(
+          new Database(dbNames[i], "no description", "file:/tmp", emptyParameters));
+
+      List<FieldSchema> cols = new ArrayList<>();
+      cols.add(new FieldSchema("col1", "int", "nocomment"));
+      SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+      StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
+          serde, null, null, emptyParameters);
+
+      List<FieldSchema> partCols = new ArrayList<>();
+      partCols.add(new FieldSchema("region", "string", ""));
+      for (int j = 0; j < parallelFactor; j++) {
+        rdbms.createTable(new Table("t" + j, dbNames[i], "me", now, now, 0, sd, partCols,
+            emptyParameters, null, null, null));
+        for (int k = 0; k < parallelFactor; k++) {
+          StorageDescriptor psd = new StorageDescriptor(sd);
+          psd.setLocation("file:/tmp/region=" + k);
+          Partition part = new Partition(Arrays.asList("p" + k), dbNames[i], "t" + j,
+              now, now, psd, emptyParameters);
+          rdbms.addPartition(part);
+        }
+      }
     }
-    // Make sure there aren't any extra roles
-    Assert.assertEquals(2, store.listRoleNames().size());
+
+    HBaseImport importer = new HBaseImport("-p", "2", "-b", "2", "-d", dbNames[0]);
+    importer.setConnections(rdbms, store);
+    importer.run();
 
     for (int i = 0; i < dbNames.length; i++) {
       Database db = store.getDatabase(dbNames[i]);
       Assert.assertNotNull(db);
-      // check one random value in the db rather than every value
-      Assert.assertEquals("file:/tmp", db.getLocationUri());
 
-      Table table = store.getTable(db.getName(), tableNames[0]);
-      Assert.assertNotNull(table);
-      Assert.assertEquals(now, table.getLastAccessTime());
-      Assert.assertEquals("input", table.getSd().getInputFormat());
+      for (int j = 0; j < parallelFactor; j++) {
+        Table table = store.getTable(db.getName(), "t" + j);
+        Assert.assertNotNull(table);
+        Assert.assertEquals(now, table.getLastAccessTime());
+        Assert.assertEquals("input", table.getSd().getInputFormat());
 
-      table = store.getTable(db.getName(), tableNames[1]);
-      Assert.assertNotNull(table);
+        for (int k = 0; k < parallelFactor; k++) {
+          Partition part =
+              store.getPartition(dbNames[i], "t" + j, Arrays.asList("p" + k));
+          Assert.assertNotNull(part);
+          Assert.assertEquals("file:/tmp/region=" + k, part.getSd().getLocation());
+        }
 
-      for (int j = 0; j < partVals.length; j++) {
-        Partition part = store.getPartition(dbNames[i], tableNames[1], Arrays.asList(partVals[j]));
-        Assert.assertNotNull(part);
-        Assert.assertEquals("file:/tmp/region=" + partVals[j], part.getSd().getLocation());
+        Assert.assertEquals(parallelFactor, store.getPartitions(dbNames[i], "t" + j, -1).size());
       }
+      Assert.assertEquals(parallelFactor, store.getAllTables(dbNames[i]).size());
 
-      Assert.assertEquals(4, store.getPartitions(dbNames[i], tableNames[1], -1).size());
-      Assert.assertEquals(2, store.getAllTables(dbNames[i]).size());
+    }
+  }
 
-      Assert.assertEquals(2, store.getFunctions(dbNames[i], "*").size());
-      for (int j = 0; j < funcNames.length; j++) {
-        Assert.assertNotNull(store.getFunction(dbNames[i], funcNames[j]));
+  // Same as the test above except we create 9 of everything instead of 10.  This is important
+  // because in using a batch size of 2 the previous test guarantees 10 /2 =5 , meaning we'll
+  // have 5 writes on the partition queue with exactly 2 entries.  In this test we'll handle the
+  // case where the last entry in the queue has fewer partitions.
+  @Test
+  public void parallelOdd() throws Exception {
+    int parallelFactor = 9;
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+
+    String[] dbNames = new String[] {"oddparalleldb1"};
+    int now = (int)System.currentTimeMillis() / 1000;
+
+    for (int i = 0; i < dbNames.length; i++) {
+      rdbms.createDatabase(
+          new Database(dbNames[i], "no description", "file:/tmp", emptyParameters));
+
+      List<FieldSchema> cols = new ArrayList<>();
+      cols.add(new FieldSchema("col1", "int", "nocomment"));
+      SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+      StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
+          serde, null, null, emptyParameters);
+
+      List<FieldSchema> partCols = new ArrayList<>();
+      partCols.add(new FieldSchema("region", "string", ""));
+      for (int j = 0; j < parallelFactor; j++) {
+        rdbms.createTable(new Table("t" + j, dbNames[i], "me", now, now, 0, sd, partCols,
+            emptyParameters, null, null, null));
+        for (int k = 0; k < parallelFactor; k++) {
+          StorageDescriptor psd = new StorageDescriptor(sd);
+          psd.setLocation("file:/tmp/region=" + k);
+          Partition part = new Partition(Arrays.asList("p" + k), dbNames[i], "t" + j,
+              now, now, psd, emptyParameters);
+          rdbms.addPartition(part);
+        }
       }
     }
 
-    Assert.assertEquals(2, store.getAllDatabases().size());
+    HBaseImport importer = new HBaseImport("-p", "2", "-b", "2", "-d", dbNames[0]);
+    importer.setConnections(rdbms, store);
+    importer.run();
+
+    for (int i = 0; i < dbNames.length; i++) {
+      Database db = store.getDatabase(dbNames[i]);
+      Assert.assertNotNull(db);
+
+      for (int j = 0; j < parallelFactor; j++) {
+        Table table = store.getTable(db.getName(), "t" + j);
+        Assert.assertNotNull(table);
+        Assert.assertEquals(now, table.getLastAccessTime());
+        Assert.assertEquals("input", table.getSd().getInputFormat());
+
+        for (int k = 0; k < parallelFactor; k++) {
+          Partition part =
+              store.getPartition(dbNames[i], "t" + j, Arrays.asList("p" + k));
+          Assert.assertNotNull(part);
+          Assert.assertEquals("file:/tmp/region=" + k, part.getSd().getLocation());
+        }
+
+        Assert.assertEquals(parallelFactor, store.getPartitions(dbNames[i], "t" + j, -1).size());
+      }
+      Assert.assertEquals(parallelFactor, store.getAllTables(dbNames[i]).size());
+
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/0fa45e4a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java
index e416b8a..e143de7 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java
@@ -19,11 +19,18 @@
 package org.apache.hadoop.hive.metastore.hbase;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.Deadline;
 import org.apache.hadoop.hive.metastore.ObjectStore;
 import org.apache.hadoop.hive.metastore.RawStore;
 import org.apache.hadoop.hive.metastore.api.Database;
@@ -36,23 +43,41 @@ import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.Table;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 /**
  * A tool to take the contents of an RDBMS based Hive metastore and import it into an HBase based
  * one.  To use this the config files for Hive configured to work with the RDBMS (that is,
- * including the JDBC string, etc.) and for HBase must be in the path.  This tool will then
+ * including the JDBC string, etc.) as well as HBase configuration files must be in the path.
+ * There should not be a hive-site.xml that specifies HBaseStore in the path.  This tool will then
  * handle connecting to the RDBMS via the {@link org.apache.hadoop.hive.metastore.ObjectStore}
  * and HBase via {@link org.apache.hadoop.hive.metastore.hbase.HBaseStore} and transferring the
  * data.
+ *
+ * This tool can import an entire metastore or only selected objects.  When selecting objects it
+ * is necessary to fully specify the object's name.  For example, if you want to import the table
+ * T in the default database it needs to be identified as default.T.  The same is true for
+ * functions.  When an object is specified, everything under that object will be imported (e.g.
+ * if you select database D, then all tables and functions in that database will be
+ * imported as well).
+ *
+ * At this point only tables and partitions are handled in parallel as it is assumed there are
+ * relatively few of everything else.
+ *
+ * Note that HBaseSchemaTool must have already been used to create the appropriate tables in HBase.
  */
 public class HBaseImport {
 
   static final private Log LOG = LogFactory.getLog(HBaseImport.class.getName());
 
   public static void main(String[] args) {
-    HBaseImport tool = new HBaseImport();
     try {
+      HBaseImport tool = new HBaseImport(args);
       tool.run();
     } catch (Exception e) {
       System.err.println("Caught exception " + e.getClass().getName() + " with message <" +
@@ -60,112 +85,399 @@ public class HBaseImport {
     }
   }
 
+  private ThreadLocal<RawStore> rdbmsStore = new ThreadLocal<RawStore>() {
+    @Override
+    protected RawStore initialValue() {
+      if (rdbmsConf == null) {
+        throw new RuntimeException("order violation, need to set rdbms conf first");
+      }
+      RawStore os = new ObjectStore();
+      os.setConf(rdbmsConf);
+      return os;
+    }
+  };
+
+  private ThreadLocal<RawStore> hbaseStore = new ThreadLocal<RawStore>() {
+    @Override
+    protected RawStore initialValue() {
+      if (hbaseConf == null) {
+        throw new RuntimeException("order violation, need to set hbase conf first");
+      }
+      RawStore hs = new HBaseStore();
+      hs.setConf(hbaseConf);
+      return hs;
+    }
+  };
+
   private Configuration rdbmsConf;
   private Configuration hbaseConf;
-  private RawStore rdbmsStore;
-  private RawStore hbaseStore;
   private List<Database> dbs;
-  private List<Table> tables;
+  private BlockingQueue<Table> partitionedTables;
+  private BlockingQueue<String[]> tableNameQueue;
+  private BlockingQueue<PartQueueEntry> partQueue;
+  private boolean writingToQueue, readersFinished;
+  private boolean doKerberos, doAll;
+  private List<String> rolesToImport, dbsToImport, tablesToImport, functionsToImport;
+  private int parallel;
+  private int batchSize;
 
   @VisibleForTesting
-  HBaseImport() {
-    dbs = new ArrayList<Database>();
-    tables = new ArrayList<Table>();
+  HBaseImport(String... args) throws ParseException {
+    Options options = new Options();
+
+    doAll = doKerberos = false;
+    parallel = 1;
+    batchSize = 1000;
+
+    options.addOption(OptionBuilder
+        .withLongOpt("all")
+        .withDescription("Import the full metastore")
+        .create('a'));
+
+    options.addOption(OptionBuilder
+        .withLongOpt("batchsize")
+        .withDescription("Number of partitions to read and write in a batch, defaults to 1000")
+            .hasArg()
+            .create('b'));
+
+    options.addOption(OptionBuilder
+        .withLongOpt("database")
+        .withDescription("Import a single database")
+        .hasArgs()
+        .create('d'));
+
+    options.addOption(OptionBuilder
+        .withLongOpt("help")
+        .withDescription("You're looking at it")
+        .create('h'));
+
+    options.addOption(OptionBuilder
+        .withLongOpt("function")
+        .withDescription("Import a single function")
+        .hasArgs()
+        .create('f'));
+
+    options.addOption(OptionBuilder
+        .withLongOpt("kerberos")
+        .withDescription("Import all kerberos related objects (master key, tokens)")
+        .create('k'));
+
+     options.addOption(OptionBuilder
+        .withLongOpt("parallel")
+        .withDescription("Parallel factor for loading (only applied to tables and partitions), " +
+            "defaults to 1")
+        .hasArg()
+        .create('p'));
+
+    options.addOption(OptionBuilder
+        .withLongOpt("role")
+        .withDescription("Import a single role")
+        .hasArgs()
+        .create('r'));
 
+   options.addOption(OptionBuilder
+        .withLongOpt("tables")
+        .withDescription("Import a single tables")
+        .hasArgs()
+        .create('t'));
+
+    CommandLine cli = new GnuParser().parse(options, args);
+
+    // Process help, if it was asked for, this must be done first
+    if (cli.hasOption('h')) {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp("hbaseschematool", options);
+      // returning here results in nothing else happening, because none of the other flags have
+      // been set.
+      return;
+    }
+
+    // Now process the other command line args
+    if (cli.hasOption('a')) {
+      doAll = true;
+    }
+    if (cli.hasOption('b')) {
+      batchSize = Integer.valueOf(cli.getOptionValue('b'));
+    }
+    if (cli.hasOption('d')) {
+      dbsToImport = Arrays.asList(cli.getOptionValues('d'));
+    }
+    if (cli.hasOption('f')) {
+      functionsToImport = Arrays.asList(cli.getOptionValues('f'));
+    }
+    if (cli.hasOption('p')) {
+      parallel = Integer.valueOf(cli.getOptionValue('p'));
+    }
+    if (cli.hasOption('r')) {
+      rolesToImport = Arrays.asList(cli.getOptionValues('r'));
+    }
+    if (cli.hasOption('k')) {
+      doKerberos = true;
+    }
+    if (cli.hasOption('t')) {
+      tablesToImport = Arrays.asList(cli.getOptionValues('t'));
+    }
+
+    dbs = new ArrayList<>();
+    // We don't want to bound the size of the table queue because we keep it all in memory
+    partitionedTables = new LinkedBlockingQueue<>();
+    tableNameQueue = new LinkedBlockingQueue<>();
+
+    // Bound the size of this queue so we don't get too much in memory.
+    partQueue = new ArrayBlockingQueue<>(parallel * 2);
   }
 
   @VisibleForTesting
   void run() throws MetaException, InstantiationException, IllegalAccessException,
-      NoSuchObjectException, InvalidObjectException {
+      NoSuchObjectException, InvalidObjectException, InterruptedException {
     // Order here is crucial, as you can't add tables until you've added databases, etc.
     init();
-    copyRoles();
-    copyDbs();
-    copyTables();
-    copyPartitions();
-    copyFunctions();
+    if (doAll || rolesToImport != null) {
+      copyRoles();
+    }
+    if (doAll || dbsToImport != null) {
+      copyDbs();
+    }
+    if (doAll || dbsToImport != null || tablesToImport != null) {
+      copyTables();
+      copyPartitions();
+    }
+    if (doAll || dbsToImport != null || functionsToImport != null) {
+      copyFunctions();
+    }
+    if (doAll || doKerberos) {
+      copyKerberos();
+    }
   }
 
   private void init() throws MetaException, IllegalAccessException, InstantiationException {
-    if (rdbmsStore != null) {
+    if (rdbmsConf != null) {
       // We've been configured for testing, so don't do anything here.
       return;
     }
-    rdbmsConf = new HiveConf();  // We're depending on having everything properly in the path
-    hbaseConf = new HiveConf();
+    // We're depending on having everything properly in the path
+    rdbmsConf = new HiveConf();
+    hbaseConf = new HiveConf();//
     HiveConf.setVar(hbaseConf, HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
         HBaseStore.class.getName());
     HiveConf.setBoolVar(hbaseConf, HiveConf.ConfVars.METASTORE_FASTPATH, true);
 
     // First get a connection to the RDBMS based store
-    rdbmsStore = new ObjectStore();
-    rdbmsStore.setConf(rdbmsConf);
+    rdbmsStore.get().setConf(rdbmsConf);
 
     // Get a connection to the HBase based store
-    hbaseStore = new HBaseStore();
-    hbaseStore.setConf(hbaseConf);
-    // This will go create the tables if they don't exist
-    hbaseStore.verifySchema();
+    hbaseStore.get().setConf(hbaseConf);
   }
 
   private void copyRoles() throws NoSuchObjectException, InvalidObjectException, MetaException {
     screen("Copying roles");
-    for (String roleName : rdbmsStore.listRoleNames()) {
-      Role role = rdbmsStore.getRole(roleName);
+    List<String> toCopy = doAll ? rdbmsStore.get().listRoleNames() : rolesToImport;
+    for (String roleName : toCopy) {
+      Role role = rdbmsStore.get().getRole(roleName);
       screen("Copying role " + roleName);
-      hbaseStore.addRole(roleName, role.getOwnerName());
+      hbaseStore.get().addRole(roleName, role.getOwnerName());
     }
   }
 
   private void copyDbs() throws MetaException, NoSuchObjectException, InvalidObjectException {
     screen("Copying databases");
-    for (String dbName : rdbmsStore.getAllDatabases()) {
-      Database db = rdbmsStore.getDatabase(dbName);
+    List<String> toCopy = doAll ? rdbmsStore.get().getAllDatabases() : dbsToImport;
+    for (String dbName : toCopy) {
+      Database db = rdbmsStore.get().getDatabase(dbName);
       dbs.add(db);
       screen("Copying database " + dbName);
-      hbaseStore.createDatabase(db);
+      hbaseStore.get().createDatabase(db);
     }
   }
 
-  private void copyTables() throws MetaException, InvalidObjectException {
+  private void copyTables() throws MetaException, InvalidObjectException, InterruptedException {
     screen("Copying tables");
+
+    // Start the parallel threads that will copy the tables
+    Thread[] copiers = new Thread[parallel];
+    writingToQueue = true;
+    for (int i = 0; i < parallel; i++) {
+      copiers[i] = new TableCopier();
+      copiers[i].start();
+    }
+
+    // Put tables from the databases we copied into the queue
     for (Database db : dbs) {
       screen("Coyping tables in database " + db.getName());
-      for (String tableName : rdbmsStore.getAllTables(db.getName())) {
-        Table table = rdbmsStore.getTable(db.getName(), tableName);
-        tables.add(table);
-        screen("Copying table " + db.getName() + "." + tableName);
-        hbaseStore.createTable(table);
+      for (String tableName : rdbmsStore.get().getAllTables(db.getName())) {
+        tableNameQueue.put(new String[]{db.getName(), tableName});
+      }
+    }
+
+    // Now put any specifically requested tables into the queue
+    if (tablesToImport != null) {
+      for (String compoundTableName : tablesToImport) {
+        String[] tn = compoundTableName.split("\\.");
+        if (tn.length != 2) {
+          error(compoundTableName + " not in proper form.  Must be in form dbname.tablename.  " +
+              "Ignoring this table and continuing.");
+        } else {
+          tableNameQueue.put(new String[]{tn[0], tn[1]});
+        }
+      }
+    }
+    writingToQueue = false;
+
+    // Wait until we've finished adding all the tables
+    for (Thread copier : copiers) copier.join();
+ }
+
+  private class TableCopier extends Thread {
+    @Override
+    public void run() {
+      while (writingToQueue || tableNameQueue.size() > 0) {
+        try {
+          String[] name = tableNameQueue.poll(1, TimeUnit.SECONDS);
+          if (name != null) {
+            Table table = rdbmsStore.get().getTable(name[0], name[1]);
+            // If this has partitions, put it in the list to fetch partions for
+            if (table.getPartitionKeys() != null && table.getPartitionKeys().size() > 0) {
+              partitionedTables.put(table);
+            }
+            screen("Copying table " + name[0] + "." + name[1]);
+            hbaseStore.get().createTable(table);
+          }
+        } catch (InterruptedException | MetaException | InvalidObjectException e) {
+          throw new RuntimeException(e);
+        }
       }
     }
   }
 
+  /* Partition copying is a little complex.  As we went through and copied the tables we put each
+   * partitioned table into a queue.  We will now go through that queue and add partitions for the
+   * tables.  We do the finding of partitions and writing of them separately and in parallel.
+   * This way if there is one table with >> partitions then all of the others that skew won't
+   * hurt us.  To avoid pulling all of the partitions for a table into memory, we batch up
+   * partitions (by default in batches of 1000) and copy them over in batches.
+   */
   private void copyPartitions() throws MetaException, NoSuchObjectException,
-      InvalidObjectException {
+      InvalidObjectException, InterruptedException {
     screen("Copying partitions");
-    for (Table table : tables) {
-      System.out.print("Copying partitions for table " + table.getDbName() + "." +
-          table.getTableName());
-      for (Partition part : rdbmsStore.getPartitions(table.getDbName(), table.getTableName(), -1)) {
-        LOG.info("Copying " + table.getTableName() + "." + table.getTableName() + "." +
-            StringUtils.join(part.getValues(), ':'));
-        System.out.print('.');
-        hbaseStore.addPartition(part);
+    readersFinished = false;
+    Thread[] readers = new Thread[parallel];
+    Thread[] writers = new Thread[parallel];
+    for (int i = 0; i < parallel; i++) {
+      readers[i] = new PartitionReader();
+      readers[i].start();
+      writers[i] = new PartitionWriter();
+      writers[i].start();
+    }
+
+    for (Thread reader : readers) reader.join();
+    readersFinished = true;
+
+    // Wait until we've finished adding all the partitions
+    for (Thread writer : writers) writer.join();
+  }
+
+  private class PartitionReader extends Thread {
+    @Override
+    public void run() {
+      while (partitionedTables.size() > 0) {
+        try {
+          Table table = partitionedTables.poll(1, TimeUnit.SECONDS);
+          if (table != null) {
+            screen("Fetching partitions for table " + table.getDbName() + "." +
+                table.getTableName());
+            List<String> partNames =
+                rdbmsStore.get().listPartitionNames(table.getDbName(), table.getTableName(),
+                    (short) -1);
+            if (partNames.size() <= batchSize) {
+              LOG.debug("Adding all partition names to queue for " + table.getDbName() + "." +
+                  table.getTableName());
+              partQueue.put(new PartQueueEntry(table.getDbName(), table.getTableName(), partNames));
+            } else {
+              int goUntil = partNames.size() % batchSize == 0 ? partNames.size() / batchSize :
+                  partNames.size() / batchSize + 1;
+              for (int i = 0; i < goUntil; i++) {
+                int start = i * batchSize;
+                int end = Math.min((i + 1) * batchSize, partNames.size());
+                LOG.debug("Adding partitions " + start + " to " + end + " for " + table.getDbName()
+                    + "." + table.getTableName());
+                partQueue.put(new PartQueueEntry(table.getDbName(), table.getTableName(),
+                    partNames.subList(start, end)));
+              }
+            }
+          }
+        } catch (InterruptedException | MetaException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+  }
+
+  private class PartitionWriter extends Thread {
+    @Override
+    public void run() {
+      // This keeps us from throwing exceptions in our raw store calls
+      Deadline.registerIfNot(1000000);
+      while (!readersFinished || partQueue.size() > 0) {
+        try {
+          PartQueueEntry entry = partQueue.poll(1, TimeUnit.SECONDS);
+          if (entry != null) {
+            LOG.info("Writing partitions " + entry.dbName + "." + entry.tableName + "." +
+                StringUtils.join(entry.partNames, ':'));
+            // Fetch these partitions and write them to HBase
+            Deadline.startTimer("hbaseimport");
+            List<Partition> parts =
+                rdbmsStore.get().getPartitionsByNames(entry.dbName, entry.tableName,
+                    entry.partNames);
+            hbaseStore.get().addPartitions(entry.dbName, entry.tableName, parts);
+            Deadline.stopTimer();
+          }
+        } catch (InterruptedException | MetaException | InvalidObjectException |
+            NoSuchObjectException e) {
+          throw new RuntimeException(e);
+        }
       }
-      System.out.println();
     }
   }
 
   private void copyFunctions() throws MetaException, NoSuchObjectException, InvalidObjectException {
     screen("Copying functions");
+    // Copy any functions from databases we copied.
     for (Database db : dbs) {
       screen("Copying functions in database " + db.getName());
-      for (String funcName : rdbmsStore.getFunctions(db.getName(), "*")) {
-        Function func = rdbmsStore.getFunction(db.getName(), funcName);
-        screen("Copying function " + db.getName() + "." + funcName);
-        hbaseStore.createFunction(func);
+      for (String funcName : rdbmsStore.get().getFunctions(db.getName(), "*")) {
+        copyOneFunction(db.getName(), funcName);
       }
     }
+    // Now do any specifically requested functions
+    if (functionsToImport != null) {
+      for (String compoundFuncName : functionsToImport) {
+        String[] fn = compoundFuncName.split("\\.");
+        if (fn.length != 2) {
+          error(compoundFuncName + " not in proper form.  Must be in form dbname.funcname.  " +
+              "Ignoring this function and continuing.");
+        } else {
+          copyOneFunction(fn[0], fn[1]);
+        }
+      }
+    }
+  }
+
+  private void copyOneFunction(String dbName, String funcName) throws MetaException,
+      InvalidObjectException {
+    Function func = rdbmsStore.get().getFunction(dbName, funcName);
+    screen("Copying function " + dbName + "." + funcName);
+    hbaseStore.get().createFunction(func);
+  }
+
+  private void copyKerberos() throws MetaException {
+    screen("Copying kerberos related items");
+    for (String tokenId : rdbmsStore.get().getAllTokenIdentifiers()) {
+      String token = rdbmsStore.get().getToken(tokenId);
+      hbaseStore.get().addToken(tokenId, token);
+    }
+    for (String masterKey : rdbmsStore.get().getMasterKeys()) {
+      hbaseStore.get().addMasterKey(masterKey);
+    }
   }
 
   private void screen(String msg) {
@@ -173,12 +485,29 @@ public class HBaseImport {
     System.out.println(msg);
   }
 
+  private void error(String msg) {
+    LOG.error(msg);
+    System.err.println("ERROR:  " + msg);
+  }
+
   @VisibleForTesting
-  HBaseImport setConnections(RawStore rdbms, RawStore hbase) {
-    rdbmsStore = rdbms;
-    hbaseStore = hbase;
+  void setConnections(RawStore rdbms, RawStore hbase) {
+    rdbmsStore.set(rdbms);
+    hbaseStore.set(hbase);
+    rdbmsConf = rdbms.getConf();
+    hbaseConf = hbase.getConf();
+  }
 
-    return new HBaseImport();
+  private static class PartQueueEntry {
+    final String dbName;
+    final String tableName;
+    final List<String> partNames;
+
+    PartQueueEntry(String d, String t, List<String> p) {
+      dbName = d;
+      tableName = t;
+      partNames = p;
+    }
   }
 
 }


[30/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
new file mode 100644
index 0000000..874ea82
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -0,0 +1,588 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMetadataRequest, PutFileMetadataRequest._Fields>, java.io.Serializable, Cloneable, Comparable<PutFileMetadataRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PutFileMetadataRequest");
+
+  private static final org.apache.thrift.protocol.TField FILE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("fileIds", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("metadata", org.apache.thrift.protocol.TType.LIST, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new PutFileMetadataRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new PutFileMetadataRequestTupleSchemeFactory());
+  }
+
+  private List<Long> fileIds; // required
+  private List<ByteBuffer> metadata; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    FILE_IDS((short)1, "fileIds"),
+    METADATA((short)2, "metadata");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FILE_IDS
+          return FILE_IDS;
+        case 2: // METADATA
+          return METADATA;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FILE_IDS, new org.apache.thrift.meta_data.FieldMetaData("fileIds", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.METADATA, new org.apache.thrift.meta_data.FieldMetaData("metadata", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PutFileMetadataRequest.class, metaDataMap);
+  }
+
+  public PutFileMetadataRequest() {
+  }
+
+  public PutFileMetadataRequest(
+    List<Long> fileIds,
+    List<ByteBuffer> metadata)
+  {
+    this();
+    this.fileIds = fileIds;
+    this.metadata = metadata;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public PutFileMetadataRequest(PutFileMetadataRequest other) {
+    if (other.isSetFileIds()) {
+      List<Long> __this__fileIds = new ArrayList<Long>(other.fileIds);
+      this.fileIds = __this__fileIds;
+    }
+    if (other.isSetMetadata()) {
+      List<ByteBuffer> __this__metadata = new ArrayList<ByteBuffer>(other.metadata);
+      this.metadata = __this__metadata;
+    }
+  }
+
+  public PutFileMetadataRequest deepCopy() {
+    return new PutFileMetadataRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.fileIds = null;
+    this.metadata = null;
+  }
+
+  public int getFileIdsSize() {
+    return (this.fileIds == null) ? 0 : this.fileIds.size();
+  }
+
+  public java.util.Iterator<Long> getFileIdsIterator() {
+    return (this.fileIds == null) ? null : this.fileIds.iterator();
+  }
+
+  public void addToFileIds(long elem) {
+    if (this.fileIds == null) {
+      this.fileIds = new ArrayList<Long>();
+    }
+    this.fileIds.add(elem);
+  }
+
+  public List<Long> getFileIds() {
+    return this.fileIds;
+  }
+
+  public void setFileIds(List<Long> fileIds) {
+    this.fileIds = fileIds;
+  }
+
+  public void unsetFileIds() {
+    this.fileIds = null;
+  }
+
+  /** Returns true if field fileIds is set (has been assigned a value) and false otherwise */
+  public boolean isSetFileIds() {
+    return this.fileIds != null;
+  }
+
+  public void setFileIdsIsSet(boolean value) {
+    if (!value) {
+      this.fileIds = null;
+    }
+  }
+
+  public int getMetadataSize() {
+    return (this.metadata == null) ? 0 : this.metadata.size();
+  }
+
+  public java.util.Iterator<ByteBuffer> getMetadataIterator() {
+    return (this.metadata == null) ? null : this.metadata.iterator();
+  }
+
+  public void addToMetadata(ByteBuffer elem) {
+    if (this.metadata == null) {
+      this.metadata = new ArrayList<ByteBuffer>();
+    }
+    this.metadata.add(elem);
+  }
+
+  public List<ByteBuffer> getMetadata() {
+    return this.metadata;
+  }
+
+  public void setMetadata(List<ByteBuffer> metadata) {
+    this.metadata = metadata;
+  }
+
+  public void unsetMetadata() {
+    this.metadata = null;
+  }
+
+  /** Returns true if field metadata is set (has been assigned a value) and false otherwise */
+  public boolean isSetMetadata() {
+    return this.metadata != null;
+  }
+
+  public void setMetadataIsSet(boolean value) {
+    if (!value) {
+      this.metadata = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FILE_IDS:
+      if (value == null) {
+        unsetFileIds();
+      } else {
+        setFileIds((List<Long>)value);
+      }
+      break;
+
+    case METADATA:
+      if (value == null) {
+        unsetMetadata();
+      } else {
+        setMetadata((List<ByteBuffer>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FILE_IDS:
+      return getFileIds();
+
+    case METADATA:
+      return getMetadata();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case FILE_IDS:
+      return isSetFileIds();
+    case METADATA:
+      return isSetMetadata();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof PutFileMetadataRequest)
+      return this.equals((PutFileMetadataRequest)that);
+    return false;
+  }
+
+  public boolean equals(PutFileMetadataRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_fileIds = true && this.isSetFileIds();
+    boolean that_present_fileIds = true && that.isSetFileIds();
+    if (this_present_fileIds || that_present_fileIds) {
+      if (!(this_present_fileIds && that_present_fileIds))
+        return false;
+      if (!this.fileIds.equals(that.fileIds))
+        return false;
+    }
+
+    boolean this_present_metadata = true && this.isSetMetadata();
+    boolean that_present_metadata = true && that.isSetMetadata();
+    if (this_present_metadata || that_present_metadata) {
+      if (!(this_present_metadata && that_present_metadata))
+        return false;
+      if (!this.metadata.equals(that.metadata))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_fileIds = true && (isSetFileIds());
+    list.add(present_fileIds);
+    if (present_fileIds)
+      list.add(fileIds);
+
+    boolean present_metadata = true && (isSetMetadata());
+    list.add(present_metadata);
+    if (present_metadata)
+      list.add(metadata);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(PutFileMetadataRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetFileIds()).compareTo(other.isSetFileIds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFileIds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileIds, other.fileIds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetMetadata()).compareTo(other.isSetMetadata());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMetadata()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metadata, other.metadata);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("PutFileMetadataRequest(");
+    boolean first = true;
+
+    sb.append("fileIds:");
+    if (this.fileIds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fileIds);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("metadata:");
+    if (this.metadata == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.metadata);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetFileIds()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'fileIds' is unset! Struct:" + toString());
+    }
+
+    if (!isSetMetadata()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'metadata' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class PutFileMetadataRequestStandardSchemeFactory implements SchemeFactory {
+    public PutFileMetadataRequestStandardScheme getScheme() {
+      return new PutFileMetadataRequestStandardScheme();
+    }
+  }
+
+  private static class PutFileMetadataRequestStandardScheme extends StandardScheme<PutFileMetadataRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // FILE_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list568 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list568.size);
+                long _elem569;
+                for (int _i570 = 0; _i570 < _list568.size; ++_i570)
+                {
+                  _elem569 = iprot.readI64();
+                  struct.fileIds.add(_elem569);
+                }
+                iprot.readListEnd();
+              }
+              struct.setFileIdsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // METADATA
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list571 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list571.size);
+                ByteBuffer _elem572;
+                for (int _i573 = 0; _i573 < _list571.size; ++_i573)
+                {
+                  _elem572 = iprot.readBinary();
+                  struct.metadata.add(_elem572);
+                }
+                iprot.readListEnd();
+              }
+              struct.setMetadataIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.fileIds != null) {
+        oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
+          for (long _iter574 : struct.fileIds)
+          {
+            oprot.writeI64(_iter574);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.metadata != null) {
+        oprot.writeFieldBegin(METADATA_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
+          for (ByteBuffer _iter575 : struct.metadata)
+          {
+            oprot.writeBinary(_iter575);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class PutFileMetadataRequestTupleSchemeFactory implements SchemeFactory {
+    public PutFileMetadataRequestTupleScheme getScheme() {
+      return new PutFileMetadataRequestTupleScheme();
+    }
+  }
+
+  private static class PutFileMetadataRequestTupleScheme extends TupleScheme<PutFileMetadataRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.fileIds.size());
+        for (long _iter576 : struct.fileIds)
+        {
+          oprot.writeI64(_iter576);
+        }
+      }
+      {
+        oprot.writeI32(struct.metadata.size());
+        for (ByteBuffer _iter577 : struct.metadata)
+        {
+          oprot.writeBinary(_iter577);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list578 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list578.size);
+        long _elem579;
+        for (int _i580 = 0; _i580 < _list578.size; ++_i580)
+        {
+          _elem579 = iprot.readI64();
+          struct.fileIds.add(_elem579);
+        }
+      }
+      struct.setFileIdsIsSet(true);
+      {
+        org.apache.thrift.protocol.TList _list581 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list581.size);
+        ByteBuffer _elem582;
+        for (int _i583 = 0; _i583 < _list581.size; ++_i583)
+        {
+          _elem582 = iprot.readBinary();
+          struct.metadata.add(_elem582);
+        }
+      }
+      struct.setMetadataIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataResult.java
new file mode 100644
index 0000000..e478cf3
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataResult.java
@@ -0,0 +1,283 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class PutFileMetadataResult implements org.apache.thrift.TBase<PutFileMetadataResult, PutFileMetadataResult._Fields>, java.io.Serializable, Cloneable, Comparable<PutFileMetadataResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PutFileMetadataResult");
+
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new PutFileMetadataResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new PutFileMetadataResultTupleSchemeFactory());
+  }
+
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PutFileMetadataResult.class, metaDataMap);
+  }
+
+  public PutFileMetadataResult() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public PutFileMetadataResult(PutFileMetadataResult other) {
+  }
+
+  public PutFileMetadataResult deepCopy() {
+    return new PutFileMetadataResult(this);
+  }
+
+  @Override
+  public void clear() {
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof PutFileMetadataResult)
+      return this.equals((PutFileMetadataResult)that);
+    return false;
+  }
+
+  public boolean equals(PutFileMetadataResult that) {
+    if (that == null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(PutFileMetadataResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("PutFileMetadataResult(");
+    boolean first = true;
+
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class PutFileMetadataResultStandardSchemeFactory implements SchemeFactory {
+    public PutFileMetadataResultStandardScheme getScheme() {
+      return new PutFileMetadataResultStandardScheme();
+    }
+  }
+
+  private static class PutFileMetadataResultStandardScheme extends StandardScheme<PutFileMetadataResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, PutFileMetadataResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, PutFileMetadataResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class PutFileMetadataResultTupleSchemeFactory implements SchemeFactory {
+    public PutFileMetadataResultTupleScheme getScheme() {
+      return new PutFileMetadataResultTupleScheme();
+    }
+  }
+
+  private static class PutFileMetadataResultTupleScheme extends TupleScheme<PutFileMetadataResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
index cffcf91..a94ce18 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ResourceUri implements org.apache.thrift.TBase<ResourceUri, ResourceUri._Fields>, java.io.Serializable, Cloneable, Comparable<ResourceUri> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ResourceUri");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java
index b9052a3..8f38145 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Role implements org.apache.thrift.TBase<Role, Role._Fields>, java.io.Serializable, Cloneable, Comparable<Role> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Role");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java
index 6e3c200..e763bdd 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class RolePrincipalGrant implements org.apache.thrift.TBase<RolePrincipalGrant, RolePrincipalGrant._Fields>, java.io.Serializable, Cloneable, Comparable<RolePrincipalGrant> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RolePrincipalGrant");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
index 85af5c5..ede0cb4 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Schema implements org.apache.thrift.TBase<Schema, Schema._Fields>, java.io.Serializable, Cloneable, Comparable<Schema> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Schema");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
index 73853a2..15af1db 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class SerDeInfo implements org.apache.thrift.TBase<SerDeInfo, SerDeInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SerDeInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SerDeInfo");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
index d09e413..ac9420f 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class SetPartitionsStatsRequest implements org.apache.thrift.TBase<SetPartitionsStatsRequest, SetPartitionsStatsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<SetPartitionsStatsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SetPartitionsStatsRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java
index 7d4d12c..0b4e754 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ShowCompactRequest implements org.apache.thrift.TBase<ShowCompactRequest, ShowCompactRequest._Fields>, java.io.Serializable, Cloneable, Comparable<ShowCompactRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index 7112f26..0c98dc4 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactResponse, ShowCompactResponse._Fields>, java.io.Serializable, Cloneable, Comparable<ShowCompactResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
index 810b140..1a5926e 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowCompactResponseElement, ShowCompactResponseElement._Fields>, java.io.Serializable, Cloneable, Comparable<ShowCompactResponseElement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactResponseElement");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java
index c13fda4..adf9350 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ShowLocksRequest implements org.apache.thrift.TBase<ShowLocksRequest, ShowLocksRequest._Fields>, java.io.Serializable, Cloneable, Comparable<ShowLocksRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index 2289195..6e577d5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksResponse, ShowLocksResponse._Fields>, java.io.Serializable, Cloneable, Comparable<ShowLocksResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java
index ba17ea7..80367ac 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ShowLocksResponseElement implements org.apache.thrift.TBase<ShowLocksResponseElement, ShowLocksResponseElement._Fields>, java.io.Serializable, Cloneable, Comparable<ShowLocksResponseElement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksResponseElement");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
index 5cb5e2b..b4fa97a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class SkewedInfo implements org.apache.thrift.TBase<SkewedInfo, SkewedInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SkewedInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SkewedInfo");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
index 6b5cf87..3759f9d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class StorageDescriptor implements org.apache.thrift.TBase<StorageDescriptor, StorageDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<StorageDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StorageDescriptor");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java
index 2160bc8..0bab26b 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class StringColumnStatsData implements org.apache.thrift.TBase<StringColumnStatsData, StringColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<StringColumnStatsData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StringColumnStatsData");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
index ca16924..f11e6aa 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, java.io.Serializable, Cloneable, Comparable<Table> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Table");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
index 11d3b03..c1092e2 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequest, TableStatsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<TableStatsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TableStatsRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
index f1104e1..d0577cf 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResult, TableStatsResult._Fields>, java.io.Serializable, Cloneable, Comparable<TableStatsResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TableStatsResult");
 


[50/52] [abbrv] hive git commit: HIVE-11783: Extending HPL/SQL parser (Dmitry Tolpeko reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-11783: Extending HPL/SQL parser (Dmitry Tolpeko reviewed by Alan Gates)


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

Branch: refs/heads/llap
Commit: 06790789bddb35f60706071d8d3682d434fa05dd
Parents: d51c62a
Author: Dmitry Tolpeko <dm...@gmail.com>
Authored: Tue Sep 22 06:38:06 2015 -0700
Committer: Dmitry Tolpeko <dm...@gmail.com>
Committed: Tue Sep 22 06:38:06 2015 -0700

----------------------------------------------------------------------
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |  70 ++--
 .../main/java/org/apache/hive/hplsql/Cmp.java   | 314 ++++++++++++++++++
 .../java/org/apache/hive/hplsql/Column.java     |  29 +-
 .../main/java/org/apache/hive/hplsql/Conn.java  |  21 ++
 .../main/java/org/apache/hive/hplsql/Copy.java  |  50 ++-
 .../main/java/org/apache/hive/hplsql/Exec.java  |  66 +++-
 .../java/org/apache/hive/hplsql/Expression.java |  33 +-
 .../main/java/org/apache/hive/hplsql/File.java  |  18 +-
 .../main/java/org/apache/hive/hplsql/Meta.java  |  28 +-
 .../main/java/org/apache/hive/hplsql/Query.java |  18 ++
 .../java/org/apache/hive/hplsql/Select.java     |  23 +-
 .../main/java/org/apache/hive/hplsql/Stmt.java  |   8 +-
 .../main/java/org/apache/hive/hplsql/Var.java   | 110 ++++++-
 .../apache/hive/hplsql/functions/Function.java  |   6 +-
 .../hive/hplsql/functions/FunctionMisc.java     | 121 +++++++
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |  18 ++
 .../apache/hive/hplsql/TestHplsqlOffline.java   |   5 +
 hplsql/src/test/queries/db/cmp_row_count.sql    |   4 +
 hplsql/src/test/queries/db/cmp_sum.sql          |   3 +
 hplsql/src/test/queries/db/copy_to_file.sql     |   2 +
 hplsql/src/test/queries/db/copy_to_hdfs.sql     |   2 +
 hplsql/src/test/queries/db/copy_to_table.sql    |   2 +
 hplsql/src/test/queries/db/part_count.sql       |  17 +
 hplsql/src/test/queries/db/part_count_by.sql    |   4 +
 hplsql/src/test/queries/db/schema.sql           |  32 ++
 hplsql/src/test/queries/db/select_into.sql      |  20 +-
 hplsql/src/test/queries/db/select_into2.sql     |  17 +
 .../test/queries/local/create_procedure2.sql    |  16 +
 hplsql/src/test/queries/local/if2.sql           |   5 +
 hplsql/src/test/queries/local/include.sql       |   2 +
 hplsql/src/test/queries/local/include_file.sql  |   1 +
 hplsql/src/test/queries/local/mult_div.sql      |   8 +
 hplsql/src/test/queries/offline/select_db2.sql  |   5 +
 .../src/test/results/db/cmp_row_count.out.txt   |  12 +
 hplsql/src/test/results/db/cmp_sum.out.txt      | 320 +++++++++++++++++++
 hplsql/src/test/results/db/copy_to_file.out.txt |   6 +
 hplsql/src/test/results/db/copy_to_hdfs.out.txt |   4 +
 .../src/test/results/db/copy_to_table.out.txt   |   2 +
 hplsql/src/test/results/db/part_count.out.txt   |  15 +
 .../src/test/results/db/part_count_by.out.txt   |  13 +
 hplsql/src/test/results/db/select_into.out.txt  |  58 +++-
 hplsql/src/test/results/db/select_into2.out.txt |  19 ++
 .../results/local/create_procedure2.out.txt     |  10 +
 hplsql/src/test/results/local/if2.out.txt       |   4 +
 hplsql/src/test/results/local/include.out.txt   |   8 +
 hplsql/src/test/results/local/mult_div.out.txt  |   7 +
 .../src/test/results/offline/select_db2.out.txt |   6 +
 47 files changed, 1471 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
----------------------------------------------------------------------
diff --git a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
index ff772fe..bbe7276 100644
--- a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
+++ b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
@@ -43,6 +43,7 @@ stmt :
      | break_stmt
      | call_stmt
      | close_stmt
+     | cmp_stmt
      | copy_from_local_stmt
      | copy_stmt
      | commit_stmt
@@ -183,7 +184,7 @@ declare_handler_item :     // Condition handler declaration
      ;
      
 declare_temporary_table_item :     // DECLARE TEMPORARY TABLE statement
-       T_GLOBAL? T_TEMPORARY T_TABLE ident T_OPEN_P create_table_columns T_CLOSE_P create_table_options?
+       T_GLOBAL? T_TEMPORARY T_TABLE ident (T_AS? T_OPEN_P select_stmt T_CLOSE_P | T_AS? select_stmt | T_OPEN_P create_table_columns T_CLOSE_P) create_table_options?
      ;
      
 create_table_stmt :
@@ -252,6 +253,7 @@ create_table_options_db2_item :
      | T_DISTRIBUTE T_BY T_HASH T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P
      | T_LOGGED 
      | T_NOT T_LOGGED
+     | T_DEFINITION T_ONLY
      ;
      
 create_table_options_hive_item :
@@ -283,6 +285,7 @@ dtype :                  // Data types
      | T_DATETIME
      | T_DEC
      | T_DECIMAL
+     | T_DOUBLE T_PRECISION?
      | T_FLOAT
      | T_INT
      | T_INTEGER
@@ -377,7 +380,7 @@ else_block :
      ;
      
 include_stmt :          // INCLUDE statement
-       T_INCLUDE file_name
+       T_INCLUDE (file_name | expr)
      ;  
      
 insert_stmt :           // INSERT statement
@@ -418,7 +421,7 @@ get_diag_stmt_rowcount_item :
      ;
      
 grant_stmt :            
-       T_GRANT grant_stmt_item (T_COMMA grant_stmt_item)* T_TO ident
+       T_GRANT grant_stmt_item (T_COMMA grant_stmt_item)* T_TO T_ROLE ident
      ;
      
 grant_stmt_item :
@@ -445,12 +448,20 @@ close_stmt :            // CLOSE cursor statement
        T_CLOSE L_ID
      ;
      
+cmp_stmt :              // CMP statement
+       T_CMP (T_ROW_COUNT | T_SUM) cmp_source T_COMMA cmp_source
+     ;
+     
+cmp_source :
+      (table_name where_clause? | T_OPEN_P select_stmt T_CLOSE_P) (T_AT ident)?
+     ;
+     
 copy_from_local_stmt :  // COPY FROM LOCAL statement
        T_COPY T_FROM T_LOCAL copy_source (T_COMMA copy_source)* T_TO copy_target copy_file_option*
      ;
      
 copy_stmt :             // COPY statement
-       T_COPY (table_name | T_OPEN_P select_stmt T_CLOSE_P) T_TO copy_target copy_option*
+       T_COPY (table_name | T_OPEN_P select_stmt T_CLOSE_P) T_TO T_HDFS? copy_target copy_option*
      ;
      
 copy_source :
@@ -458,7 +469,7 @@ copy_source :
      ;
 
 copy_target :
-       (ident | expr | L_FILE)
+       (file_name | expr) 
      ;
     
 copy_option :
@@ -615,7 +626,7 @@ select_list_item :
      ;
      
 select_list_alias :
-       {!_input.LT(1).getText().equalsIgnoreCase("FROM")}? T_AS? ident
+       {!_input.LT(1).getText().equalsIgnoreCase("INTO") && !_input.LT(1).getText().equalsIgnoreCase("FROM")}? T_AS? ident
      | T_OPEN_P T_TITLE L_S_STRING T_CLOSE_P
      ;
      
@@ -642,7 +653,7 @@ from_table_name_clause :
      ;     
 
 from_subselect_clause :
-       T_OPEN_P subselect_stmt T_CLOSE_P from_alias_clause?
+       T_OPEN_P select_stmt T_CLOSE_P from_alias_clause?
      ;
      
 from_join_clause :
@@ -669,7 +680,8 @@ from_alias_clause :
         !_input.LT(1).getText().equalsIgnoreCase("EXECUTE") && 
         !_input.LT(1).getText().equalsIgnoreCase("GROUP") &&
         !_input.LT(1).getText().equalsIgnoreCase("ORDER") &&
-        !_input.LT(1).getText().equalsIgnoreCase("LIMIT")}?
+        !_input.LT(1).getText().equalsIgnoreCase("LIMIT") &&
+        !_input.LT(1).getText().equalsIgnoreCase("WITH")}?
        T_AS? ident (T_OPEN_P L_ID (T_COMMA L_ID)* T_CLOSE_P)? 
      ;
      
@@ -699,7 +711,7 @@ select_options :
 
 select_options_item :
        T_LIMIT expr
-     | T_WITH (T_RR | T_RS | T_CS | T_UR)
+     | T_WITH (T_RR | T_RS | T_CS | T_UR) (T_USE T_AND T_KEEP (T_EXCLUSIVE | T_UPDATE | T_SHARE) T_LOCKS)?
      ;
 
 update_stmt :                              // UPDATE statement
@@ -738,7 +750,7 @@ delete_stmt :                             // DELETE statement
      ;
      
 bool_expr :                               // Boolean condition
-       T_OPEN_P bool_expr T_CLOSE_P 
+       T_NOT? T_OPEN_P bool_expr T_CLOSE_P 
      | bool_expr bool_expr_logical_operator bool_expr 
      | bool_expr_atom
      ;
@@ -900,6 +912,7 @@ expr_spec_func :
      | T_MIN_PART_INT T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P 
      | T_MAX_PART_DATE T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P 
      | T_MIN_PART_DATE T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P 
+     | T_PART_COUNT T_OPEN_P expr (T_COMMA expr T_EQUAL expr)* T_CLOSE_P 
      | T_PART_LOC T_OPEN_P expr (T_COMMA expr T_EQUAL expr)+ (T_COMMA expr)? T_CLOSE_P 
      | T_TRIM T_OPEN_P expr T_CLOSE_P
      | T_SUBSTRING T_OPEN_P expr T_FROM expr (T_FOR expr)? T_CLOSE_P
@@ -946,7 +959,7 @@ host_stmt :
      ;
      
 file_name :
-       L_ID | L_FILE
+       L_FILE | '/'? ident ('/' ident)*
      ;
      
 date_literal :                             // DATE 'YYYY-MM-DD' literal
@@ -1012,6 +1025,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_CLIENT     
      | T_CLOSE 
      | T_CLUSTERED
+     | T_CMP
      | T_COLLECTION     
      | T_COPY
      | T_COMMIT
@@ -1043,6 +1057,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_DEFERRED
      | T_DEFINED
      | T_DEFINER
+     | T_DEFINITION
      | T_DELETE
      | T_DELIMITED
      | T_DELIMITER
@@ -1051,7 +1066,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_DIAGNOSTICS
      | T_DISTINCT 
      | T_DISTRIBUTE
-     | T_DO         
+     | T_DO        
+     | T_DOUBLE     
      | T_DROP    
      | T_DYNAMIC      
      // T_ELSE reserved word         
@@ -1062,7 +1078,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_EXCEPT       
      | T_EXEC         
      | T_EXECUTE      
-     | T_EXCEPTION    
+     | T_EXCEPTION  
+     | T_EXCLUSIVE     
      | T_EXISTS
      | T_EXIT         
      | T_FETCH  
@@ -1085,6 +1102,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_HANDLER      
      | T_HASH
      | T_HAVING       
+     | T_HDFS
      | T_HIVE         
      | T_HOST    
      | T_IDENTITY     
@@ -1106,7 +1124,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_ITEMS     
      | T_IS    
      | T_ISOPEN
-     | T_JOIN     
+     | T_JOIN  
+     | T_KEEP     
      | T_KEY
      | T_KEYS
      | T_LAG
@@ -1121,6 +1140,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_LOCAL     
      | T_LOCATOR
      | T_LOCATORS
+     | T_LOCKS
      | T_LOGGED    
      | T_LOGGING     
      | T_LOOP    
@@ -1157,10 +1177,12 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_OVER
      | T_OVERWRITE
      | T_OWNER
+     | T_PART_COUNT
      | T_PART_LOC 
      | T_PARTITION  
      | T_PCTFREE
-     | T_PCTUSED     
+     | T_PCTUSED  
+     | T_PRECISION     
      | T_PRESERVE
      | T_PRIMARY
      | T_PRINT 
@@ -1181,7 +1203,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_REVERSE    
      | T_RIGHT
      | T_RLIKE
-     | T_RS     
+     | T_RS 
+     | T_ROLE     
      | T_ROLLBACK
      | T_ROW
      | T_ROWS
@@ -1194,6 +1217,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_SELECT       
      | T_SET 
      | T_SETS     
+     | T_SHARE
      | T_SIGNAL
      | T_SMALLDATETIME
      | T_SMALLINT     
@@ -1277,6 +1301,7 @@ T_CHARACTER       : C H A R A C T E R ;
 T_CLIENT          : C L I E N T ;
 T_CLOSE           : C L O S E ;
 T_CLUSTERED       : C L U S T E R E D;
+T_CMP             : C M P ; 
 T_COLLECTION      : C O L L E C T I O N ; 
 T_COPY            : C O P Y ;
 T_COMMIT          : C O M M I T ; 
@@ -1304,6 +1329,7 @@ T_DEFAULT         : D E F A U L T ;
 T_DEFERRED        : D E F E R R E D ; 
 T_DEFINED         : D E F I N E D ; 
 T_DEFINER         : D E F I N E R ;
+T_DEFINITION      : D E F I N I T I O N ; 
 T_DELETE          : D E L E T E ;
 T_DELIMITED       : D E L I M I T E D ; 
 T_DELIMITER       : D E L I M I T E R ; 
@@ -1312,6 +1338,7 @@ T_DIAGNOSTICS     : D I A G N O S T I C S ;
 T_DISTINCT        : D I S T I N C T ;
 T_DISTRIBUTE      : D I S T R I B U T E ;
 T_DO              : D O ;
+T_DOUBLE          : D O U B L E ;
 T_DROP            : D R O P ;
 T_DYNAMIC         : D Y N A M I C ; 
 T_ELSE            : E L S E ;
@@ -1323,6 +1350,7 @@ T_EXCEPT          : E X C E P T ;
 T_EXEC            : E X E C ;
 T_EXECUTE         : E X E C U T E ;
 T_EXCEPTION       : E X C E P T I O N ;
+T_EXCLUSIVE       : E X C L U S I V E ; 
 T_EXISTS          : E X I S T S ; 
 T_EXIT            : E X I T ;
 T_FETCH           : F E T C H ;
@@ -1344,6 +1372,7 @@ T_GROUP           : G R O U P ;
 T_HANDLER         : H A N D L E R ;
 T_HASH            : H A S H ;
 T_HAVING          : H A V I N G ;
+T_HDFS            : H D F S ; 
 T_HIVE            : H I V E ;
 T_HOST            : H O S T ;
 T_IDENTITY        : I D E N T I T Y ; 
@@ -1366,6 +1395,7 @@ T_IS              : I S ;
 T_ISOPEN          : I S O P E N ;
 T_ITEMS           : I T E M S ; 
 T_JOIN            : J O I N ;
+T_KEEP            : K E E P; 
 T_KEY             : K E Y ;
 T_KEYS            : K E Y S ;
 T_LANGUAGE        : L A N G U A G E ;
@@ -1377,6 +1407,7 @@ T_LINES           : L I N E S ;
 T_LOCAL           : L O C A L ;
 T_LOCATOR         : L O C A T O R ; 
 T_LOCATORS        : L O C A T O R S ; 
+T_LOCKS           : L O C K S ; 
 T_LOGGED          : L O G G E D ; 
 T_LOGGING         : L O G G I N G ; 
 T_LOOP            : L O O P ;
@@ -1416,6 +1447,7 @@ T_OWNER           : O W N E R ;
 T_PARTITION       : P A R T I T I O N ; 
 T_PCTFREE         : P C T F R E E ; 
 T_PCTUSED         : P C T U S E D ;
+T_PRECISION       : P R E C I S I O N ; 
 T_PRESERVE        : P R E S E R V E ; 
 T_PRIMARY         : P R I M A R Y ;
 T_PRINT           : P R I N T ; 
@@ -1434,6 +1466,7 @@ T_RETURNS         : R E T U R N S ;
 T_REVERSE         : R E V E R S E ;
 T_RIGHT           : R I G H T ;
 T_RLIKE           : R L I K E ;
+T_ROLE            : R O L E ;
 T_ROLLBACK        : R O L L B A C K ;
 T_ROW             : R O W ; 
 T_ROWS            : R O W S ; 
@@ -1449,6 +1482,7 @@ T_SEL             : S E L ;
 T_SELECT          : S E L E C T ; 
 T_SET             : S E T ;
 T_SETS            : S E T S;
+T_SHARE           : S H A R E ; 
 T_SIGNAL          : S I G N A L ;
 T_SMALLDATETIME   : S M A L L D A T E T I M E ;
 T_SMALLINT        : S M A L L I N T ;
@@ -1513,6 +1547,7 @@ T_MAX_PART_INT         : M A X '_' P A R T '_' I N T ;
 T_MIN_PART_INT         : M I N '_' P A R T '_' I N T ;
 T_MAX_PART_DATE        : M A X '_' P A R T '_' D A T E ;
 T_MIN_PART_DATE        : M I N '_' P A R T '_' D A T E ;
+T_PART_COUNT           : P A R T '_' C O U N T ; 
 T_PART_LOC             : P A R T '_' L O C ;
 T_RANK                 : R A N K ;
 T_ROW_NUMBER           : R O W '_' N U M B E R;
@@ -1566,8 +1601,7 @@ L_WS        : L_BLANK+ -> skip ;                                       // Whites
 L_M_COMMENT : '/*' .*? '*/' -> channel(HIDDEN) ;                       // Multiline comment
 L_S_COMMENT : ('--' | '//')  .*? '\r'? '\n' -> channel(HIDDEN) ;       // Single line comment
 
-L_FILE      : '/'? L_ID ('/' L_ID)*                                    // File path
-            | ([a-zA-Z] ':' '\\'?)? L_ID ('\\' L_ID)*
+L_FILE      : ([a-zA-Z] ':' '\\'?)? L_ID ('\\' L_ID)*                  // File path (a/b/c Linux path causes conflicts with division operator and handled at parser level)
             ; 
 
 L_LABEL     : ([a-zA-Z] | L_DIGIT | '_')* ':'            

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Cmp.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Cmp.java b/hplsql/src/main/java/org/apache/hive/hplsql/Cmp.java
new file mode 100644
index 0000000..ee65a88
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Cmp.java
@@ -0,0 +1,314 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.hplsql;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.util.ArrayList;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+
+public class Cmp implements Runnable {
+  
+  Exec exec;
+  Timer timer = new Timer();  
+  boolean trace = false;
+  boolean info = false;
+  
+  Query query;
+  String conn;
+  HplsqlParser.Cmp_stmtContext ctx;
+  
+  int tests = 0;
+  int failedTests = 0;
+  int failedTestsHighDiff = 0;
+  
+  Cmp(Exec e) {
+    exec = e;  
+    trace = exec.getTrace();
+    info = exec.getInfo();
+  }
+  
+  Cmp(Exec e, HplsqlParser.Cmp_stmtContext c, Query q, String cn) {
+    exec = e;  
+    trace = exec.getTrace();
+    info = exec.getInfo();
+    ctx = c;
+    query = q;
+    conn = cn;
+  }
+  
+  /**
+   * Run CMP command
+   */
+  Integer run(HplsqlParser.Cmp_stmtContext ctx) {
+    trace(ctx, "CMP");
+    this.ctx = ctx;
+    timer.start();
+    StringBuilder conn1 = new StringBuilder();
+    StringBuilder conn2 = new StringBuilder();
+    Query query1 = new Query();
+    Query query2 = new Query();
+    Boolean equal = null;
+    try {
+      String sql1 = getSql(ctx, conn1, 0);
+      String sql2 = getSql(ctx, conn2, 1);
+      if (trace) {
+        trace(ctx, "Query 1: " + sql1);
+        trace(ctx, "Query 2: " + sql2);
+      }
+      query1.setSql(sql1);
+      query2.setSql(sql2);
+      Cmp cmp1 = new Cmp(exec, ctx, query1, conn1.toString());
+      Cmp cmp2 = new Cmp(exec, ctx, query2, conn2.toString());
+      Thread t1 = new Thread(cmp1);
+      Thread t2 = new Thread(cmp2);
+      t1.start();
+      t2.start();
+      t1.join();
+      t2.join();
+      equal = compare(query1, query2);      
+    }
+    catch(Exception e) {
+      exec.signal(e);
+      return -1;
+    }
+    finally {
+      long elapsed = timer.stop();
+      if (info) {
+        String message = "CMP ";
+        if (equal != null) {
+          if (equal) {
+            message += "Equal, " + tests + " tests";
+          }
+          else {
+            message += "Not Equal, " + failedTests + " of " + tests + " tests failed";
+            message += ", " + failedTestsHighDiff + " failed tests with more than 0.01% difference";
+          }
+        }
+        else {
+          message += "Failed";
+        }
+        info(ctx, message + ", " + timer.format());
+      }
+      exec.closeQuery(query1, conn1.toString());
+      exec.closeQuery(query2, conn2.toString());
+    }
+    return 0;
+  }
+  
+  /**
+   * Get data for comparison from the source
+   */
+  public void run() {
+    exec.executeQuery(ctx, query, conn);
+  }
+  
+  /**
+   * Compare the results
+   */
+  Boolean compare(Query query1, Query query2) {
+    if (query1.error()) { 
+      exec.signal(query1);
+      return null;
+    }
+    else if (query2.error()) { 
+      exec.signal(query2);
+      return null;
+    }
+    ResultSet rs1 = query1.getResultSet();
+    ResultSet rs2 = query2.getResultSet();
+    if (rs1 == null || rs2 == null) {
+      exec.setSqlCode(-1);
+      return null;
+    }
+    boolean equal = true;
+    tests = 0;
+    failedTests = 0;
+    try {
+      ResultSetMetaData rm1 = rs1.getMetaData();
+      ResultSetMetaData rm2 = rs2.getMetaData();
+      int cnt1 = rm1.getColumnCount();
+      int cnt2 = rm2.getColumnCount();
+      tests = cnt1;
+      while (rs1.next() && rs2.next()) {
+        for (int i = 1; i <= tests; i++) {
+          Var v1 = new Var(Var.Type.DERIVED_TYPE);
+          Var v2 = new Var(Var.Type.DERIVED_TYPE);
+          v1.setValue(rs1, rm1, i);
+          if (i <= cnt2) {
+            v2.setValue(rs2, rm2, i);
+          }
+          boolean e = true;
+          if (!(v1.isNull() && v2.isNull()) && !v1.equals(v2)) {
+            equal = false;
+            e = false;
+            failedTests++;
+          }
+          if (trace || info) {
+            String m = rm1.getColumnName(i) + "\t" + v1.toString() + "\t" + v2.toString();
+            if (!e) {
+              m += "\tNot equal";
+              BigDecimal diff = v1.percentDiff(v2);
+              if (diff != null) {
+                if (diff.compareTo(BigDecimal.ZERO) != 0) {
+                  m += ", " + diff + "% difference";
+                  failedTestsHighDiff++;
+                }
+                else {
+                  m += ", less then 0.01% difference";
+                }
+              }
+              else {
+                failedTestsHighDiff++;
+              }
+            }
+            if (trace) {
+              trace(null, m);
+            }
+            else {
+              info(null, m);
+            }
+          }
+        }
+        if (equal) {
+          exec.setSqlSuccess();
+        }
+        else {
+          exec.setSqlCode(1);
+        }
+      }
+    }
+    catch(Exception e) {
+      exec.signal(e);
+      return null;
+    }
+    return new Boolean(equal);
+  }
+  
+  /**
+   * Define the SQL query to access data
+   */
+  String getSql(HplsqlParser.Cmp_stmtContext ctx, StringBuilder conn, int idx) throws Exception {
+    StringBuilder sql = new StringBuilder();
+    String table = null;
+    String query = null;
+    if (ctx.cmp_source(idx).table_name() != null) {
+      table = evalPop(ctx.cmp_source(idx).table_name()).toString();
+    }
+    else {
+      query = evalPop(ctx.cmp_source(idx).select_stmt()).toString();
+    }
+    if (ctx.cmp_source(idx).T_AT() != null) {
+      conn.append(ctx.cmp_source(idx).ident().getText());
+    }
+    else if (table != null) {
+      conn.append(exec.getObjectConnection(ctx.cmp_source(idx).table_name().getText()));
+    }  
+    else {
+      conn.append(exec.getStatementConnection());
+    }
+    sql.append("SELECT ");
+    sql.append(getSelectList(ctx, conn.toString(), table, query));
+    sql.append(" FROM ");
+    if (table != null) {
+      sql.append(table);
+      if (ctx.cmp_source(idx).where_clause() != null) {
+        sql.append(" " + evalPop(ctx.cmp_source(idx).where_clause()).toString());
+      }
+    }
+    else {
+      sql.append("(");
+      sql.append(query);
+      sql.append(") t");
+    }
+    return sql.toString();
+  }
+  
+  /**
+   * Define SELECT listto access data
+   */
+  String getSelectList(HplsqlParser.Cmp_stmtContext ctx, String conn, String table, String query) throws Exception {
+    StringBuilder sql = new StringBuilder();
+    sql.append("COUNT(1) AS row_count");
+    if (ctx.T_SUM() != null && table != null) {
+      Row row = exec.meta.getRowDataType(ctx, conn, table);
+      if (row != null) {
+        ArrayList<Column> cols = row.getColumns();
+        int cnt = row.size();
+        sql.append(",\n");
+        for (int i = 0; i < cnt; i++) {
+          Column col = cols.get(i);
+          String name = col.getName();
+          Var.Type type = Var.defineType(col.getType());
+          sql.append("COUNT(" + name + ") AS " + name + "_COUNT_NOT_NULL");
+          if (type == Var.Type.STRING) {
+            sql.append(",\n");
+            sql.append("SUM(LENGTH(" + name + ")) AS " + name + "_SUM_LENGTH,\n");
+            sql.append("MIN(LENGTH(" + name + ")) AS " + name + "_MIN_LENGTH,\n");
+            sql.append("MAX(LENGTH(" + name + ")) AS " + name + "_MAX_LENGTH");
+          }
+          else if (type == Var.Type.BIGINT || type == Var.Type.DECIMAL || type == Var.Type.DOUBLE) {
+            sql.append(",\n");
+            sql.append("SUM(" + name + ") AS " + name + "_SUM,\n");
+            sql.append("MIN(" + name + ") AS " + name + "_MIN,\n");
+            sql.append("MAX(" + name + ") AS " + name + "_MAX");
+          }
+          else if (type == Var.Type.DATE || type == Var.Type.TIMESTAMP) {
+            sql.append(",\n");
+            sql.append("SUM(YEAR(" + name + ")) AS " + name + "_SUM_YEAR,\n");
+            sql.append("SUM(MONTH(" + name + ")) AS " + name + "_SUM_MONTH,\n");
+            sql.append("SUM(DAY(" + name + ")) AS " + name + "_SUM_DAY,\n");
+            sql.append("MIN(" + name + ") AS " + name + "_MIN,\n");
+            sql.append("MAX(" + name + ") AS " + name + "_MAX");
+          }
+          if (i + 1 < cnt) {
+            sql.append(",\n");
+          }
+        }
+      }
+    }
+    return sql.toString();
+  }
+  
+  /**
+   * Evaluate the expression and pop value from the stack
+   */
+  Var evalPop(ParserRuleContext ctx) {
+    exec.visit(ctx);
+    if (!exec.stack.isEmpty()) { 
+      return exec.stackPop();
+    }
+    return Var.Empty;
+  }
+
+  /**
+   * Trace and information
+   */
+  public void trace(ParserRuleContext ctx, String message) {
+    exec.trace(ctx, message);
+  }
+  
+  public void info(ParserRuleContext ctx, String message) {
+    exec.info(ctx, message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Column.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Column.java b/hplsql/src/main/java/org/apache/hive/hplsql/Column.java
index 252a870..e4e914c 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Column.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Column.java
@@ -27,9 +27,36 @@ public class Column {
   String type;
   Var value;
   
+  int len;
+  int scale;
+  
   Column(String name, String type) {
     this.name = name;
-    this.type = type;
+    len = 0;
+    scale = 0;
+    setType(type);
+  }
+  
+  /**
+   * Set the column type with its length/precision
+   */
+  void setType(String type) {
+    int open = type.indexOf('(');
+    if (open == -1) {
+      this.type = type;
+    }
+    else {
+      this.type = type.substring(0, open);
+      int comma = type.indexOf(',', open);
+      int close = type.indexOf(')', open);
+      if (comma == -1) {
+        len = Integer.parseInt(type.substring(open + 1, close));
+      }
+      else {
+        len = Integer.parseInt(type.substring(open + 1, comma));
+        scale = Integer.parseInt(type.substring(comma + 1, close));
+      }
+    }
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
index c8cc910..12f43c9 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
@@ -26,6 +26,7 @@ import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.ResultSet;
+import java.sql.PreparedStatement;
 
 public class Conn {
  
@@ -76,6 +77,25 @@ public class Conn {
   }
   
   /**
+   * Prepare a SQL query
+   */
+  public Query prepareQuery(Query query, String connName) {
+    try {
+      Connection conn = getConnection(connName);
+      timer.start();
+      PreparedStatement stmt = conn.prepareStatement(query.sql);
+      timer.stop();
+      query.set(conn, stmt);      
+      if (info) {
+        exec.info(null, "Prepared statement executed successfully (" + timer.format() + ")");
+      }      
+    } catch (Exception e) {
+      query.setError(e);
+    }
+    return query;
+  }
+  
+  /**
    * Execute a SQL statement
    */
   public Query executeSql(String sql, String connName) {
@@ -117,6 +137,7 @@ public class Conn {
     if (sqls != null) {
       Statement s = conn.createStatement();
       for (String sql : sqls) {
+        exec.info(null, "Starting pre-SQL statement");
         s.execute(sql);
       }
       s.close();

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java b/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java
index 30b98ca..9968b24 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.List;
 import java.io.FileOutputStream;
+import java.io.OutputStream;
 import java.io.IOException;
 
 import org.apache.hadoop.fs.FileSystem;
@@ -165,8 +166,8 @@ public class Copy {
     exec.returnConnection(targetConn, conn);
     exec.setRowCount(rows);
     long elapsed = timer.stop();
-    if (trace) {
-      trace(ctx, "COPY completed: " + rows + " row(s), " + timer.format() + ", " + rows/(elapsed/1000) + " rows/sec");
+    if (info) {
+      info(ctx, "COPY completed: " + rows + " row(s), " + timer.format() + ", " + rows/(elapsed/1000) + " rows/sec");
     }
   }
   
@@ -192,16 +193,35 @@ public class Copy {
     byte[] nullstr = "NULL".getBytes();
     int cols = rm.getColumnCount();
     int rows = 0;
-    if (trace) {
-      trace(ctx, "SELECT executed: " + cols + " columns, output file: " + filename);
+    if (trace || info) {
+      String mes = "Query executed: " + cols + " columns, output file: " + filename;
+      if (trace) {
+        trace(ctx, mes);
+      }
+      else {
+        info(ctx, mes);
+      }
     } 
-    java.io.File file = new java.io.File(filename);
-    FileOutputStream out = null;
+    java.io.File file = null;
+    File hdfsFile = null;
+    if (ctx.T_HDFS() == null) {
+      file = new java.io.File(filename);
+    }
+    else {
+      hdfsFile = new File();
+    }     
+    OutputStream out = null;
+    timer.start();
     try {      
-      if (!file.exists()) {
-        file.createNewFile();
+      if (file != null) {
+        if (!file.exists()) {
+          file.createNewFile();
+        }
+        out = new FileOutputStream(file, false /*append*/);
+      }
+      else {
+        out = hdfsFile.create(filename, true /*overwrite*/);
       }
-      out = new FileOutputStream(file, false /*append*/);
       String col;
       String sql = "";
       if (sqlInsert) {
@@ -237,8 +257,9 @@ public class Copy {
         out.close();
       }
     }
-    if (trace) {
-      trace(ctx, "COPY rows: " + rows);
+    long elapsed = timer.stop();
+    if (info) {
+      info(ctx, "COPY completed: " + rows + " row(s), " + timer.format() + ", " + rows/elapsed/1000 + " rows/sec");
     }
   }
   
@@ -376,7 +397,12 @@ public class Copy {
       }
       else if (option.T_AT() != null) {
         targetConn = option.ident().getText();
-        sqlInsertName = ctx.copy_target().ident().getText();
+        if (ctx.copy_target().expr() != null) {
+          sqlInsertName = evalPop(ctx.copy_target().expr()).toString();
+        }
+        else {
+          sqlInsertName = ctx.copy_target().getText();
+        }
       }
       else if (option.T_BATCHSIZE() != null) {
         batchSize = evalPop(option.expr()).intValue();

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
index f5592e1..38b5380 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
@@ -506,6 +506,24 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   public Query executeQuery(ParserRuleContext ctx, String sql, String connProfile) {
     return executeQuery(ctx, new Query(sql), connProfile);
   }
+  
+  /**
+   * Prepare a SQL query (SELECT)
+   */
+  public Query prepareQuery(ParserRuleContext ctx, Query query, String connProfile) {
+    if (!exec.offline) {
+      exec.rowCount = 0;
+      exec.conn.prepareQuery(query, connProfile);
+      return query;
+    }
+    setSqlNoData();
+    info(ctx, "Not executed - offline mode set");
+    return query;
+  }
+
+  public Query prepareQuery(ParserRuleContext ctx, String sql, String connProfile) {
+    return prepareQuery(ctx, new Query(sql), connProfile);
+  }
 
   /**
    * Execute a SQL statement 
@@ -950,6 +968,11 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   }
   
   @Override 
+  public Integer visitFrom_subselect_clause(HplsqlParser.From_subselect_clauseContext ctx) { 
+    return exec.select.fromSubselect(ctx); 
+  }
+  
+  @Override 
   public Integer visitFrom_join_clause(HplsqlParser.From_join_clauseContext ctx) { 
     return exec.select.fromJoin(ctx); 
   }
@@ -1162,6 +1185,14 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   }
   
   /**
+   * CMP statement
+   */
+  @Override 
+  public Integer visitCmp_stmt(HplsqlParser.Cmp_stmtContext ctx) { 
+    return new Cmp(exec).run(ctx); 
+  }
+  
+  /**
    * COPY statement
    */
   @Override 
@@ -1926,8 +1957,13 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
    */
   @Override 
   public Integer visitDate_literal(HplsqlParser.Date_literalContext ctx) { 
-    String str = evalPop(ctx.string()).toString();
-    stackPush(new Var(Var.Type.DATE, Utils.toDate(str))); 
+    if (!exec.buildSql) {
+      String str = evalPop(ctx.string()).toString();
+      stackPush(new Var(Var.Type.DATE, Utils.toDate(str)));
+    }
+    else {
+      stackPush(getFormattedText(ctx));
+    }
     return 0; 
   }
 
@@ -1936,16 +1972,21 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
    */
   @Override 
   public Integer visitTimestamp_literal(HplsqlParser.Timestamp_literalContext ctx) { 
-    String str = evalPop(ctx.string()).toString();
-    int len = str.length();
-    int precision = 0;
-    if (len > 19 && len <= 29) {
-      precision = len - 20;
-      if (precision > 3) {
-        precision = 3;
+    if (!exec.buildSql) {
+      String str = evalPop(ctx.string()).toString();
+      int len = str.length();
+      int precision = 0;
+      if (len > 19 && len <= 29) {
+        precision = len - 20;
+        if (precision > 3) {
+          precision = 3;
+        }
       }
+      stackPush(new Var(Utils.toTimestamp(str), precision));
+    }
+    else {
+      stackPush(getFormattedText(ctx));
     }
-    stackPush(new Var(Utils.toTimestamp(str), precision)); 
     return 0; 
   }
   
@@ -1979,6 +2020,9 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
    * @throws Exception 
    */
   Connection getConnection(String conn) throws Exception {
+    if (conn == null || conn.equalsIgnoreCase("default")) {
+      conn = exec.conf.defaultConnection;
+    }
     return exec.conn.getConnection(conn);
   }
   
@@ -1993,7 +2037,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
    * Define the database type by profile name
    */
   Conn.Type getConnectionType(String conn) {
-    return exec.conn.getType(conn);
+    return exec.conn.getTypeByProfile(conn);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
index 7269798..7c500a8 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
@@ -49,6 +49,9 @@ public class Expression {
       else if (ctx.T_SUB() != null) {
         operatorSub(ctx); 
       }
+      else if (ctx.T_MUL() != null) {
+        operatorMultiply(ctx); 
+      }
       else if (ctx.T_DIV() != null) {
         operatorDiv(ctx); 
       }
@@ -98,16 +101,17 @@ public class Expression {
    * Evaluate a boolean expression
    */
   public void execBool(HplsqlParser.Bool_exprContext ctx) {
-    if (ctx.T_OPEN_P() != null) {
-      eval(ctx.bool_expr(0));
-      return;
-    }
-    else if (ctx.bool_expr_atom() != null) {
+    if (ctx.bool_expr_atom() != null) {
       eval(ctx.bool_expr_atom());
       return;
     }
     Var result = evalPop(ctx.bool_expr(0));
-    if (ctx.bool_expr_logical_operator() != null) {
+    if (ctx.T_OPEN_P() != null) {
+      if (ctx.T_NOT() != null) {
+        result.negate();
+      }
+    }
+    else if (ctx.bool_expr_logical_operator() != null) {
       if (ctx.bool_expr_logical_operator().T_AND() != null) {
         if (result.isTrue()) {
           result = evalPop(ctx.bool_expr(1));
@@ -359,6 +363,23 @@ public class Expression {
   }
   
   /**
+   * Multiplication operator
+   */
+  public void operatorMultiply(HplsqlParser.ExprContext ctx) {
+    Var v1 = evalPop(ctx.expr(0));
+    Var v2 = evalPop(ctx.expr(1));
+    if (v1.value == null || v2.value == null) {
+      evalNull();
+    }
+    else if (v1.type == Type.BIGINT && v2.type == Type.BIGINT) {
+      exec.stackPush(new Var((Long)v1.value * (Long)v2.value)); 
+    }
+    else {
+      exec.signal(Signal.Type.UNSUPPORTED_OPERATION, "Unsupported data types in multiplication operator");
+    }
+  }
+  
+  /**
    * Division operator
    */
   public void operatorDiv(HplsqlParser.ExprContext ctx) {

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/File.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/File.java b/hplsql/src/main/java/org/apache/hive/hplsql/File.java
index 6a8ddfe..e748772 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/File.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/File.java
@@ -46,17 +46,27 @@ public class File {
   /**
    * Create a file
    */
-  public void create(String dir, String file, boolean overwrite) {
-    path = new Path(dir, file);
+  public FSDataOutputStream create(boolean overwrite) {
     try {
       if (fs == null) {
-        fs = FileSystem.get(new Configuration());
+        fs = createFs();
       }
       out = fs.create(path, overwrite);
     } 
     catch (IOException e) {
       e.printStackTrace();
     }
+    return out;
+  }
+  
+  public FSDataOutputStream create(String dir, String file, boolean overwrite) {
+    path = new Path(dir, file);
+    return create(overwrite);
+  }
+
+  public FSDataOutputStream create(String file, boolean overwrite) {
+    path = new Path(file);
+    return create(overwrite);
   }
   
   /**
@@ -66,7 +76,7 @@ public class File {
    path = new Path(dir, file);
    try {
      if (fs == null) {
-       fs = FileSystem.get(new Configuration());
+       fs = createFs();
      }
      in = fs.open(path);
    } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Meta.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Meta.java b/hplsql/src/main/java/org/apache/hive/hplsql/Meta.java
index 485bcdf..2e04ef9 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Meta.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Meta.java
@@ -18,7 +18,9 @@
 
 package org.apache.hive.hplsql;
 
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
 import java.util.ArrayList;
 import java.util.HashMap;
 
@@ -90,12 +92,9 @@ public class Meta {
    */
   Row readColumns(ParserRuleContext ctx, String conn, String table, HashMap<String, Row> map) {
     Row row = null;
-    String sql = null;
     Conn.Type connType = exec.getConnectionType(conn); 
     if (connType == Conn.Type.HIVE) {
-      sql = "DESCRIBE " + table;
-    }
-    if (sql != null) {
+      String sql = "DESCRIBE " + table;
       Query query = new Query(sql);
       exec.executeQuery(ctx, query, conn); 
       if (!query.error()) {
@@ -115,6 +114,27 @@ public class Meta {
       }
       exec.closeQuery(query, conn);
     }
+    else {
+      Query query = exec.prepareQuery(ctx, "SELECT * FROM " + table, conn); 
+      if (!query.error()) {
+        try {
+          PreparedStatement stmt = query.getPreparedStatement();
+          ResultSetMetaData rm = stmt.getMetaData();
+          int cols = rm.getColumnCount();
+          for (int i = 1; i <= cols; i++) {
+            String col = rm.getColumnName(i);
+            String typ = rm.getColumnTypeName(i);
+            if (row == null) {
+              row = new Row();
+            }
+            row.addColumn(col.toUpperCase(), typ);
+          }
+          map.put(table, row);
+        }
+        catch (Exception e) {}
+      }
+      exec.closeQuery(query, conn);
+    }
     return row;
   }
   

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
index 08cd6a7..e196f86 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
@@ -21,6 +21,7 @@ package org.apache.hive.hplsql;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.PreparedStatement;
 import java.sql.Statement;
 
 import org.antlr.v4.runtime.ParserRuleContext;
@@ -32,6 +33,7 @@ public class Query {
   
   Connection conn;
   Statement stmt;
+  PreparedStatement pstmt;
   ResultSet rs;
   Exception exception;
 
@@ -59,6 +61,11 @@ public class Query {
     }
   }
   
+  public void set(Connection conn, PreparedStatement pstmt) {
+    this.conn = conn;
+    this.pstmt = pstmt;
+  }
+  
   /**
    * Set the fetch status
    */
@@ -132,6 +139,10 @@ public class Query {
         stmt.close();
         stmt = null;
       }
+      if(pstmt != null) {
+        pstmt.close();
+        pstmt = null;
+      }
       state = State.CLOSE;
     } catch (SQLException e) {
       e.printStackTrace();
@@ -190,6 +201,13 @@ public class Query {
   }
   
   /**
+   * Get the prepared statement object
+   */
+  public PreparedStatement getPreparedStatement() {
+    return pstmt;
+  }
+  
+  /**
    * Get the connection object
    */
   public Connection getConnection() {

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Select.java b/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
index 71ca848..56fbb05 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
@@ -218,7 +218,10 @@ public class Select {
       sql.append(" " + getText(ctx.order_by_clause()));
     }
     if (ctx.select_options() != null) {
-      sql.append(" " + evalPop(ctx.select_options()));
+      Var opt = evalPop(ctx.select_options());
+      if (!opt.isNull()) {
+        sql.append(" " + opt.toString());
+      }
     }
     if (ctx.select_list().select_list_limit() != null) {
       sql.append(" LIMIT " + evalPop(ctx.select_list().select_list_limit().expr()));
@@ -281,6 +284,21 @@ public class Select {
     exec.stackPush(sql);
     return 0; 
   }
+  
+  /**
+   * Subselect in FROM
+   */
+  public Integer fromSubselect(HplsqlParser.From_subselect_clauseContext ctx) {     
+    StringBuilder sql = new StringBuilder();
+    sql.append("(");
+    sql.append(evalPop(ctx.select_stmt()).toString());
+    sql.append(")");
+    if (ctx.from_alias_clause() != null) {
+      sql.append(" ").append(exec.getText(ctx.from_alias_clause()));
+    }
+    exec.stackPush(sql);
+    return 0; 
+  }
  
   /**
    * JOIN clause in FROM
@@ -341,10 +359,13 @@ public class Select {
    * WHERE clause
    */
   public Integer where(HplsqlParser.Where_clauseContext ctx) { 
+    boolean oldBuildSql = exec.buildSql; 
+    exec.buildSql = true;
     StringBuilder sql = new StringBuilder();
     sql.append(ctx.T_WHERE().getText());
     sql.append(" " + evalPop(ctx.bool_expr()));
     exec.stackPush(sql);
+    exec.buildSql = oldBuildSql;
     return 0;
   }
   

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
index 6193f49..db9ea65 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
@@ -435,7 +435,13 @@ public class Stmt {
    * INCLUDE statement
    */
   public Integer include(HplsqlParser.Include_stmtContext ctx) {
-    String file = ctx.file_name().getText();
+    String file;
+    if (ctx.file_name() != null) {
+      file = ctx.file_name().getText();
+    }
+    else {
+      file = evalPop(ctx.expr()).toString();
+    }    
     trace(ctx, "INCLUDE " + file);
     exec.includeFile(file);
     return 0; 

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
index b31a14d..150e8b4 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
@@ -19,6 +19,7 @@
 package org.apache.hive.hplsql;
 
 import java.math.BigDecimal;
+import java.math.RoundingMode;
 import java.util.ArrayList;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
@@ -32,7 +33,7 @@ import java.sql.Timestamp;
 public class Var {
 
 	// Data types
-	public enum Type {BOOL, CURSOR, DATE, DECIMAL, DERIVED_TYPE, DERIVED_ROWTYPE, FILE, IDENT, BIGINT, INTERVAL, ROW, 
+	public enum Type {BOOL, CURSOR, DATE, DECIMAL, DERIVED_TYPE, DERIVED_ROWTYPE, DOUBLE, FILE, IDENT, BIGINT, INTERVAL, ROW, 
 	                  RS_LOCATOR, STRING, STRINGLIST, TIMESTAMP, NULL};
 	public static final String DERIVED_TYPE = "DERIVED%TYPE";
 	public static final String DERIVED_ROWTYPE = "DERIVED%ROWTYPE";
@@ -79,6 +80,11 @@ public class Var {
     this.value = value;
   }
   
+	public Var(Double value) {
+    this.type = Type.DOUBLE;
+    this.value = value;
+  }
+	
 	public Var(Date value) {
     this.type = Type.DATE;
     this.value = value;
@@ -169,6 +175,9 @@ public class Var {
 	    if (val.type == Type.BIGINT) {
 	      value = BigDecimal.valueOf(val.longValue());
 	    }
+	    else if (val.type == Type.DOUBLE) {
+	      value = BigDecimal.valueOf(val.doubleValue());
+	    }
 	  }
 	  else if (type == Type.DATE) {
 	    value = Utils.toDate(val.toString());
@@ -238,6 +247,9 @@ public class Var {
     else if (type == java.sql.Types.DECIMAL || type == java.sql.Types.NUMERIC) {
       cast(new Var(rs.getBigDecimal(idx)));
     }
+    else if (type == java.sql.Types.FLOAT || type == java.sql.Types.DOUBLE) {
+      cast(new Var(new Double(rs.getDouble(idx))));
+    }
     return this;
   }
   
@@ -287,6 +299,9 @@ public class Var {
     else if (type.equalsIgnoreCase("DEC") || type.equalsIgnoreCase("DECIMAL") || type.equalsIgnoreCase("NUMERIC")) {
       return Type.DECIMAL;
     }
+    else if (type.equalsIgnoreCase("FLOAT") || type.toUpperCase().startsWith("DOUBLE")) {
+      return Type.DOUBLE;
+    }
     else if (type.equalsIgnoreCase("DATE")) {
       return Type.DATE;
     }
@@ -332,34 +347,59 @@ public class Var {
     scale = 0;
 	}
 	
-	/*
+	/**
 	 * Compare values
 	 */
 	@Override
   public boolean equals(Object obj) {
-	  if (this == obj) {
+	  if (getClass() != obj.getClass()) {
+      return false;
+    }    
+	  Var var = (Var)obj;  
+	  if (this == var) {
       return true;
 	  }
-	  else if (obj == null || this.value == null) {
+	  else if (var == null || var.value == null || this.value == null) {
       return false;
     }
-	  else if (getClass() != obj.getClass()) {
-      return false;
-	  }
-	  
-    Var var = (Var)obj;    
-    if (type == Type.BIGINT && var.type == Type.BIGINT &&
-       ((Long)value).longValue() == ((Long)var.value).longValue()) {
-      return true;
+    if (type == Type.BIGINT) {
+      if (var.type == Type.BIGINT && ((Long)value).longValue() == ((Long)var.value).longValue()) {
+        return true;
+      }
+      else if (var.type == Type.DECIMAL) {
+        return equals((BigDecimal)var.value, (Long)value);
+      }
     }
     else if (type == Type.STRING && var.type == Type.STRING &&
             ((String)value).equals((String)var.value)) {
       return true;
     }
+    else if (type == Type.DECIMAL && var.type == Type.DECIMAL &&
+            ((BigDecimal)value).compareTo((BigDecimal)var.value) == 0) {
+      return true;
+    }
+    else if (type == Type.DOUBLE) {
+      if (var.type == Type.DOUBLE && ((Double)value).compareTo((Double)var.value) == 0) {
+        return true;
+      }
+      else if (var.type == Type.DECIMAL && ((Double)value).compareTo(((BigDecimal)var.value).doubleValue()) == 0) {
+        return true;
+      }
+    }
     return false;
 	}
+    
+  /**
+   * Check if variables of different data types are equal
+   */
+  public boolean equals(BigDecimal d, Long i) {
+    if (d.compareTo(new BigDecimal(i)) == 0) {
+      return true;
+    }
+    return false;
+  }
 	
-	/*
+	/**
    * Compare values
    */
   public int compareTo(Var v) {
@@ -377,6 +417,20 @@ public class Var {
     }
     return -1;
   }
+  
+  /**
+   * Calculate difference between values in percent
+   */
+  public BigDecimal percentDiff(Var var) {
+    BigDecimal d1 = new Var(Var.Type.DECIMAL).cast(this).decimalValue();
+    BigDecimal d2 = new Var(Var.Type.DECIMAL).cast(var).decimalValue();
+    if (d1 != null && d2 != null) {
+      if (d1.compareTo(BigDecimal.ZERO) != 0) {
+        return d1.subtract(d2).abs().multiply(new BigDecimal(100)).divide(d1, 2, RoundingMode.HALF_UP);
+      }
+    }
+    return null;
+  }
 	
 	 /**
    * Increment an integer value
@@ -417,6 +471,26 @@ public class Var {
     }
     return -1;
   }
+  
+  /**
+   * Return a decimal value
+   */
+  public BigDecimal decimalValue() {
+    if (type == Type.DECIMAL) {
+      return (BigDecimal)value;
+    }
+    return null;
+  }
+  
+  /**
+   * Return a double value
+   */
+  public double doubleValue() {
+    if (type == Type.DOUBLE) {
+      return ((Double)value).doubleValue();
+    }
+    return -1;
+  }
 	
 	/**
 	 * Return true/false for BOOL type
@@ -429,6 +503,16 @@ public class Var {
 	}
 	
 	/**
+	 * Negate the boolean value
+	 */
+	public void negate() {
+    if(type == Type.BOOL && value != null) {
+      boolean v = ((Boolean)value).booleanValue();
+      value = Boolean.valueOf(!v);
+    }
+  }
+	
+	/**
 	 * Check if the variable contains NULL
 	 */
 	public boolean isNull() {

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
index ae7acae..aa40a0a 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
@@ -88,7 +88,7 @@ public class Function {
     if (trace && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
       trace(ctx, "FUNC " + name);      
     }
-    FuncCommand func = map.get(name);    
+    FuncCommand func = map.get(name.toUpperCase());    
     if (func != null) {
       func.run(ctx);
     }    
@@ -693,6 +693,10 @@ public class Function {
     exec.stackPush(new Var(i)); 
   }
   
+  void evalInt(int i) {
+    evalInt(new Long(i));
+  }
+  
   /**
    * Evaluate the expression to specified Date value
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
index e022024..091552f 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
@@ -18,6 +18,12 @@
 
 package org.apache.hive.hplsql.functions;
 
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.hive.hplsql.*;
 
 public class FunctionMisc extends Function {
@@ -34,11 +40,13 @@ public class FunctionMisc extends Function {
     f.map.put("DECODE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { decode(ctx); }});
     f.map.put("NVL", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { nvl(ctx); }});
     f.map.put("NVL2", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { nvl2(ctx); }});
+    f.map.put("PART_COUNT_BY", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { partCountBy(ctx); }});
     
     f.specMap.put("ACTIVITY_COUNT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { activityCount(ctx); }});
     f.specMap.put("CAST", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { cast(ctx); }});
     f.specMap.put("CURRENT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { current(ctx); }});
     f.specMap.put("CURRENT_USER", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentUser(ctx); }});
+    f.specMap.put("PART_COUNT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { partCount(ctx); }});
     f.specMap.put("USER", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentUser(ctx); }});
 
     f.specSqlMap.put("CURRENT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentSql(ctx); }});
@@ -185,4 +193,117 @@ public class FunctionMisc extends Function {
       evalNull();
     }
   }
+  
+  /**
+   * PART_COUNT function
+   */
+  public void partCount(HplsqlParser.Expr_spec_funcContext ctx) {
+    String tabname = evalPop(ctx.expr(0)).toString();
+    StringBuilder sql = new StringBuilder();
+    sql.append("SHOW PARTITIONS ");
+    sql.append(tabname);    
+    int cnt = ctx.expr().size();   
+    if (cnt > 1) {
+      sql.append(" PARTITION (");
+      int i = 1;
+      while (i + 1 < cnt) {
+        String col = evalPop(ctx.expr(i)).toString();
+        String val = evalPop(ctx.expr(i + 1)).toSqlString();
+        if (i > 2) {
+          sql.append(", ");
+        }
+        sql.append(col);
+        sql.append("=");
+        sql.append(val);        
+        i += 2;
+      }
+      sql.append(")");
+    }
+    if (trace) {
+      trace(ctx, "Query: " + sql);
+    }
+    if (exec.getOffline()) {
+      evalNull();
+      return;
+    }
+    Query query = exec.executeQuery(ctx, sql.toString(), exec.conf.defaultConnection);
+    if (query.error()) {
+      evalNullClose(query, exec.conf.defaultConnection);
+      return;
+    }
+    int result = 0;
+    ResultSet rs = query.getResultSet();
+    try {
+      while (rs.next()) {
+        result++;
+      }
+    } catch (SQLException e) {
+      evalNullClose(query, exec.conf.defaultConnection);
+      return;
+    }
+    evalInt(result);
+    exec.closeQuery(query, exec.conf.defaultConnection);
+  }
+  
+  /**
+   * PART_COUNT_BY function
+   */
+  public void partCountBy(HplsqlParser.Expr_func_paramsContext ctx) {
+    int cnt = ctx.func_param().size();
+    if (cnt < 1 || exec.getOffline()) {
+      return;
+    }
+    String tabname = evalPop(ctx.func_param(0).expr()).toString();
+    ArrayList<String> keys = null;
+    if (cnt > 1) {
+      keys = new ArrayList<String>();
+      for (int i = 1; i < cnt; i++) {
+        keys.add(evalPop(ctx.func_param(i).expr()).toString().toUpperCase());
+      }
+    }    
+    String sql = "SHOW PARTITIONS " + tabname;
+    Query query = exec.executeQuery(ctx, sql, exec.conf.defaultConnection);
+    if (query.error()) {
+      exec.closeQuery(query, exec.conf.defaultConnection);
+      return;
+    }
+    ResultSet rs = query.getResultSet();
+    HashMap<String, Integer> group = new HashMap<String, Integer>();
+    try {
+      while (rs.next()) {
+        String part = rs.getString(1);
+        String[] parts = part.split("/");
+        String key = parts[0];
+        if (cnt > 1) {
+          StringBuilder k = new StringBuilder();
+          for (int i = 0; i < parts.length; i++) {
+            if (keys.contains(parts[i].split("=")[0].toUpperCase())) {
+              if (k.length() > 0) {
+                k.append("/");
+              }
+              k.append(parts[i]);
+            }
+          }
+          key = k.toString();
+        }
+        Integer count = group.get(key);
+        if (count == null) {
+          count = new Integer(0); 
+        }
+        group.put(key, count + 1);        
+      }
+    } catch (SQLException e) {
+      exec.closeQuery(query, exec.conf.defaultConnection);
+      return;
+    }
+    if (cnt == 1) {
+      evalInt(group.size());
+    }
+    else {
+      for (Map.Entry<String, Integer> i : group.entrySet()) {
+        System.out.println(i.getKey() + '\t' + i.getValue());
+      }
+    }
+    exec.closeQuery(query, exec.conf.defaultConnection);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
----------------------------------------------------------------------
diff --git a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
index 6a67cd0..8299828 100644
--- a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
+++ b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
@@ -94,6 +94,10 @@ public class TestHplsqlLocal {
   }
   
   @Test
+  public void testCreateProcedure2() throws Exception {
+    run("create_procedure2");
+  }
+  @Test
   public void testCreateProcedureNoParams() throws Exception {
     run("create_procedure_no_params");
   }
@@ -162,8 +166,17 @@ public class TestHplsqlLocal {
   public void testIf() throws Exception {
     run("if");
   }
+  
+  @Test
+  public void testIf2() throws Exception {
+    run("if2");
+  }
 
   @Test
+  public void testInclude() throws Exception {
+    run("include");
+  }
+  @Test
   public void testInstr() throws Exception {
     run("instr");
   }
@@ -199,6 +212,11 @@ public class TestHplsqlLocal {
   }
 
   @Test
+  public void testMultDiv() throws Exception {
+    run("mult_div");
+  }
+
+  @Test
   public void testNvl() throws Exception {
     run("nvl");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java
----------------------------------------------------------------------
diff --git a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java
index eeaa395..55238ed 100644
--- a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java
+++ b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java
@@ -43,6 +43,11 @@ public class TestHplsqlOffline {
     run("create_table_ora");
   }
   
+  @Test
+  public void testSelectDb2() throws Exception {
+    run("select_db2");
+  }
+
   /**
    * Run a test file
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/cmp_row_count.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/cmp_row_count.sql b/hplsql/src/test/queries/db/cmp_row_count.sql
new file mode 100644
index 0000000..b33d841
--- /dev/null
+++ b/hplsql/src/test/queries/db/cmp_row_count.sql
@@ -0,0 +1,4 @@
+cmp row_count src, src at hive2conn;
+cmp row_count src where 1=1, src at hive2conn;
+cmp row_count (select 'A' from src), src where 2=2 at hive2conn;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/cmp_sum.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/cmp_sum.sql b/hplsql/src/test/queries/db/cmp_sum.sql
new file mode 100644
index 0000000..32347e1
--- /dev/null
+++ b/hplsql/src/test/queries/db/cmp_sum.sql
@@ -0,0 +1,3 @@
+cmp sum src_dt, src_dt at hive2conn;
+cmp sum src_dt where 1=1, src_dt at hive2conn;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/copy_to_file.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/copy_to_file.sql b/hplsql/src/test/queries/db/copy_to_file.sql
new file mode 100644
index 0000000..6135471
--- /dev/null
+++ b/hplsql/src/test/queries/db/copy_to_file.sql
@@ -0,0 +1,2 @@
+copy src to target/tmp/src.txt;
+copy (select * from src) to target/tmp/src2.txt sqlinsert src2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/copy_to_hdfs.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/copy_to_hdfs.sql b/hplsql/src/test/queries/db/copy_to_hdfs.sql
new file mode 100644
index 0000000..fd01d7b
--- /dev/null
+++ b/hplsql/src/test/queries/db/copy_to_hdfs.sql
@@ -0,0 +1,2 @@
+--copy src to hdfs src.txt;
+copy (select * from src) to hdfs /user/hplsql/src2.txt delimiter '\01';

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/copy_to_table.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/copy_to_table.sql b/hplsql/src/test/queries/db/copy_to_table.sql
new file mode 100644
index 0000000..674c0fc
--- /dev/null
+++ b/hplsql/src/test/queries/db/copy_to_table.sql
@@ -0,0 +1,2 @@
+copy src to src2 at mysqlconn;
+copy (select * from src) to src2 at mysqlconn;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/part_count.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/part_count.sql b/hplsql/src/test/queries/db/part_count.sql
new file mode 100644
index 0000000..9d62c38
--- /dev/null
+++ b/hplsql/src/test/queries/db/part_count.sql
@@ -0,0 +1,17 @@
+if part_count(partition_date_1) = 5 then
+  print 'success';
+else 
+  print 'failed';
+end if;  
+
+if part_count(partition_date_1, region='1') = 2 then
+  print 'success';
+else 
+  print 'failed';
+end if;  
+
+if part_count(partition_date_1a) is null then    -- table does not exist  
+  print 'success';
+else 
+  print 'failed';
+end if;  

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/part_count_by.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/part_count_by.sql b/hplsql/src/test/queries/db/part_count_by.sql
new file mode 100644
index 0000000..599dc5b
--- /dev/null
+++ b/hplsql/src/test/queries/db/part_count_by.sql
@@ -0,0 +1,4 @@
+part_count_by(partition_date_1); 
+part_count_by(partition_date_1, dt); 
+part_count_by(partition_date_1, dt, region); 
+part_count_by(partition_date_1, region); 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/schema.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/schema.sql b/hplsql/src/test/queries/db/schema.sql
new file mode 100644
index 0000000..0c41569
--- /dev/null
+++ b/hplsql/src/test/queries/db/schema.sql
@@ -0,0 +1,32 @@
+drop table if exists src_dt;
+
+create table src_dt (
+  c1  string,
+  c2  varchar(30),
+  c3  char(30),
+  c4  tinyint,
+  c5  smallint,
+  c6  int,
+  c7  bigint,
+  c8  decimal(19,4),
+  c9  float,
+  c10 double,
+  c11 date,
+  c12 timestamp
+);
+
+insert overwrite table src_dt
+select 
+  value c1,
+  value c2,
+  value c3,
+  cast(key as tinyint) c4,
+  cast(key as smallint) c5,
+  cast(key as int) c6,
+  cast(key as bigint) c7,
+  cast(key as decimal)/10 c8,
+  cast(key as float)/10 c9,
+  cast(key as double)/10 c10,
+  date '2015-09-07' c11,
+  cast(date '2015-09-07' as timestamp) c12
+from src;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/select_into.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/select_into.sql b/hplsql/src/test/queries/db/select_into.sql
index 3995ba2..1da610a 100644
--- a/hplsql/src/test/queries/db/select_into.sql
+++ b/hplsql/src/test/queries/db/select_into.sql
@@ -1,17 +1,33 @@
+DECLARE v_bint BIGINT;
 DECLARE v_int INT;
+DECLARE v_sint SMALLINT;
+DECLARE v_tint TINYINT;
 DECLARE v_dec DECIMAL(18,2);
 DECLARE v_dec0 DECIMAL(18,0);
+DECLARE v_str STRING;
 
 SELECT TOP 1 
+  CAST(1 AS BIGINT),
   CAST(1 AS INT), 
+  CAST(1 AS SMALLINT), 
+  CAST(1 AS TINYINT), 
   CAST(1.1 AS DECIMAL(18,2)),
   CAST(1.1 AS DECIMAL(18,0))   
 INTO 
+  v_bint,
   v_int,
+  v_sint,
+  v_tint,
   v_dec,
   v_dec0  
-FROM src ;
+FROM src;
         
+PRINT 'BIGINT: ' || v_bint;
 PRINT 'INT: ' || v_int;
+PRINT 'SMALLINT: ' || v_sint;
+PRINT 'TINYINT: ' || v_tint;
 PRINT 'DECIMAL: ' || v_dec;
-PRINT 'DECIMAL0: ' || v_dec0;
\ No newline at end of file
+PRINT 'DECIMAL0: ' || v_dec0;
+
+select 'a' into v_str from src limit 1;
+print 'string: ' || v_str;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/db/select_into2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/select_into2.sql b/hplsql/src/test/queries/db/select_into2.sql
new file mode 100644
index 0000000..e0f738c
--- /dev/null
+++ b/hplsql/src/test/queries/db/select_into2.sql
@@ -0,0 +1,17 @@
+declare v_float float;
+declare v_double double;
+declare v_double2 double precision;
+
+select
+  cast(1.1 as float),
+  cast(1.1 as double),
+  cast(1.1 as double)  
+into
+  v_float,
+  v_double,
+  v_double2
+from src limit 1;
+        
+print 'float: ' || v_float;
+print 'double: ' || v_double;
+print 'double precision: ' || v_double2;

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/local/create_procedure2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/create_procedure2.sql b/hplsql/src/test/queries/local/create_procedure2.sql
new file mode 100644
index 0000000..8875c6a
--- /dev/null
+++ b/hplsql/src/test/queries/local/create_procedure2.sql
@@ -0,0 +1,16 @@
+CREATE PROCEDURE set_message(IN name STRING, OUT result STRING)
+BEGIN
+  DECLARE str STRING DEFAULT 'Hello, ' || name || '!';
+  Work: begin
+    declare continue handler for sqlexception begin
+      set result = null;
+      print 'error';
+    end;
+    set result = str;
+  end;
+END;
+ 
+DECLARE str STRING;
+CALL set_message('world', str);
+PRINT str;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/local/if2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/if2.sql b/hplsql/src/test/queries/local/if2.sql
new file mode 100644
index 0000000..b645b86
--- /dev/null
+++ b/hplsql/src/test/queries/local/if2.sql
@@ -0,0 +1,5 @@
+if not (coalesce(1,0) between 3 and 5) then
+  print 'correct';
+else 
+  print 'failed';
+end if;

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/local/include.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/include.sql b/hplsql/src/test/queries/local/include.sql
new file mode 100644
index 0000000..c1dfb96
--- /dev/null
+++ b/hplsql/src/test/queries/local/include.sql
@@ -0,0 +1,2 @@
+include src/test/queries/local/include_file.sql
+include 'src/test/queries/local/include_file' || '.sql'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/local/include_file.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/include_file.sql b/hplsql/src/test/queries/local/include_file.sql
new file mode 100644
index 0000000..ac5e0f0
--- /dev/null
+++ b/hplsql/src/test/queries/local/include_file.sql
@@ -0,0 +1 @@
+print 'file included successfully';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/local/mult_div.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/mult_div.sql b/hplsql/src/test/queries/local/mult_div.sql
new file mode 100644
index 0000000..ebad8f4
--- /dev/null
+++ b/hplsql/src/test/queries/local/mult_div.sql
@@ -0,0 +1,8 @@
+declare a int default 8;
+declare b int default 4;
+declare c int default 2;
+
+print a/b/c;
+
+set a = 4 * 2 / cast(4 as int) /2;
+set b = 4 * 2 /cast(4 as int)/2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/queries/offline/select_db2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/offline/select_db2.sql b/hplsql/src/test/queries/offline/select_db2.sql
new file mode 100644
index 0000000..a0d2da5
--- /dev/null
+++ b/hplsql/src/test/queries/offline/select_db2.sql
@@ -0,0 +1,5 @@
+select coalesce(max(info_id)+1,0) into NextID from sproc_info with rr use and keep exclusive locks;
+
+select cd, cd + inc days, cd - inc days + coalesce(inc, 0) days
+from (select date '2015-09-02' as cd, 3 as inc from sysibm.sysdummy1);
+

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/cmp_row_count.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/cmp_row_count.out.txt b/hplsql/src/test/results/db/cmp_row_count.out.txt
new file mode 100644
index 0000000..16fadfd
--- /dev/null
+++ b/hplsql/src/test/results/db/cmp_row_count.out.txt
@@ -0,0 +1,12 @@
+Ln:1 CMP
+Ln:1 Query 1: SELECT COUNT(1) AS row_count FROM src
+Ln:1 Query 2: SELECT COUNT(1) AS row_count FROM src
+row_count	500	500
+Ln:2 CMP
+Ln:2 Query 1: SELECT COUNT(1) AS row_count FROM src where 1 = 1
+Ln:2 Query 2: SELECT COUNT(1) AS row_count FROM src
+row_count	500	500
+Ln:3 CMP
+Ln:3 Query 1: SELECT COUNT(1) AS row_count FROM (select 'A' from src) t
+Ln:3 Query 2: SELECT COUNT(1) AS row_count FROM src where 2 = 2
+row_count	500	500
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/cmp_sum.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/cmp_sum.out.txt b/hplsql/src/test/results/db/cmp_sum.out.txt
new file mode 100644
index 0000000..fad64b7
--- /dev/null
+++ b/hplsql/src/test/results/db/cmp_sum.out.txt
@@ -0,0 +1,320 @@
+Ln:1 CMP
+Ln:1 Query 1: SELECT COUNT(1) AS row_count,
+COUNT(C1) AS C1_COUNT_NOT_NULL,
+SUM(LENGTH(C1)) AS C1_SUM_LENGTH,
+MIN(LENGTH(C1)) AS C1_MIN_LENGTH,
+MAX(LENGTH(C1)) AS C1_MAX_LENGTH,
+COUNT(C2) AS C2_COUNT_NOT_NULL,
+SUM(LENGTH(C2)) AS C2_SUM_LENGTH,
+MIN(LENGTH(C2)) AS C2_MIN_LENGTH,
+MAX(LENGTH(C2)) AS C2_MAX_LENGTH,
+COUNT(C3) AS C3_COUNT_NOT_NULL,
+SUM(LENGTH(C3)) AS C3_SUM_LENGTH,
+MIN(LENGTH(C3)) AS C3_MIN_LENGTH,
+MAX(LENGTH(C3)) AS C3_MAX_LENGTH,
+COUNT(C4) AS C4_COUNT_NOT_NULL,
+SUM(C4) AS C4_SUM,
+MIN(C4) AS C4_MIN,
+MAX(C4) AS C4_MAX,
+COUNT(C5) AS C5_COUNT_NOT_NULL,
+SUM(C5) AS C5_SUM,
+MIN(C5) AS C5_MIN,
+MAX(C5) AS C5_MAX,
+COUNT(C6) AS C6_COUNT_NOT_NULL,
+SUM(C6) AS C6_SUM,
+MIN(C6) AS C6_MIN,
+MAX(C6) AS C6_MAX,
+COUNT(C7) AS C7_COUNT_NOT_NULL,
+SUM(C7) AS C7_SUM,
+MIN(C7) AS C7_MIN,
+MAX(C7) AS C7_MAX,
+COUNT(C8) AS C8_COUNT_NOT_NULL,
+SUM(C8) AS C8_SUM,
+MIN(C8) AS C8_MIN,
+MAX(C8) AS C8_MAX,
+COUNT(C9) AS C9_COUNT_NOT_NULL,
+SUM(C9) AS C9_SUM,
+MIN(C9) AS C9_MIN,
+MAX(C9) AS C9_MAX,
+COUNT(C10) AS C10_COUNT_NOT_NULL,
+SUM(C10) AS C10_SUM,
+MIN(C10) AS C10_MIN,
+MAX(C10) AS C10_MAX,
+COUNT(C11) AS C11_COUNT_NOT_NULL,
+SUM(YEAR(C11)) AS C11_SUM_YEAR,
+SUM(MONTH(C11)) AS C11_SUM_MONTH,
+SUM(DAY(C11)) AS C11_SUM_DAY,
+MIN(C11) AS C11_MIN,
+MAX(C11) AS C11_MAX,
+COUNT(C12) AS C12_COUNT_NOT_NULL,
+SUM(YEAR(C12)) AS C12_SUM_YEAR,
+SUM(MONTH(C12)) AS C12_SUM_MONTH,
+SUM(DAY(C12)) AS C12_SUM_DAY,
+MIN(C12) AS C12_MIN,
+MAX(C12) AS C12_MAX FROM src_dt
+Ln:1 Query 2: SELECT COUNT(1) AS row_count,
+COUNT(C1) AS C1_COUNT_NOT_NULL,
+SUM(LENGTH(C1)) AS C1_SUM_LENGTH,
+MIN(LENGTH(C1)) AS C1_MIN_LENGTH,
+MAX(LENGTH(C1)) AS C1_MAX_LENGTH,
+COUNT(C2) AS C2_COUNT_NOT_NULL,
+SUM(LENGTH(C2)) AS C2_SUM_LENGTH,
+MIN(LENGTH(C2)) AS C2_MIN_LENGTH,
+MAX(LENGTH(C2)) AS C2_MAX_LENGTH,
+COUNT(C3) AS C3_COUNT_NOT_NULL,
+SUM(LENGTH(C3)) AS C3_SUM_LENGTH,
+MIN(LENGTH(C3)) AS C3_MIN_LENGTH,
+MAX(LENGTH(C3)) AS C3_MAX_LENGTH,
+COUNT(C4) AS C4_COUNT_NOT_NULL,
+SUM(C4) AS C4_SUM,
+MIN(C4) AS C4_MIN,
+MAX(C4) AS C4_MAX,
+COUNT(C5) AS C5_COUNT_NOT_NULL,
+SUM(C5) AS C5_SUM,
+MIN(C5) AS C5_MIN,
+MAX(C5) AS C5_MAX,
+COUNT(C6) AS C6_COUNT_NOT_NULL,
+SUM(C6) AS C6_SUM,
+MIN(C6) AS C6_MIN,
+MAX(C6) AS C6_MAX,
+COUNT(C7) AS C7_COUNT_NOT_NULL,
+SUM(C7) AS C7_SUM,
+MIN(C7) AS C7_MIN,
+MAX(C7) AS C7_MAX,
+COUNT(C8) AS C8_COUNT_NOT_NULL,
+SUM(C8) AS C8_SUM,
+MIN(C8) AS C8_MIN,
+MAX(C8) AS C8_MAX,
+COUNT(C9) AS C9_COUNT_NOT_NULL,
+SUM(C9) AS C9_SUM,
+MIN(C9) AS C9_MIN,
+MAX(C9) AS C9_MAX,
+COUNT(C10) AS C10_COUNT_NOT_NULL,
+SUM(C10) AS C10_SUM,
+MIN(C10) AS C10_MIN,
+MAX(C10) AS C10_MAX,
+COUNT(C11) AS C11_COUNT_NOT_NULL,
+SUM(YEAR(C11)) AS C11_SUM_YEAR,
+SUM(MONTH(C11)) AS C11_SUM_MONTH,
+SUM(DAY(C11)) AS C11_SUM_DAY,
+MIN(C11) AS C11_MIN,
+MAX(C11) AS C11_MAX,
+COUNT(C12) AS C12_COUNT_NOT_NULL,
+SUM(YEAR(C12)) AS C12_SUM_YEAR,
+SUM(MONTH(C12)) AS C12_SUM_MONTH,
+SUM(DAY(C12)) AS C12_SUM_DAY,
+MIN(C12) AS C12_MIN,
+MAX(C12) AS C12_MAX FROM src_dt
+row_count	500	500
+c1_count_not_null	500	500
+c1_sum_length	3406	3406
+c1_min_length	5	5
+c1_max_length	7	7
+c2_count_not_null	500	500
+c2_sum_length	3406	3406
+c2_min_length	5	5
+c2_max_length	7	7
+c3_count_not_null	500	500
+c3_sum_length	3406	3406
+c3_min_length	5	5
+c3_max_length	7	7
+c4_count_not_null	106	106
+c4_sum	6697	6697
+c4_min	0	0
+c4_max	126	126
+c5_count_not_null	500	500
+c5_sum	130091	130091
+c5_min	0	0
+c5_max	498	498
+c6_count_not_null	500	500
+c6_sum	130091	130091
+c6_min	0	0
+c6_max	498	498
+c7_count_not_null	500	500
+c7_sum	130091	130091
+c7_min	0	0
+c7_max	498	498
+c8_count_not_null	500	500
+c8_sum	13009.1	13009.1
+c8_min	0	0
+c8_max	49.8	49.8
+c9_count_not_null	500	500
+c9_sum	13009.10001783073	13009.10001783073
+c9_min	0.0	0.0
+c9_max	49.79999923706055	49.79999923706055
+c10_count_not_null	500	500
+c10_sum	13009.09999999999	13009.09999999999
+c10_min	0.0	0.0
+c10_max	49.8	49.8
+c11_count_not_null	500	500
+c11_sum_year	1007500	1007500
+c11_sum_month	4500	4500
+c11_sum_day	3500	3500
+c11_min	null	null
+c11_max	null	null
+c12_count_not_null	500	500
+c12_sum_year	1007500	1007500
+c12_sum_month	4500	4500
+c12_sum_day	3500	3500
+c12_min	null	null
+c12_max	null	null
+Ln:2 CMP
+Ln:2 Query 1: SELECT COUNT(1) AS row_count,
+COUNT(C1) AS C1_COUNT_NOT_NULL,
+SUM(LENGTH(C1)) AS C1_SUM_LENGTH,
+MIN(LENGTH(C1)) AS C1_MIN_LENGTH,
+MAX(LENGTH(C1)) AS C1_MAX_LENGTH,
+COUNT(C2) AS C2_COUNT_NOT_NULL,
+SUM(LENGTH(C2)) AS C2_SUM_LENGTH,
+MIN(LENGTH(C2)) AS C2_MIN_LENGTH,
+MAX(LENGTH(C2)) AS C2_MAX_LENGTH,
+COUNT(C3) AS C3_COUNT_NOT_NULL,
+SUM(LENGTH(C3)) AS C3_SUM_LENGTH,
+MIN(LENGTH(C3)) AS C3_MIN_LENGTH,
+MAX(LENGTH(C3)) AS C3_MAX_LENGTH,
+COUNT(C4) AS C4_COUNT_NOT_NULL,
+SUM(C4) AS C4_SUM,
+MIN(C4) AS C4_MIN,
+MAX(C4) AS C4_MAX,
+COUNT(C5) AS C5_COUNT_NOT_NULL,
+SUM(C5) AS C5_SUM,
+MIN(C5) AS C5_MIN,
+MAX(C5) AS C5_MAX,
+COUNT(C6) AS C6_COUNT_NOT_NULL,
+SUM(C6) AS C6_SUM,
+MIN(C6) AS C6_MIN,
+MAX(C6) AS C6_MAX,
+COUNT(C7) AS C7_COUNT_NOT_NULL,
+SUM(C7) AS C7_SUM,
+MIN(C7) AS C7_MIN,
+MAX(C7) AS C7_MAX,
+COUNT(C8) AS C8_COUNT_NOT_NULL,
+SUM(C8) AS C8_SUM,
+MIN(C8) AS C8_MIN,
+MAX(C8) AS C8_MAX,
+COUNT(C9) AS C9_COUNT_NOT_NULL,
+SUM(C9) AS C9_SUM,
+MIN(C9) AS C9_MIN,
+MAX(C9) AS C9_MAX,
+COUNT(C10) AS C10_COUNT_NOT_NULL,
+SUM(C10) AS C10_SUM,
+MIN(C10) AS C10_MIN,
+MAX(C10) AS C10_MAX,
+COUNT(C11) AS C11_COUNT_NOT_NULL,
+SUM(YEAR(C11)) AS C11_SUM_YEAR,
+SUM(MONTH(C11)) AS C11_SUM_MONTH,
+SUM(DAY(C11)) AS C11_SUM_DAY,
+MIN(C11) AS C11_MIN,
+MAX(C11) AS C11_MAX,
+COUNT(C12) AS C12_COUNT_NOT_NULL,
+SUM(YEAR(C12)) AS C12_SUM_YEAR,
+SUM(MONTH(C12)) AS C12_SUM_MONTH,
+SUM(DAY(C12)) AS C12_SUM_DAY,
+MIN(C12) AS C12_MIN,
+MAX(C12) AS C12_MAX FROM src_dt where 1 = 1
+Ln:2 Query 2: SELECT COUNT(1) AS row_count,
+COUNT(C1) AS C1_COUNT_NOT_NULL,
+SUM(LENGTH(C1)) AS C1_SUM_LENGTH,
+MIN(LENGTH(C1)) AS C1_MIN_LENGTH,
+MAX(LENGTH(C1)) AS C1_MAX_LENGTH,
+COUNT(C2) AS C2_COUNT_NOT_NULL,
+SUM(LENGTH(C2)) AS C2_SUM_LENGTH,
+MIN(LENGTH(C2)) AS C2_MIN_LENGTH,
+MAX(LENGTH(C2)) AS C2_MAX_LENGTH,
+COUNT(C3) AS C3_COUNT_NOT_NULL,
+SUM(LENGTH(C3)) AS C3_SUM_LENGTH,
+MIN(LENGTH(C3)) AS C3_MIN_LENGTH,
+MAX(LENGTH(C3)) AS C3_MAX_LENGTH,
+COUNT(C4) AS C4_COUNT_NOT_NULL,
+SUM(C4) AS C4_SUM,
+MIN(C4) AS C4_MIN,
+MAX(C4) AS C4_MAX,
+COUNT(C5) AS C5_COUNT_NOT_NULL,
+SUM(C5) AS C5_SUM,
+MIN(C5) AS C5_MIN,
+MAX(C5) AS C5_MAX,
+COUNT(C6) AS C6_COUNT_NOT_NULL,
+SUM(C6) AS C6_SUM,
+MIN(C6) AS C6_MIN,
+MAX(C6) AS C6_MAX,
+COUNT(C7) AS C7_COUNT_NOT_NULL,
+SUM(C7) AS C7_SUM,
+MIN(C7) AS C7_MIN,
+MAX(C7) AS C7_MAX,
+COUNT(C8) AS C8_COUNT_NOT_NULL,
+SUM(C8) AS C8_SUM,
+MIN(C8) AS C8_MIN,
+MAX(C8) AS C8_MAX,
+COUNT(C9) AS C9_COUNT_NOT_NULL,
+SUM(C9) AS C9_SUM,
+MIN(C9) AS C9_MIN,
+MAX(C9) AS C9_MAX,
+COUNT(C10) AS C10_COUNT_NOT_NULL,
+SUM(C10) AS C10_SUM,
+MIN(C10) AS C10_MIN,
+MAX(C10) AS C10_MAX,
+COUNT(C11) AS C11_COUNT_NOT_NULL,
+SUM(YEAR(C11)) AS C11_SUM_YEAR,
+SUM(MONTH(C11)) AS C11_SUM_MONTH,
+SUM(DAY(C11)) AS C11_SUM_DAY,
+MIN(C11) AS C11_MIN,
+MAX(C11) AS C11_MAX,
+COUNT(C12) AS C12_COUNT_NOT_NULL,
+SUM(YEAR(C12)) AS C12_SUM_YEAR,
+SUM(MONTH(C12)) AS C12_SUM_MONTH,
+SUM(DAY(C12)) AS C12_SUM_DAY,
+MIN(C12) AS C12_MIN,
+MAX(C12) AS C12_MAX FROM src_dt
+row_count	500	500
+c1_count_not_null	500	500
+c1_sum_length	3406	3406
+c1_min_length	5	5
+c1_max_length	7	7
+c2_count_not_null	500	500
+c2_sum_length	3406	3406
+c2_min_length	5	5
+c2_max_length	7	7
+c3_count_not_null	500	500
+c3_sum_length	3406	3406
+c3_min_length	5	5
+c3_max_length	7	7
+c4_count_not_null	106	106
+c4_sum	6697	6697
+c4_min	0	0
+c4_max	126	126
+c5_count_not_null	500	500
+c5_sum	130091	130091
+c5_min	0	0
+c5_max	498	498
+c6_count_not_null	500	500
+c6_sum	130091	130091
+c6_min	0	0
+c6_max	498	498
+c7_count_not_null	500	500
+c7_sum	130091	130091
+c7_min	0	0
+c7_max	498	498
+c8_count_not_null	500	500
+c8_sum	13009.1	13009.1
+c8_min	0	0
+c8_max	49.8	49.8
+c9_count_not_null	500	500
+c9_sum	13009.10001783073	13009.10001783073
+c9_min	0.0	0.0
+c9_max	49.79999923706055	49.79999923706055
+c10_count_not_null	500	500
+c10_sum	13009.09999999999	13009.09999999999
+c10_min	0.0	0.0
+c10_max	49.8	49.8
+c11_count_not_null	500	500
+c11_sum_year	1007500	1007500
+c11_sum_month	4500	4500
+c11_sum_day	3500	3500
+c11_min	null	null
+c11_max	null	null
+c12_count_not_null	500	500
+c12_sum_year	1007500	1007500
+c12_sum_month	4500	4500
+c12_sum_day	3500	3500
+c12_min	null	null
+c12_max	null	null
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/copy_to_file.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/copy_to_file.out.txt b/hplsql/src/test/results/db/copy_to_file.out.txt
new file mode 100644
index 0000000..e571d36
--- /dev/null
+++ b/hplsql/src/test/results/db/copy_to_file.out.txt
@@ -0,0 +1,6 @@
+Ln:1 COPY
+Ln:1 Query executed: 2 columns, output file: target/tmp/src.txt
+Ln:2 COPY
+Ln:2 Statement:
+select * from src
+Ln:2 Query executed: 2 columns, output file: target/tmp/src2.txt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/copy_to_hdfs.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/copy_to_hdfs.out.txt b/hplsql/src/test/results/db/copy_to_hdfs.out.txt
new file mode 100644
index 0000000..23c0cb2
--- /dev/null
+++ b/hplsql/src/test/results/db/copy_to_hdfs.out.txt
@@ -0,0 +1,4 @@
+Ln:2 COPY
+Ln:2 Statement:
+select * from src
+Ln:2 Query executed: 2 columns, output file: /user/hplsql/src2.txt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/copy_to_table.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/copy_to_table.out.txt b/hplsql/src/test/results/db/copy_to_table.out.txt
new file mode 100644
index 0000000..411b425
--- /dev/null
+++ b/hplsql/src/test/results/db/copy_to_table.out.txt
@@ -0,0 +1,2 @@
+Ln:1 COPY
+Ln:1 SELECT executed: 2 columns
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/part_count.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/part_count.out.txt b/hplsql/src/test/results/db/part_count.out.txt
new file mode 100644
index 0000000..485ffe1
--- /dev/null
+++ b/hplsql/src/test/results/db/part_count.out.txt
@@ -0,0 +1,15 @@
+Ln:1 IF
+Ln:1 Query: SHOW PARTITIONS partition_date_1
+Ln:1 IF TRUE executed
+Ln:2 PRINT
+success
+Ln:7 IF
+Ln:7 Query: SHOW PARTITIONS partition_date_1 PARTITION (region='1')
+Ln:7 IF TRUE executed
+Ln:8 PRINT
+success
+Ln:13 IF
+Ln:13 Query: SHOW PARTITIONS partition_date_1a
+Ln:13 IF TRUE executed
+Ln:14 PRINT
+success
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/part_count_by.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/part_count_by.out.txt b/hplsql/src/test/results/db/part_count_by.out.txt
new file mode 100644
index 0000000..61f51cd
--- /dev/null
+++ b/hplsql/src/test/results/db/part_count_by.out.txt
@@ -0,0 +1,13 @@
+3
+dt=2000-01-01	2
+dt=2013-12-10	1
+dt=2013-08-08	2
+dt=2013-08-08/region=1	1
+dt=2000-01-01/region=1	1
+dt=2013-12-10/region=2020-20-20	1
+dt=2000-01-01/region=2	1
+dt=2013-08-08/region=10	1
+region=10	1
+region=2020-20-20	1
+region=2	1
+region=1	2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/select_into.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/select_into.out.txt b/hplsql/src/test/results/db/select_into.out.txt
index 80d067e..3f4ae31 100644
--- a/hplsql/src/test/results/db/select_into.out.txt
+++ b/hplsql/src/test/results/db/select_into.out.txt
@@ -1,19 +1,43 @@
-Ln:1 DECLARE v_int INT
-Ln:2 DECLARE v_dec DECIMAL
-Ln:3 DECLARE v_dec0 DECIMAL
-Ln:5 SELECT
-Ln:5 SELECT CAST(1 AS INT), CAST(1.1 AS DECIMAL(18,2)), CAST(1.1 AS DECIMAL(18,0)) FROM src LIMIT 1
-Ln:5 SELECT completed successfully
-Ln:5 SELECT INTO statement executed
-Ln:5 COLUMN: _c0, int
-Ln:5 SET v_int = 1
-Ln:5 COLUMN: _c1, decimal
-Ln:5 SET v_dec = 1.1
-Ln:5 COLUMN: _c2, decimal
-Ln:5 SET v_dec0 = 1
-Ln:15 PRINT
+Ln:1 DECLARE v_bint BIGINT
+Ln:2 DECLARE v_int INT
+Ln:3 DECLARE v_sint SMALLINT
+Ln:4 DECLARE v_tint TINYINT
+Ln:5 DECLARE v_dec DECIMAL
+Ln:6 DECLARE v_dec0 DECIMAL
+Ln:7 DECLARE v_str STRING
+Ln:9 SELECT
+Ln:9 SELECT CAST(1 AS BIGINT), CAST(1 AS INT), CAST(1 AS SMALLINT), CAST(1 AS TINYINT), CAST(1.1 AS DECIMAL(18,2)), CAST(1.1 AS DECIMAL(18,0)) FROM src LIMIT 1
+Ln:9 SELECT completed successfully
+Ln:9 SELECT INTO statement executed
+Ln:9 COLUMN: _c0, bigint
+Ln:9 SET v_bint = 1
+Ln:9 COLUMN: _c1, int
+Ln:9 SET v_int = 1
+Ln:9 COLUMN: _c2, smallint
+Ln:9 SET v_sint = 1
+Ln:9 COLUMN: _c3, tinyint
+Ln:9 SET v_tint = 1
+Ln:9 COLUMN: _c4, decimal
+Ln:9 SET v_dec = 1.1
+Ln:9 COLUMN: _c5, decimal
+Ln:9 SET v_dec0 = 1
+Ln:25 PRINT
+BIGINT: 1
+Ln:26 PRINT
 INT: 1
-Ln:16 PRINT
+Ln:27 PRINT
+SMALLINT: 1
+Ln:28 PRINT
+TINYINT: 1
+Ln:29 PRINT
 DECIMAL: 1.1
-Ln:17 PRINT
-DECIMAL0: 1
\ No newline at end of file
+Ln:30 PRINT
+DECIMAL0: 1
+Ln:32 SELECT
+Ln:32 select 'a' from src LIMIT 1
+Ln:32 SELECT completed successfully
+Ln:32 SELECT INTO statement executed
+Ln:32 COLUMN: _c0, string
+Ln:32 SET v_str = a
+Ln:33 PRINT
+string: a
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/db/select_into2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/select_into2.out.txt b/hplsql/src/test/results/db/select_into2.out.txt
new file mode 100644
index 0000000..03e67ad
--- /dev/null
+++ b/hplsql/src/test/results/db/select_into2.out.txt
@@ -0,0 +1,19 @@
+Ln:1 DECLARE v_float float
+Ln:2 DECLARE v_double double
+Ln:3 DECLARE v_double2 double precision
+Ln:5 SELECT
+Ln:5 select cast(1.1 as float), cast(1.1 as double), cast(1.1 as double) from src LIMIT 1
+Ln:5 SELECT completed successfully
+Ln:5 SELECT INTO statement executed
+Ln:5 COLUMN: _c0, float
+Ln:5 SET v_float = 1.100000023841858
+Ln:5 COLUMN: _c1, double
+Ln:5 SET v_double = 1.1
+Ln:5 COLUMN: _c2, double
+Ln:5 SET v_double2 = 1.1
+Ln:15 PRINT
+float: 1.100000023841858
+Ln:16 PRINT
+double: 1.1
+Ln:17 PRINT
+double precision: 1.1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/local/create_procedure2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/create_procedure2.out.txt b/hplsql/src/test/results/local/create_procedure2.out.txt
new file mode 100644
index 0000000..765faa9
--- /dev/null
+++ b/hplsql/src/test/results/local/create_procedure2.out.txt
@@ -0,0 +1,10 @@
+Ln:1 CREATE PROCEDURE set_message
+Ln:13 DECLARE str STRING
+Ln:14 EXEC PROCEDURE set_message
+Ln:14 SET PARAM name = world
+Ln:14 SET PARAM result = null
+Ln:3 DECLARE str STRING = 'Hello, world!'
+Ln:5 DECLARE HANDLER
+Ln:9 SET result = 'Hello, world!'
+Ln:15 PRINT
+Hello, world!
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/local/if2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/if2.out.txt b/hplsql/src/test/results/local/if2.out.txt
new file mode 100644
index 0000000..63a6213
--- /dev/null
+++ b/hplsql/src/test/results/local/if2.out.txt
@@ -0,0 +1,4 @@
+Ln:1 IF
+Ln:1 IF TRUE executed
+Ln:2 PRINT
+correct
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/local/include.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/include.out.txt b/hplsql/src/test/results/local/include.out.txt
new file mode 100644
index 0000000..86cfa05
--- /dev/null
+++ b/hplsql/src/test/results/local/include.out.txt
@@ -0,0 +1,8 @@
+Ln:1 INCLUDE src/test/queries/local/include_file.sql
+INLCUDE CONTENT src/test/queries/local/include_file.sql (non-empty)
+Ln:1 PRINT
+file included successfully
+Ln:2 INCLUDE src/test/queries/local/include_file.sql
+INLCUDE CONTENT src/test/queries/local/include_file.sql (non-empty)
+Ln:1 PRINT
+file included successfully
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/06790789/hplsql/src/test/results/local/mult_div.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/mult_div.out.txt b/hplsql/src/test/results/local/mult_div.out.txt
new file mode 100644
index 0000000..cd17c16
--- /dev/null
+++ b/hplsql/src/test/results/local/mult_div.out.txt
@@ -0,0 +1,7 @@
+Ln:1 DECLARE a int = 8
+Ln:2 DECLARE b int = 4
+Ln:3 DECLARE c int = 2
+Ln:5 PRINT
+1
+Ln:7 SET a = 1
+Ln:8 SET b = 1
\ No newline at end of file


[42/52] [abbrv] hive git commit: HIVE-11692: Fix UT regressions on hbase-metastore branch (Daniel Dai reviewed by Thejas Nair)

Posted by se...@apache.org.
HIVE-11692: Fix UT regressions on hbase-metastore branch (Daniel Dai reviewed by Thejas Nair)


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

Branch: refs/heads/llap
Commit: 8b0ededf574ff33c7fe4a952aad42ece1467237d
Parents: 3d170ca
Author: Daniel Dai <da...@hortonworks.com>
Authored: Tue Sep 1 13:18:35 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Tue Sep 1 13:18:35 2015 -0700

----------------------------------------------------------------------
 data/conf/hbase/hive-site.xml                   | 263 -------------------
 .../hive/metastore/TestHiveMetaStore.java       |   3 +
 .../hive/metastore/hbase/TestHBaseImport.java   |  18 +-
 itests/qtest/pom.xml                            |  19 --
 .../test/resources/testconfiguration.properties |  46 ----
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   4 +
 metastore/pom.xml                               |   6 +
 .../hadoop/hive/metastore/TestObjectStore.java  |  43 ++-
 .../dynpart_sort_opt_vectorization.q.out        |  12 +-
 .../dynpart_sort_optimization.q.out             |  12 +-
 10 files changed, 76 insertions(+), 350 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/data/conf/hbase/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/hbase/hive-site.xml b/data/conf/hbase/hive-site.xml
deleted file mode 100644
index 2cde40f..0000000
--- a/data/conf/hbase/hive-site.xml
+++ /dev/null
@@ -1,263 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-
-<configuration>
-
-<property>
-  <name>hive.in.test</name>
-  <value>true</value>
-  <description>Internal marker for test. Used for masking env-dependent values</description>
-</property>
-
-<!-- Hive Configuration can either be stored in this file or in the hadoop configuration files  -->
-<!-- that are implied by Hadoop setup variables.                                                -->
-<!-- Aside from Hadoop setup variables - this file is provided as a convenience so that Hive    -->
-<!-- users do not have to edit hadoop configuration files (that may be managed as a centralized -->
-<!-- resource).                                                                                 -->
-
-<!-- Hive Execution Parameters -->
-<property>
-  <name>hadoop.tmp.dir</name>
-  <value>${test.tmp.dir}/hadoop-tmp</value>
-  <description>A base for other temporary directories.</description>
-</property>
-
-<!--
-<property>
-  <name>hive.exec.reducers.max</name>
-  <value>1</value>
-  <description>maximum number of reducers</description>
-</property>
--->
-
-<property>
-  <name>hive.exec.scratchdir</name>
-  <value>${test.tmp.dir}/scratchdir</value>
-  <description>Scratch space for Hive jobs</description>
-</property>
-
-<property>
-  <name>hive.exec.local.scratchdir</name>
-  <value>${test.tmp.dir}/localscratchdir/</value>
-  <description>Local scratch space for Hive jobs</description>
-</property>
-
-<property>
-  <name>javax.jdo.option.ConnectionURL</name>
-  <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
-</property>
-
-<property>
-  <name>hive.stats.dbconnectionstring</name>
-  <value>jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true</value>
-</property>
-
-
-<property>
-  <name>javax.jdo.option.ConnectionDriverName</name>
-  <value>org.apache.derby.jdbc.EmbeddedDriver</value>
-</property>
-
-<property>
-  <name>javax.jdo.option.ConnectionUserName</name>
-  <value>APP</value>
-</property>
-
-<property>
-  <name>javax.jdo.option.ConnectionPassword</name>
-  <value>mine</value>
-</property>
-
-<property>
-  <!--  this should eventually be deprecated since the metastore should supply this -->
-  <name>hive.metastore.warehouse.dir</name>
-  <value>${test.warehouse.dir}</value>
-  <description></description>
-</property>
-
-<property>
-  <name>hive.metastore.metadb.dir</name>
-  <value>file://${test.tmp.dir}/metadb/</value>
-  <description>
-  Required by metastore server or if the uris argument below is not supplied
-  </description>
-</property>
-
-<property>
-  <name>test.log.dir</name>
-  <value>${test.tmp.dir}/log/</value>
-  <description></description>
-</property>
-
-<property>
-  <name>test.data.files</name>
-  <value>${hive.root}/data/files</value>
-  <description></description>
-</property>
-
-<property>
-  <name>test.data.scripts</name>
-  <value>${hive.root}/data/scripts</value>
-  <description></description>
-</property>
-
-<property>
-  <name>hive.jar.path</name>
-  <value>${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar</value>
-  <description></description>
-</property>
-
-<property>
-  <name>hive.querylog.location</name>
-  <value>${test.tmp.dir}/tmp</value>
-  <description>Location of the structured hive logs</description>
-</property>
-
-<property>
-  <name>hive.exec.pre.hooks</name>
-  <value>org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables</value>
-  <description>Pre Execute Hook for Tests</description>
-</property>
-
-<property>
-  <name>hive.exec.post.hooks</name>
-  <value>org.apache.hadoop.hive.ql.hooks.PostExecutePrinter</value>
-  <description>Post Execute Hook for Tests</description>
-</property>
-
-<property>
-  <name>hive.support.concurrency</name>
-  <value>false</value>
-  <description>Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks.</description>
-</property>
-
-<property>
-  <key>hive.unlock.numretries</key>
-  <value>2</value>
-  <description>The number of times you want to retry to do one unlock</description>
-</property>
-
-<property>
-  <key>hive.lock.sleep.between.retries</key>
-  <value>2</value>
-  <description>The sleep time (in seconds) between various retries</description>
-</property>
-
-
-<property>
-  <name>fs.pfile.impl</name>
-  <value>org.apache.hadoop.fs.ProxyLocalFileSystem</value>
-  <description>A proxy for local file system used for cross file system testing</description>
-</property>
-
-<property>
-  <name>hive.exec.mode.local.auto</name>
-  <value>false</value>
-  <description>
-    Let hive determine whether to run in local mode automatically
-    Disabling this for tests so that minimr is not affected
-  </description>
-</property>
-
-<property>
-  <name>hive.auto.convert.join</name>
-  <value>false</value>
-  <description>Whether Hive enable the optimization about converting common join into mapjoin based on the input file size</description>
-</property>
-
-<property>
-  <name>hive.ignore.mapjoin.hint</name>
-  <value>false</value>
-  <description>Whether Hive ignores the mapjoin hint</description>
-</property>
-
-<property>
-  <name>hive.input.format</name>
-  <value>org.apache.hadoop.hive.ql.io.CombineHiveInputFormat</value>
-  <description>The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. </description>
-</property>
-
-<property>
-  <name>hive.default.rcfile.serde</name>
-  <value>org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe</value>
-  <description>The default SerDe hive will use for the rcfile format</description>
-</property>
-
-<property>
-  <name>hive.stats.dbclass</name>
-  <value>jdbc:derby</value>
-  <description>The storage for temporary stats generated by tasks. Currently, jdbc, hbase and counter types are supported</description>
-</property>
-
-<property>
-  <name>hive.stats.key.prefix.reserve.length</name>
-  <value>0</value>
-</property>
-
-<property>
-  <name>hive.conf.restricted.list</name>
-  <value>dummy.config.value</value>
-  <description>Using dummy config value above because you cannot override config with empty value</description>
-</property>
-
-<property>
-  <name>hive.exec.submit.local.task.via.child</name>
-  <value>false</value>
-</property>
-
-
-<property>
-  <name>hive.dummyparam.test.server.specific.config.override</name>
-  <value>from.hive-site.xml</value>
-  <description>Using dummy param to test server specific configuration</description>
-</property>
-
-<property>
-  <name>hive.dummyparam.test.server.specific.config.hivesite</name>
-  <value>from.hive-site.xml</value>
-  <description>Using dummy param to test server specific configuration</description>
-</property>
-
-<property>
-  <name>hive.ql.log.PerfLogger.level</name>
-  <value>WARN,DRFA</value>
-  <description>Used to change the perflogger level</description>
-</property>
-
-<property>
-  <name>hive.fetch.task.conversion</name>
-  <value>minimal</value>
-</property>
-
-<property>
-  <name>hive.users.in.admin.role</name>
-  <value>hive_admin_user</value>
-</property>
-
-<property>
-  <name>hive.metastore.fastpath</name>
-  <value>true</value>
-</property>
-
-<property>
-  <name>hive.metastore.rawstore.impl</name>
-  <value>org.apache.hadoop.hive.metastore.hbase.HBaseStore</value>
-</property>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 160667d..06061c0 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -2541,6 +2541,9 @@ public abstract class TestHiveMetaStore extends TestCase {
 
     try {
       cleanUp(dbName, null, null);
+      for (Function f : client.getAllFunctions().getFunctions()) {
+        client.dropFunction(f.getDbName(), f.getFunctionName());
+      }
 
       createDb(dbName);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
index 1ac10f0..2d2bd46 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
@@ -22,6 +22,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.metastore.ObjectStore;
 import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.TestObjectStore;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Function;
@@ -57,22 +58,34 @@ import java.util.Set;
  */
 public class TestHBaseImport extends HBaseIntegrationTests {
 
-  private static final Log LOG = LogFactory.getLog(TestHBaseStoreIntegration.class.getName());
+  private static final Log LOG = LogFactory.getLog(TestHBaseImport.class.getName());
 
   private static final String[] tableNames = new String[] {"allnonparttable", "allparttable"};
   private static final String[] partVals = new String[] {"na", "emea", "latam", "apac"};
   private static final String[] funcNames = new String[] {"allfunc1", "allfunc2"};
 
+  private static final List<Integer> masterKeySeqs = new ArrayList<Integer>();
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @BeforeClass
   public static void startup() throws Exception {
     HBaseIntegrationTests.startMiniCluster();
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+    TestObjectStore.dropAllStoreObjects(rdbms);
   }
 
   @AfterClass
   public static void shutdown() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+    TestObjectStore.dropAllStoreObjects(rdbms);
+    for (int seq : masterKeySeqs) {
+      rdbms.removeMasterKey(seq);
+    }
     HBaseIntegrationTests.shutdownMiniCluster();
   }
 
@@ -316,7 +329,6 @@ public class TestHBaseImport extends HBaseIntegrationTests {
     Assert.assertEquals(baseNumToks, store.getAllTokenIdentifiers().size());
     String[] hbaseKeys = store.getMasterKeys();
     Assert.assertEquals(baseNumKeys, hbaseKeys.length);
-
   }
 
   @Test
@@ -502,7 +514,7 @@ public class TestHBaseImport extends HBaseIntegrationTests {
     }
     for (int i = 0; i < tokenIds.length; i++) rdbms.addToken(tokenIds[i], tokens[i]);
     for (int i = 0; i < masterKeys.length; i++) {
-      rdbms.addMasterKey(masterKeys[i]);
+      masterKeySeqs.add(rdbms.addMasterKey(masterKeys[i]));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index 664068b..122e3f6 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -533,25 +533,6 @@
                   <else>
                   </else>
                 </if>
-                <!-- HBase Metastore -->
-                <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
-                          outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hadoop/hive/cli/"
-                          templatePath="${basedir}/${hive.path.to.root}/ql/src/test/templates/" template="TestCliDriver.vm"
-                          queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientpositive/"
-                          queryFile="${qfile}"
-                          excludeQueryFile="${minimr.query.files},${minitez.query.files},${encrypted.query.files}"
-                          includeQueryFile="${miniHbaseMetastore.query.files}"
-                          queryFileRegex="${qfile_regex}"
-                          clusterMode="${clustermode}"
-                          runDisabled="${run_disabled}"
-                          hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/hbase"
-                          resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/" 
-                          className="TestMiniHBaseMetastoreCliDriver"
-                          logFile="${project.build.directory}/testminihbasemetastoreclidrivergen.log"
-                          logDirectory="${project.build.directory}/qfile-results/clientpositive/"
-                          hadoopVersion="${active.hadoop.version}"
-                          initScript="q_test_init.sql"
-                          cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- Negative Minimr -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index c877f85..bed621d 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1184,52 +1184,6 @@ miniSparkOnYarn.query.files=auto_sortmerge_join_16.q,\
   truncate_column_buckets.q,\
   uber_reduce.q
 
-miniHbaseMetastore.query.files=join1.q,\
-join2.q,\
-mapjoin1.q,\
-add_part_multiple.q,\
-annotate_stats_join.q,\
-authorization_parts.q,\
-auto_join1.q,\
-bucket1.q,\
-compute_stats_string.q,\
-create_1.q,\
-groupby1.q,\
-groupby12.q,\
-having.q,\
-innerjoin.q,\
-input_part10.q,\
-input20.q,\
-join1.q,\
-join20.q,\
-leftsemijoin.q,\
-mapjoin1.q,\
-multi_insert_gby.q,\
-orc_create.q,\
-orc_merge1.q,\
-show_roles.q,\
-stats0.q,\
-statsfs.q,\
-temp_table.q,\
-union.q,\
-union10.q,\
-alter_partition_change_col,\
-alter1.q,\
-analyze_tbl_part.q,\
-authorization_1.q,\
-columnstats_part_coltype.q,\
-ctas.q,\
-database.q,\
-drop_partition_with_stats.q,\
-drop_table_with_stats.q,\
-inputddl8.q,\
-order2.q,\
-partition_date.q,\
-partition_multilevels.q,\
-show_partitions.q,\
-sort.q,\
-view.q
-
 spark.query.negative.files=groupby2_map_skew_multi_distinct.q,\
   groupby2_multi_distinct.q,\
   groupby3_map_skew_multi_distinct.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 9f112ad..70df41e 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -288,6 +288,10 @@ public class QTestUtil {
       conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED, true);
     }
 
+    // Plug verifying metastore in for testing.
+    conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
+      "org.apache.hadoop.hive.metastore.VerifyingObjectStore");
+
     if (mr != null) {
       assert dfs != null;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index 255726c..2ab5cd3 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -135,6 +135,12 @@
       <groupId>co.cask.tephra</groupId>
       <artifactId>tephra-core</artifactId>
       <version>${tephra.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.ow2.asm</groupId>
+          <artifactId>asm-all</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>co.cask.tephra</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index a4f9f6c..78a9ea0 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -69,22 +70,13 @@ public class TestObjectStore {
   }
 
   @Before
-  public void setUp() {
+  public void setUp() throws Exception {
     HiveConf conf = new HiveConf();
     conf.setVar(HiveConf.ConfVars.METASTORE_EXPRESSION_PROXY_CLASS, MockPartitionExpressionProxy.class.getName());
 
     objectStore = new ObjectStore();
     objectStore.setConf(conf);
-
-    Deadline.registerIfNot(100000);
-    try {
-      objectStore.dropDatabase(DB1);
-    } catch (Exception e) {
-    }
-    try {
-      objectStore.dropDatabase(DB2);
-    } catch (Exception e) {
-    }
+    dropAllStoreObjects(objectStore);
   }
 
   @After
@@ -227,4 +219,33 @@ public class TestObjectStore {
     objectStore.revokeRole(role1, USER1, PrincipalType.USER, false);
     objectStore.removeRole(ROLE1);
   }
+
+  public static void dropAllStoreObjects(RawStore store) throws MetaException, InvalidObjectException, InvalidInputException {
+    try {
+      Deadline.registerIfNot(100000);
+      List<Function> funcs = store.getAllFunctions();
+      for (Function func : funcs) {
+        store.dropFunction(func.getDbName(), func.getFunctionName());
+      }
+      List<String> dbs = store.getAllDatabases();
+      for (int i = 0; i < dbs.size(); i++) {
+        String db = dbs.get(i);
+        List<String> tbls = store.getAllTables(db);
+        for (String tbl : tbls) {
+          Deadline.startTimer("getPartition");
+          List<Partition> parts = store.getPartitions(db, tbl, 100);
+          for (Partition part : parts) {
+            store.dropPartition(db, tbl, part.getValues());
+          }
+          store.dropTable(db, tbl);
+        }
+        store.dropDatabase(db);
+      }
+      List<String> roles = store.listRoleNames();
+      for (String role : roles) {
+        store.removeRole(role);
+      }
+    } catch (NoSuchObjectException e) {
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
index 216a79c..1f6339a 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
@@ -1755,13 +1755,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2_orc
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2_orc
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2_orc
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2_orc
 POSTHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
@@ -1900,13 +1902,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2_orc
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2_orc
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2_orc
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2_orc
 POSTHOOK: Input: default@over1k_part2_orc@ds=foo/t=27

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
index 41049bd..ebf4461 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
@@ -1655,13 +1655,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2
 PREHOOK: Input: default@over1k_part2@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2
 POSTHOOK: Input: default@over1k_part2@ds=foo/t=27
@@ -1800,13 +1802,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2
 PREHOOK: Input: default@over1k_part2@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2
 POSTHOOK: Input: default@over1k_part2@ds=foo/t=27


[51/52] [abbrv] hive git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/hive

Posted by se...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/hive


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

Branch: refs/heads/llap
Commit: 451381cfb28d1b9ff2775d55d92924956b3a5cdb
Parents: 0679078 5238303
Author: Dmitry Tolpeko <dm...@gmail.com>
Authored: Tue Sep 22 06:41:46 2015 -0700
Committer: Dmitry Tolpeko <dm...@gmail.com>
Committed: Tue Sep 22 06:41:46 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ant/QTestGenTask.java    |    11 +
 beeline/src/main/resources/beeline-log4j2.xml   |     5 +-
 bin/beeline                                     |     5 +
 bin/ext/hbaseimport.cmd                         |    35 +
 bin/ext/hbaseimport.sh                          |    27 +
 bin/ext/hbaseschematool.sh                      |    27 +
 .../apache/hadoop/hive/common/ObjectPair.java   |     5 +
 .../hadoop/hive/common/jsonexplain/tez/Op.java  |     8 +-
 .../hive/common/jsonexplain/tez/Stage.java      |    14 +-
 .../common/jsonexplain/tez/TezJsonParser.java   |    17 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   111 +-
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |   196 +
 .../apache/hive/common/util/BloomFilter.java    |    20 +-
 common/src/main/resources/hive-log4j2.xml       |     5 +-
 .../test/resources/hive-exec-log4j2-test.xml    |     5 +-
 common/src/test/resources/hive-log4j2-test.xml  |     5 +-
 data/conf/hive-log4j2.xml                       |     5 +-
 data/conf/tez/hive-site.xml                     |    10 +
 data/files/dynpartdata1.txt                     |     5 +
 data/files/dynpartdata2.txt                     |     6 +
 .../HiveHBaseTableSnapshotInputFormat.java      |    21 +-
 .../queries/positive/hbase_handler_snapshot.q   |     4 +
 .../positive/hbase_handler_snapshot.q.out       |    22 +
 .../deployers/config/hive/hive-log4j2.xml       |     5 +-
 .../svr/src/main/config/webhcat-log4j2.xml      |     5 +-
 .../benchmark/serde/LazySimpleSerDeBench.java   |   453 +
 .../hive/thrift/TestHadoop20SAuthBridge.java    |   420 -
 .../hive/thrift/TestHadoopAuthBridge23.java     |   423 +
 itests/hive-unit/pom.xml                        |    35 +
 .../hadoop/hive/metastore/TestAdminUser.java    |     4 +-
 .../hive/metastore/TestHiveMetaStore.java       |     3 +
 .../metastore/hbase/HBaseIntegrationTests.java  |   117 +
 .../TestHBaseAggrStatsCacheIntegration.java     |   691 +
 .../hive/metastore/hbase/TestHBaseImport.java   |   650 +
 .../metastore/hbase/TestHBaseMetastoreSql.java  |   223 +
 .../hbase/TestHBaseStoreIntegration.java        |  1794 +
 .../hbase/TestStorageDescriptorSharing.java     |   191 +
 .../hive/ql/security/FolderPermissionBase.java  |    17 +-
 itests/qtest/pom.xml                            |    10 +-
 .../test/resources/testconfiguration.properties |     3 +
 itests/util/pom.xml                             |    32 +
 .../metastore/hbase/HBaseStoreTestUtil.java     |    45 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    50 +-
 .../hive/jdbc/ZooKeeperHiveClientHelper.java    |    32 +-
 metastore/if/hive_metastore.thrift              |    54 +
 metastore/pom.xml                               |    82 +
 .../metastore/hbase/HbaseMetastoreProto.java    | 34901 +++++++++++++++++
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  6919 ++--
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |   664 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    25 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  1294 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   371 +
 .../hive/metastore/api/AbortTxnRequest.java     |     2 +-
 .../metastore/api/AddDynamicPartitions.java     |     2 +-
 .../metastore/api/AddPartitionsRequest.java     |     2 +-
 .../hive/metastore/api/AddPartitionsResult.java |     2 +-
 .../hadoop/hive/metastore/api/AggrStats.java    |     2 +-
 .../metastore/api/AlreadyExistsException.java   |     2 +-
 .../metastore/api/BinaryColumnStatsData.java    |     2 +-
 .../metastore/api/BooleanColumnStatsData.java   |     2 +-
 .../hive/metastore/api/CheckLockRequest.java    |     2 +-
 .../metastore/api/ClearFileMetadataRequest.java |   438 +
 .../metastore/api/ClearFileMetadataResult.java  |   283 +
 .../hive/metastore/api/ColumnStatistics.java    |     2 +-
 .../metastore/api/ColumnStatisticsDesc.java     |     2 +-
 .../hive/metastore/api/ColumnStatisticsObj.java |     2 +-
 .../hive/metastore/api/CommitTxnRequest.java    |     2 +-
 .../hive/metastore/api/CompactionRequest.java   |     2 +-
 .../api/ConfigValSecurityException.java         |     2 +-
 .../api/CurrentNotificationEventId.java         |     2 +-
 .../hadoop/hive/metastore/api/Database.java     |     2 +-
 .../apache/hadoop/hive/metastore/api/Date.java  |     2 +-
 .../hive/metastore/api/DateColumnStatsData.java |     2 +-
 .../hadoop/hive/metastore/api/Decimal.java      |     2 +-
 .../metastore/api/DecimalColumnStatsData.java   |     2 +-
 .../metastore/api/DoubleColumnStatsData.java    |     2 +-
 .../hive/metastore/api/DropPartitionsExpr.java  |     2 +-
 .../metastore/api/DropPartitionsRequest.java    |     2 +-
 .../metastore/api/DropPartitionsResult.java     |     2 +-
 .../hive/metastore/api/EnvironmentContext.java  |     2 +-
 .../hadoop/hive/metastore/api/FieldSchema.java  |     2 +-
 .../hive/metastore/api/FireEventRequest.java    |     2 +-
 .../hive/metastore/api/FireEventResponse.java   |     2 +-
 .../hadoop/hive/metastore/api/Function.java     |     2 +-
 .../metastore/api/GetAllFunctionsResponse.java  |    38 +-
 .../api/GetFileMetadataByExprRequest.java       |   548 +
 .../api/GetFileMetadataByExprResult.java        |   703 +
 .../metastore/api/GetFileMetadataRequest.java   |   438 +
 .../metastore/api/GetFileMetadataResult.java    |   540 +
 .../metastore/api/GetOpenTxnsInfoResponse.java  |     2 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |     2 +-
 .../api/GetPrincipalsInRoleRequest.java         |     2 +-
 .../api/GetPrincipalsInRoleResponse.java        |     2 +-
 .../api/GetRoleGrantsForPrincipalRequest.java   |     2 +-
 .../api/GetRoleGrantsForPrincipalResponse.java  |     2 +-
 .../api/GrantRevokePrivilegeRequest.java        |     2 +-
 .../api/GrantRevokePrivilegeResponse.java       |     2 +-
 .../metastore/api/GrantRevokeRoleRequest.java   |     2 +-
 .../metastore/api/GrantRevokeRoleResponse.java  |     2 +-
 .../hive/metastore/api/HeartbeatRequest.java    |     2 +-
 .../metastore/api/HeartbeatTxnRangeRequest.java |     2 +-
 .../api/HeartbeatTxnRangeResponse.java          |     2 +-
 .../hive/metastore/api/HiveObjectPrivilege.java |     2 +-
 .../hive/metastore/api/HiveObjectRef.java       |     2 +-
 .../apache/hadoop/hive/metastore/api/Index.java |     2 +-
 .../api/IndexAlreadyExistsException.java        |     2 +-
 .../metastore/api/InsertEventRequestData.java   |     2 +-
 .../metastore/api/InvalidInputException.java    |     2 +-
 .../metastore/api/InvalidObjectException.java   |     2 +-
 .../api/InvalidOperationException.java          |     2 +-
 .../api/InvalidPartitionException.java          |     2 +-
 .../hive/metastore/api/LockComponent.java       |     2 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |     2 +-
 .../hadoop/hive/metastore/api/LockResponse.java |     2 +-
 .../hive/metastore/api/LongColumnStatsData.java |     2 +-
 .../hive/metastore/api/MetaException.java       |     2 +-
 .../hive/metastore/api/MetadataPpdResult.java   |   508 +
 .../hive/metastore/api/NoSuchLockException.java |     2 +-
 .../metastore/api/NoSuchObjectException.java    |     2 +-
 .../hive/metastore/api/NoSuchTxnException.java  |     2 +-
 .../hive/metastore/api/NotificationEvent.java   |     2 +-
 .../metastore/api/NotificationEventRequest.java |     2 +-
 .../api/NotificationEventResponse.java          |     2 +-
 .../hive/metastore/api/OpenTxnRequest.java      |     2 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Order.java |     2 +-
 .../hadoop/hive/metastore/api/Partition.java    |     2 +-
 .../api/PartitionListComposingSpec.java         |     2 +-
 .../hive/metastore/api/PartitionSpec.java       |     2 +-
 .../api/PartitionSpecWithSharedSD.java          |     2 +-
 .../hive/metastore/api/PartitionWithoutSD.java  |     2 +-
 .../metastore/api/PartitionsByExprRequest.java  |     2 +-
 .../metastore/api/PartitionsByExprResult.java   |     2 +-
 .../metastore/api/PartitionsStatsRequest.java   |     2 +-
 .../metastore/api/PartitionsStatsResult.java    |     2 +-
 .../metastore/api/PrincipalPrivilegeSet.java    |     2 +-
 .../hadoop/hive/metastore/api/PrivilegeBag.java |     2 +-
 .../hive/metastore/api/PrivilegeGrantInfo.java  |     2 +-
 .../metastore/api/PutFileMetadataRequest.java   |   588 +
 .../metastore/api/PutFileMetadataResult.java    |   283 +
 .../hadoop/hive/metastore/api/ResourceUri.java  |     2 +-
 .../apache/hadoop/hive/metastore/api/Role.java  |     2 +-
 .../hive/metastore/api/RolePrincipalGrant.java  |     2 +-
 .../hadoop/hive/metastore/api/Schema.java       |     2 +-
 .../hadoop/hive/metastore/api/SerDeInfo.java    |     2 +-
 .../api/SetPartitionsStatsRequest.java          |     2 +-
 .../hive/metastore/api/ShowCompactRequest.java  |     2 +-
 .../hive/metastore/api/ShowCompactResponse.java |     2 +-
 .../api/ShowCompactResponseElement.java         |     2 +-
 .../hive/metastore/api/ShowLocksRequest.java    |     2 +-
 .../hive/metastore/api/ShowLocksResponse.java   |     2 +-
 .../metastore/api/ShowLocksResponseElement.java |     2 +-
 .../hadoop/hive/metastore/api/SkewedInfo.java   |     2 +-
 .../hive/metastore/api/StorageDescriptor.java   |     2 +-
 .../metastore/api/StringColumnStatsData.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Table.java |     2 +-
 .../hive/metastore/api/TableStatsRequest.java   |     2 +-
 .../hive/metastore/api/TableStatsResult.java    |     2 +-
 .../hive/metastore/api/ThriftHiveMetastore.java |  8422 ++--
 .../hive/metastore/api/TxnAbortedException.java |     2 +-
 .../hadoop/hive/metastore/api/TxnInfo.java      |     2 +-
 .../hive/metastore/api/TxnOpenException.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Type.java  |     2 +-
 .../hive/metastore/api/UnknownDBException.java  |     2 +-
 .../api/UnknownPartitionException.java          |     2 +-
 .../metastore/api/UnknownTableException.java    |     2 +-
 .../hive/metastore/api/UnlockRequest.java       |     2 +-
 .../hadoop/hive/metastore/api/Version.java      |     2 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   |  2810 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |  1009 +-
 .../hive_metastore/ThriftHiveMetastore-remote   |    49 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  1563 +-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   734 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   167 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   267 +
 .../hadoop/hive/metastore/HiveAlterHandler.java |    38 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   272 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   112 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |     6 +
 .../hadoop/hive/metastore/ObjectStore.java      |   437 +-
 .../hive/metastore/PartFilterExprUtil.java      |   149 +
 .../apache/hadoop/hive/metastore/RawStore.java  |    66 +-
 .../hadoop/hive/metastore/RawStoreProxy.java    |     5 +-
 .../hive/metastore/RetryingHMSHandler.java      |    33 +-
 .../hbase/AggrStatsInvalidatorFilter.java       |   121 +
 .../hadoop/hive/metastore/hbase/Counter.java    |    53 +
 .../hive/metastore/hbase/HBaseConnection.java   |    96 +
 .../metastore/hbase/HBaseFilterPlanUtil.java    |   612 +
 .../hive/metastore/hbase/HBaseImport.java       |   535 +
 .../hive/metastore/hbase/HBaseReadWrite.java    |  2106 +
 .../hive/metastore/hbase/HBaseSchemaTool.java   |   239 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |  2387 ++
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |  1340 +
 .../hive/metastore/hbase/ObjectCache.java       |    81 +
 .../hive/metastore/hbase/PartitionCache.java    |   168 +
 .../metastore/hbase/PartitionKeyComparator.java |   292 +
 .../hbase/SharedStorageDescriptor.java          |   251 +
 .../hadoop/hive/metastore/hbase/StatsCache.java |   326 +
 .../metastore/hbase/TephraHBaseConnection.java  |   127 +
 .../metastore/hbase/VanillaHBaseConnection.java |   137 +
 .../stats/BinaryColumnStatsAggregator.java      |    35 +
 .../stats/BooleanColumnStatsAggregator.java     |    35 +
 .../hbase/stats/ColumnStatsAggregator.java      |    26 +
 .../stats/ColumnStatsAggregatorFactory.java     |    94 +
 .../stats/DecimalColumnStatsAggregator.java     |    43 +
 .../stats/DoubleColumnStatsAggregator.java      |    36 +
 .../hbase/stats/LongColumnStatsAggregator.java  |    36 +
 .../stats/StringColumnStatsAggregator.java      |    36 +
 .../hive/metastore/parser/ExpressionTree.java   |     9 +-
 .../hive/metastore/tools/HiveMetaTool.java      |     5 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    61 +-
 .../metastore/hbase/hbase_metastore_proto.proto |   282 +
 .../DummyRawStoreControlledCommit.java          |    56 +-
 .../DummyRawStoreForJdoConnection.java          |    50 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |    43 +-
 .../hadoop/hive/metastore/hbase/MockUtils.java  |   199 +
 .../hbase/TestHBaseAggregateStatsCache.java     |   316 +
 .../hbase/TestHBaseFilterPlanUtil.java          |   483 +
 .../hive/metastore/hbase/TestHBaseStore.java    |  1307 +
 .../metastore/hbase/TestHBaseStoreCached.java   |   378 +
 .../hbase/TestSharedStorageDescriptor.java      |   153 +
 pom.xml                                         |    25 +-
 .../hadoop/hive/ql/plan/api/Adjacency.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Graph.java   |     2 +-
 .../hadoop/hive/ql/plan/api/Operator.java       |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Query.java   |     2 +-
 .../hadoop/hive/ql/plan/api/QueryPlan.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Stage.java   |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Task.java    |     2 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |    17 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   333 +-
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |    24 +-
 .../hadoop/hive/ql/exec/KeyWrapperFactory.java  |     4 +
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |     8 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    31 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |     3 +-
 .../ql/exec/SparkHashTableSinkOperator.java     |     3 +-
 .../hadoop/hive/ql/exec/StatsNoJobTask.java     |    25 +-
 .../apache/hadoop/hive/ql/exec/StatsTask.java   |    13 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |    11 +-
 .../persistence/BytesBytesMultiHashMap.java     |    11 +-
 .../persistence/HybridHashTableContainer.java   |    68 +-
 .../ql/exec/persistence/PTFRowContainer.java    |    14 +-
 .../hive/ql/exec/persistence/RowContainer.java  |    12 +-
 .../hadoop/hive/ql/exec/spark/SparkPlan.java    |     3 +-
 .../hive/ql/exec/spark/SparkPlanGenerator.java  |     3 +-
 .../hive/ql/exec/spark/SparkRecordHandler.java  |     3 +-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |     2 +-
 .../ql/exec/spark/status/SparkJobMonitor.java   |     2 +-
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |     3 +
 .../hive/ql/exec/tez/HashTableLoader.java       |     7 +-
 .../hadoop/hive/ql/exec/tez/InPlaceUpdates.java |    65 +
 .../hive/ql/exec/tez/RecordProcessor.java       |     3 +-
 .../hive/ql/exec/tez/ReduceRecordProcessor.java |     1 -
 .../hive/ql/exec/tez/ReduceRecordSource.java    |     3 +-
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  |    70 +-
 .../hadoop/hive/ql/exec/tez/TezProcessor.java   |     3 +-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |    20 +-
 .../ql/exec/tez/tools/KeyValuesInputMerger.java |     1 -
 .../ql/exec/vector/VectorizationContext.java    |    10 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |     9 +-
 .../hadoop/hive/ql/hooks/LineageLogger.java     |    95 +-
 .../hive/ql/io/CombineHiveInputFormat.java      |    10 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |     5 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   151 +-
 .../apache/hadoop/hive/ql/io/orc/OrcSerde.java  |     1 +
 .../apache/hadoop/hive/ql/io/orc/OrcStruct.java |     2 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |     4 +-
 .../hive/ql/io/parquet/ProjectionPusher.java    |     3 +-
 .../serde/ParquetHiveArrayInspector.java        |    12 +
 .../ql/io/parquet/timestamp/NanoTimeUtils.java  |    23 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |     4 +
 .../apache/hadoop/hive/ql/lib/RuleRegExp.java   |    61 +-
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |   195 -
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    34 +-
 .../hadoop/hive/ql/metadata/Partition.java      |    29 +-
 .../ql/optimizer/ColumnPrunerProcFactory.java   |     3 +
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |    22 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |    57 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |    19 +-
 .../calcite/reloperators/HiveBetween.java       |    75 +
 .../optimizer/calcite/reloperators/HiveIn.java  |    41 +
 .../calcite/reloperators/HiveLimit.java         |    57 -
 .../calcite/reloperators/HiveSort.java          |   110 -
 .../calcite/reloperators/HiveSortLimit.java     |   110 +
 .../rules/HiveAggregateProjectMergeRule.java    |   151 +
 .../calcite/rules/HivePreFilteringRule.java     |    37 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      |   145 +-
 .../calcite/stats/HiveRelMdMemory.java          |     9 +-
 .../calcite/stats/HiveRelMdParallelism.java     |     4 +-
 .../calcite/translator/ASTConverter.java        |    24 +-
 .../calcite/translator/HiveOpConverter.java     |     8 +-
 .../translator/PlanModifierForASTConv.java      |    14 +-
 .../translator/PlanModifierForReturnPath.java   |     4 -
 .../calcite/translator/PlanModifierUtil.java    |     4 +-
 .../translator/SqlFunctionConverter.java        |    16 +-
 .../hive/ql/optimizer/lineage/LineageCtx.java   |     8 +-
 .../hive/ql/optimizer/ppr/PartitionPruner.java  |     7 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    28 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    17 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |     9 +-
 .../apache/hadoop/hive/ql/parse/QBSubQuery.java |     7 -
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |     4 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |     2 +
 .../hadoop/hive/ql/parse/SubQueryUtils.java     |    11 -
 .../hive/ql/parse/spark/SparkCompiler.java      |     3 +-
 .../org/apache/hadoop/hive/ql/plan/DDLWork.java |    21 +
 .../hadoop/hive/ql/plan/HiveOperation.java      |     1 +
 .../hive/ql/plan/ShowCreateDatabaseDesc.java    |    94 +
 .../AuthorizationPreEventListener.java          |     2 +-
 .../authorization/plugin/HiveOperationType.java |     1 +
 .../plugin/sqlstd/Operation2Privilege.java      |     2 +
 .../sqlstd/SQLStdHiveAccessController.java      |     5 +
 .../hadoop/hive/ql/session/SessionState.java    |    37 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |    87 +-
 .../org/apache/hadoop/hive/ql/udf/UDFJson.java  |     2 +
 .../hive/ql/udf/generic/GenericUDAFMax.java     |    16 +-
 ql/src/main/resources/hive-exec-log4j2.xml      |     5 +-
 ql/src/main/resources/tez-container-log4j2.xml  |     5 +-
 .../hive/metastore/TestMetastoreExpr.java       |     2 +-
 .../persistence/TestBytesBytesMultiHashMap.java |     3 +
 .../ql/exec/persistence/TestHashPartition.java  |    29 +
 .../exec/persistence/TestPTFRowContainer.java   |    31 +-
 .../hadoop/hive/ql/io/orc/TestOrcStruct.java    |     2 +
 .../serde/TestParquetTimestampUtils.java        |    38 +-
 ...nMapRedUtilsUsePartitionColumnsNegative.java |    73 +
 ...nMapRedUtilsUsePartitionColumnsPositive.java |    61 +
 .../authorization_set_show_current_role.q       |     3 +
 .../clientpositive/bucket_map_join_tez1.q       |    31 +
 .../queries/clientpositive/cbo_rp_auto_join17.q |    14 +
 .../cbo_rp_cross_product_check_2.q              |    31 +
 .../clientpositive/drop_table_with_index.q      |    35 +
 .../test/queries/clientpositive/dynpart_merge.q |    28 +
 .../dynpart_sort_opt_vectorization.q            |     2 +
 .../clientpositive/dynpart_sort_optimization.q  |     2 +
 .../queries/clientpositive/exchgpartition2lel.q |    32 +
 ql/src/test/queries/clientpositive/lineage3.q   |    26 +
 .../test/queries/clientpositive/load_orc_part.q |     5 +
 .../parquet_mixed_partition_formats.q           |    42 +
 .../clientpositive/parquet_ppd_boolean.q        |    42 +-
 .../queries/clientpositive/parquet_ppd_char.q   |    46 +-
 .../queries/clientpositive/parquet_ppd_date.q   |    64 +-
 .../clientpositive/parquet_ppd_decimal.q        |   106 +-
 .../clientpositive/parquet_ppd_timestamp.q      |    62 +-
 .../clientpositive/parquet_ppd_varchar.q        |    46 +-
 .../clientpositive/parquet_predicate_pushdown.q |    20 +-
 .../clientpositive/show_create_database.q       |     3 +
 .../queries/clientpositive/subquery_views.q     |    22 +-
 .../queries/clientpositive/union_fast_stats.q   |    68 +
 .../queries/clientpositive/vector_char_cast.q   |     9 +
 .../queries/clientpositive/windowing_udaf.q     |     4 +
 .../subquery_exists_implicit_gby.q.out          |     8 +-
 .../subquery_nested_subquery.q.out              |     4 +-
 .../subquery_notexists_implicit_gby.q.out       |     8 +-
 .../subquery_windowing_corr.q.out               |     7 +-
 .../alter_partition_coltype.q.out               |     8 +-
 .../clientpositive/annotate_stats_groupby.q.out |   106 +-
 .../annotate_stats_groupby2.q.out               |    28 +-
 .../authorization_explain.q.java1.7.out         |     2 +-
 .../authorization_explain.q.java1.8.out         |     2 +-
 .../authorization_set_show_current_role.q.out   |     8 +
 .../results/clientpositive/auto_join18.q.out    |    12 +-
 .../auto_join18_multi_distinct.q.out            |    12 +-
 .../results/clientpositive/auto_join27.q.out    |    18 +-
 .../results/clientpositive/auto_join32.q.out    |     4 +-
 .../clientpositive/binarysortable_1.q.out       |   Bin 4329 -> 4325 bytes
 .../clientpositive/cbo_rp_auto_join17.q.out     |   118 +
 .../cbo_rp_cross_product_check_2.q.out          |   699 +
 .../clientpositive/correlationoptimizer2.q.out  |   220 +-
 .../clientpositive/correlationoptimizer6.q.out  |   232 +-
 ql/src/test/results/clientpositive/count.q.out  |    14 +-
 .../results/clientpositive/ctas_colname.q.out   |    52 +-
 .../test/results/clientpositive/database.q.out  |     2 +-
 .../clientpositive/decimal_precision.q.out      |     4 +-
 .../results/clientpositive/decimal_udf.q.out    |    30 +-
 .../results/clientpositive/distinct_stats.q.out |    14 +-
 .../clientpositive/drop_table_with_index.q.out  |   152 +
 .../results/clientpositive/dynpart_merge.q.out  |    99 +
 .../dynpart_sort_opt_vectorization.q.out        |   117 +-
 .../dynpart_sort_optimization.q.out             |   117 +-
 ...ryption_select_read_only_encrypted_tbl.q.out |     4 +-
 .../clientpositive/exchgpartition2lel.q.out     |   182 +
 .../clientpositive/explain_dependency.q.out     |    18 +-
 .../clientpositive/explain_dependency2.q.out    |    16 +-
 .../clientpositive/explain_logical.q.out        |    78 +-
 .../clientpositive/fetch_aggregation.q.out      |     4 +-
 .../test/results/clientpositive/gby_star.q.out  |    54 +-
 .../test/results/clientpositive/groupby12.q.out |     6 +-
 .../results/clientpositive/groupby5_map.q.out   |     4 +-
 .../clientpositive/groupby5_map_skew.q.out      |     4 +-
 .../results/clientpositive/groupby_cube1.q.out  |    12 +-
 .../groupby_distinct_samekey.q.out              |     6 +-
 .../clientpositive/groupby_grouping_sets2.q.out |    10 +-
 .../clientpositive/groupby_grouping_sets3.q.out |    12 +-
 .../clientpositive/groupby_grouping_sets5.q.out |     8 +-
 .../clientpositive/groupby_grouping_sets6.q.out |     8 +-
 .../clientpositive/groupby_position.q.out       |    36 +-
 .../clientpositive/groupby_resolution.q.out     |    60 +-
 .../clientpositive/groupby_rollup1.q.out        |    12 +-
 .../clientpositive/groupby_sort_10.q.out        |     8 +-
 .../clientpositive/groupby_sort_11.q.out        |    10 +-
 .../results/clientpositive/groupby_sort_8.q.out |    12 +-
 ql/src/test/results/clientpositive/having.q.out |    62 +-
 .../test/results/clientpositive/having2.q.out   |    12 +-
 .../clientpositive/index_auto_mult_tables.q.out |    12 +-
 .../clientpositive/index_auto_self_join.q.out   |    12 +-
 .../clientpositive/index_auto_update.q.out      |     6 +-
 .../index_bitmap_auto_partitioned.q.out         |     6 +-
 .../index_bitmap_compression.q.out              |     6 +-
 .../infer_bucket_sort_dyn_part.q.out            |     4 +-
 .../infer_bucket_sort_map_operators.q.out       |     4 +-
 .../results/clientpositive/input4.q.java1.7.out |     2 +-
 .../results/clientpositive/input4.q.java1.8.out |     2 +-
 .../results/clientpositive/join0.q.java1.7.out  |     2 +-
 .../results/clientpositive/join0.q.java1.8.out  |     4 +-
 ql/src/test/results/clientpositive/join18.q.out |    12 +-
 .../clientpositive/join18_multi_distinct.q.out  |    12 +-
 ql/src/test/results/clientpositive/join31.q.out |    36 +-
 .../limit_partition_metadataonly.q.out          |     4 +-
 .../results/clientpositive/limit_pushdown.q.out |    36 +-
 .../test/results/clientpositive/lineage2.q.out  |     2 +-
 .../test/results/clientpositive/lineage3.q.out  |    72 +-
 .../list_bucket_dml_6.q.java1.7.out             |    12 +-
 .../list_bucket_dml_6.q.java1.8.out             |    12 +-
 .../clientpositive/list_bucket_dml_7.q.out      |    12 +-
 .../list_bucket_query_multiskew_3.q.out         |     2 +-
 .../results/clientpositive/load_orc_part.q.out  |    26 +
 .../clientpositive/mapjoin_mapjoin.q.out        |    32 +-
 .../clientpositive/metadata_only_queries.q.out  |     4 +-
 .../results/clientpositive/metadataonly1.q.out  |   112 +-
 .../results/clientpositive/multiMapJoin2.q.out  |   226 +-
 .../nonblock_op_deduplicate.q.out               |     8 +-
 .../results/clientpositive/nonmr_fetch.q.out    |    14 +-
 .../results/clientpositive/parallel_join0.q.out |     2 +-
 .../parquet_mixed_partition_formats.q.out       |   303 +
 .../clientpositive/parquet_ppd_boolean.q.out    |   194 +-
 .../clientpositive/parquet_ppd_char.q.out       |   224 +-
 .../clientpositive/parquet_ppd_date.q.out       |   324 +-
 .../clientpositive/parquet_ppd_decimal.q.out    |   594 +-
 .../clientpositive/parquet_ppd_timestamp.q.out  |   314 +-
 .../clientpositive/parquet_ppd_varchar.q.out    |   224 +-
 .../parquet_predicate_pushdown.q.out            |   118 +-
 .../clientpositive/partition_multilevels.q.out  |     8 +-
 .../clientpositive/plan_json.q.java1.7.out      |     2 +-
 .../clientpositive/plan_json.q.java1.8.out      |     2 +-
 .../test/results/clientpositive/ppd_gby.q.out   |    12 +-
 .../test/results/clientpositive/ppd_gby2.q.out  |    60 +-
 .../clientpositive/ppd_join_filter.q.out        |    98 +-
 .../ql_rewrite_gbtoidx_cbo_1.q.out              |   168 +-
 .../ql_rewrite_gbtoidx_cbo_2.q.out              |    94 +-
 .../reduce_deduplicate_extended.q.out           |    32 +-
 .../clientpositive/selectDistinctStar.q.out     |    44 +-
 .../clientpositive/show_create_database.q.out   |    19 +
 .../clientpositive/spark/auto_join18.q.out      |    10 +-
 .../spark/auto_join18_multi_distinct.q.out      |    12 +-
 .../clientpositive/spark/auto_join27.q.out      |    18 +-
 .../clientpositive/spark/auto_join32.q.out      |    53 +-
 .../spark/bucket_map_join_tez1.q.out            |   357 +
 .../results/clientpositive/spark/count.q.out    |    14 +-
 .../clientpositive/spark/groupby5_map.q.out     |     4 +-
 .../spark/groupby5_map_skew.q.out               |     4 +-
 .../clientpositive/spark/groupby_cube1.q.out    |    12 +-
 .../clientpositive/spark/groupby_position.q.out |    18 +-
 .../spark/groupby_resolution.q.out              |    60 +-
 .../clientpositive/spark/groupby_rollup1.q.out  |    12 +-
 .../results/clientpositive/spark/having.q.out   |    62 +-
 .../spark/infer_bucket_sort_map_operators.q.out |     4 +-
 .../results/clientpositive/spark/join18.q.out   |    10 +-
 .../spark/join18_multi_distinct.q.out           |    12 +-
 .../results/clientpositive/spark/join31.q.out   |    36 +-
 .../spark/limit_partition_metadataonly.q.out    |     4 +-
 .../clientpositive/spark/limit_pushdown.q.out   |    34 +-
 .../clientpositive/spark/mapjoin_mapjoin.q.out  |    24 +-
 .../spark/metadata_only_queries.q.out           |     4 +-
 .../clientpositive/spark/ppd_join_filter.q.out  |    90 +-
 .../spark/ql_rewrite_gbtoidx_cbo_1.q.out        |   168 +-
 .../clientpositive/spark/stats_only_null.q.out  |     8 +-
 .../clientpositive/spark/subquery_in.q.out      |    36 +-
 .../results/clientpositive/spark/union11.q.out  |    42 +-
 .../results/clientpositive/spark/union14.q.out  |    28 +-
 .../results/clientpositive/spark/union15.q.out  |    28 +-
 .../results/clientpositive/spark/union28.q.out  |     4 +-
 .../results/clientpositive/spark/union30.q.out  |     4 +-
 .../results/clientpositive/spark/union33.q.out  |     8 +-
 .../results/clientpositive/spark/union5.q.out   |    34 +-
 .../results/clientpositive/spark/union7.q.out   |    28 +-
 .../clientpositive/spark/union_remove_21.q.out  |     4 +-
 .../spark/vector_count_distinct.q.out           |     4 +-
 .../spark/vector_decimal_aggregate.q.out        |    12 +-
 .../spark/vector_distinct_2.q.out               |    28 +-
 .../clientpositive/spark/vector_groupby_3.q.out |    30 +-
 .../spark/vector_mapjoin_reduce.q.out           |    36 +-
 .../clientpositive/spark/vector_orderby_5.q.out |     6 +-
 .../clientpositive/spark/vectorization_0.q.out  |    16 +-
 .../clientpositive/spark/vectorization_13.q.out |    32 +-
 .../clientpositive/spark/vectorization_15.q.out |    16 +-
 .../clientpositive/spark/vectorization_16.q.out |    16 +-
 .../clientpositive/spark/vectorization_9.q.out  |    16 +-
 .../spark/vectorization_pushdown.q.out          |     4 +-
 .../spark/vectorization_short_regress.q.out     |    74 +-
 .../spark/vectorized_nested_mapjoin.q.out       |    18 +-
 .../spark/vectorized_timestamp_funcs.q.out      |    12 +-
 .../clientpositive/stats_only_null.q.out        |     8 +-
 .../results/clientpositive/stats_ppr_all.q.out  |    16 +-
 .../subq_where_serialization.q.out              |    18 +-
 .../clientpositive/subquery_exists_having.q.out |    48 +-
 .../results/clientpositive/subquery_in.q.out    |    36 +-
 .../clientpositive/subquery_in_having.q.out     |   260 +-
 .../clientpositive/subquery_notexists.q.out     |    18 +-
 .../subquery_notexists_having.q.out             |    26 +-
 .../results/clientpositive/subquery_notin.q.out |    24 +-
 .../subquery_notin_having.q.java1.7.out         |    50 +-
 .../subquery_unqualcolumnrefs.q.out             |    74 +-
 .../results/clientpositive/subquery_views.q.out |   124 +-
 .../tez/bucket_map_join_tez1.q.out              |   333 +
 .../clientpositive/tez/constprog_dpp.q.out      |     4 +-
 .../test/results/clientpositive/tez/count.q.out |    14 +-
 .../tez/dynamic_partition_pruning.q.out         |    88 +-
 .../tez/dynpart_sort_opt_vectorization.q.out    |   102 +-
 .../tez/dynpart_sort_optimization.q.out         |   101 +-
 .../clientpositive/tez/explainuser_1.q.out      |  2799 +-
 .../clientpositive/tez/explainuser_2.q.out      |  4004 +-
 .../clientpositive/tez/explainuser_3.q.out      |    10 +-
 .../results/clientpositive/tez/having.q.out     |    62 +-
 .../clientpositive/tez/limit_pushdown.q.out     |    34 +-
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |    24 +-
 .../tez/metadata_only_queries.q.out             |     4 +-
 .../clientpositive/tez/metadataonly1.q.out      |    44 +-
 .../test/results/clientpositive/tez/mrr.q.out   |    94 +-
 .../clientpositive/tez/selectDistinctStar.q.out |    44 +-
 .../tez/show_create_database.q.out              |    19 +
 .../clientpositive/tez/stats_only_null.q.out    |     8 +-
 .../clientpositive/tez/subquery_in.q.out        |    36 +-
 .../results/clientpositive/tez/tez_dml.q.out    |     6 +-
 .../results/clientpositive/tez/union5.q.out     |    44 +-
 .../results/clientpositive/tez/union7.q.out     |    28 +-
 .../clientpositive/tez/unionDistinct_1.q.out    |     8 +-
 .../clientpositive/tez/union_fast_stats.q.out   |   526 +
 .../clientpositive/tez/vector_aggregate_9.q.out |     4 +-
 .../tez/vector_binary_join_groupby.q.out        |     4 +-
 .../clientpositive/tez/vector_char_cast.q.out   |    35 +
 .../tez/vector_count_distinct.q.out             |     4 +-
 .../tez/vector_decimal_aggregate.q.out          |    12 +-
 .../tez/vector_decimal_precision.q.out          |     4 +-
 .../clientpositive/tez/vector_decimal_udf.q.out |    30 +-
 .../clientpositive/tez/vector_distinct_2.q.out  |    28 +-
 .../clientpositive/tez/vector_groupby_3.q.out   |    30 +-
 .../tez/vector_groupby_reduce.q.out             |     8 +-
 .../tez/vector_grouping_sets.q.out              |     8 +-
 .../tez/vector_mapjoin_reduce.q.out             |    36 +-
 .../clientpositive/tez/vector_orderby_5.q.out   |     6 +-
 .../clientpositive/tez/vector_outer_join2.q.out |    20 +-
 .../tez/vector_partition_diff_num_cols.q.out    |    20 +-
 .../tez/vector_partitioned_date_time.q.out      |    12 +-
 .../tez/vector_reduce_groupby_decimal.q.out     |    24 +-
 .../clientpositive/tez/vectorization_0.q.out    |    16 +-
 .../clientpositive/tez/vectorization_13.q.out   |    32 +-
 .../clientpositive/tez/vectorization_15.q.out   |    16 +-
 .../clientpositive/tez/vectorization_16.q.out   |    16 +-
 .../clientpositive/tez/vectorization_9.q.out    |    16 +-
 .../tez/vectorization_limit.q.out               |    14 +-
 .../tez/vectorization_pushdown.q.out            |     4 +-
 .../tez/vectorization_short_regress.q.out       |    74 +-
 .../tez/vectorized_distinct_gby.q.out           |     8 +-
 .../vectorized_dynamic_partition_pruning.q.out  |    88 +-
 .../tez/vectorized_nested_mapjoin.q.out         |    18 +-
 .../clientpositive/tez/vectorized_parquet.q.out |     6 +-
 .../tez/vectorized_timestamp_funcs.q.out        |    12 +-
 ql/src/test/results/clientpositive/udf8.q.out   |     4 +-
 .../test/results/clientpositive/udf_count.q.out |    16 +-
 .../test/results/clientpositive/union11.q.out   |    70 +-
 .../test/results/clientpositive/union14.q.out   |    32 +-
 .../test/results/clientpositive/union15.q.out   |    38 +-
 .../test/results/clientpositive/union28.q.out   |     8 +-
 .../test/results/clientpositive/union30.q.out   |     8 +-
 .../test/results/clientpositive/union33.q.out   |     8 +-
 ql/src/test/results/clientpositive/union5.q.out |    48 +-
 ql/src/test/results/clientpositive/union7.q.out |    32 +-
 .../clientpositive/unionDistinct_1.q.out        |     8 +-
 .../clientpositive/union_fast_stats.q.out       |   526 +
 .../clientpositive/union_remove_21.q.out        |     8 +-
 .../clientpositive/vector_aggregate_9.q.out     |     4 +-
 .../vector_aggregate_without_gby.q.out          |     4 +-
 .../vector_binary_join_groupby.q.out            |     4 +-
 .../clientpositive/vector_char_cast.q.out       |    35 +
 .../clientpositive/vector_count_distinct.q.out  |     6 +-
 .../vector_decimal_aggregate.q.out              |    12 +-
 .../vector_decimal_precision.q.out              |     4 +-
 .../clientpositive/vector_decimal_udf.q.out     |    30 +-
 .../clientpositive/vector_distinct_2.q.out      |    28 +-
 .../clientpositive/vector_groupby_3.q.out       |    30 +-
 .../clientpositive/vector_groupby_reduce.q.out  |     8 +-
 .../clientpositive/vector_grouping_sets.q.out   |     8 +-
 .../clientpositive/vector_left_outer_join.q.out |     8 +-
 .../clientpositive/vector_mapjoin_reduce.q.out  |    36 +-
 .../clientpositive/vector_orderby_5.q.out       |     6 +-
 .../clientpositive/vector_outer_join1.q.out     |     8 +-
 .../clientpositive/vector_outer_join2.q.out     |    28 +-
 .../clientpositive/vector_outer_join3.q.out     |    24 +-
 .../clientpositive/vector_outer_join4.q.out     |     8 +-
 .../clientpositive/vector_outer_join5.q.out     |    48 +-
 .../vector_partition_diff_num_cols.q.out        |    20 +-
 .../vector_partitioned_date_time.q.out          |    12 +-
 .../vector_reduce_groupby_decimal.q.out         |    24 +-
 .../clientpositive/vectorization_0.q.out        |    16 +-
 .../clientpositive/vectorization_13.q.out       |    32 +-
 .../clientpositive/vectorization_15.q.out       |    16 +-
 .../clientpositive/vectorization_16.q.out       |    16 +-
 .../clientpositive/vectorization_9.q.out        |    16 +-
 .../clientpositive/vectorization_limit.q.out    |    16 +-
 .../clientpositive/vectorization_pushdown.q.out |     4 +-
 .../vectorization_short_regress.q.out           |    74 +-
 .../vectorized_distinct_gby.q.out               |    12 +-
 .../vectorized_nested_mapjoin.q.out             |    26 +-
 .../clientpositive/vectorized_parquet.q.out     |     6 +-
 .../vectorized_parquet_types.q.out              |     6 +-
 .../vectorized_timestamp_funcs.q.out            |    12 +-
 .../results/clientpositive/windowing_udaf.q.out |    12 +
 ql/src/test/templates/TestCliDriver.vm          |     3 +-
 .../hadoop/hive/serde/test/InnerStruct.java     |     2 +-
 .../hadoop/hive/serde/test/ThriftTestObj.java   |     2 +-
 .../hadoop/hive/serde2/thrift/test/Complex.java |     2 +-
 .../hive/serde2/thrift/test/IntString.java      |     2 +-
 .../hive/serde2/thrift/test/MegaStruct.java     |     2 +-
 .../hive/serde2/thrift/test/MiniStruct.java     |     2 +-
 .../hive/serde2/thrift/test/SetIntString.java   |     2 +-
 .../hive/serde2/ColumnProjectionUtils.java      |    22 +
 .../apache/hadoop/hive/serde2/WriteBuffers.java |    10 +-
 .../BinarySortableSerDeWithEndPrefix.java       |    41 +
 .../hadoop/hive/serde2/lazy/LazyByte.java       |     4 +
 .../hadoop/hive/serde2/lazy/LazyDouble.java     |     4 +
 .../hadoop/hive/serde2/lazy/LazyFloat.java      |     4 +
 .../hadoop/hive/serde2/lazy/LazyInteger.java    |     4 +
 .../hadoop/hive/serde2/lazy/LazyLong.java       |     4 +
 .../hadoop/hive/serde2/lazy/LazyShort.java      |     4 +
 .../hadoop/hive/serde2/lazy/LazyUtils.java      |    28 +
 .../hadoop/hive/service/HiveClusterStatus.java  |     2 +-
 .../hive/service/HiveServerException.java       |     2 +-
 .../apache/hadoop/hive/service/ThriftHive.java  |     2 +-
 .../service/cli/thrift/TArrayTypeEntry.java     |     2 +-
 .../hive/service/cli/thrift/TBinaryColumn.java  |     2 +-
 .../hive/service/cli/thrift/TBoolColumn.java    |     2 +-
 .../hive/service/cli/thrift/TBoolValue.java     |     2 +-
 .../hive/service/cli/thrift/TByteColumn.java    |     2 +-
 .../hive/service/cli/thrift/TByteValue.java     |     2 +-
 .../hive/service/cli/thrift/TCLIService.java    |     2 +-
 .../cli/thrift/TCancelDelegationTokenReq.java   |     2 +-
 .../cli/thrift/TCancelDelegationTokenResp.java  |     2 +-
 .../service/cli/thrift/TCancelOperationReq.java |     2 +-
 .../cli/thrift/TCancelOperationResp.java        |     2 +-
 .../service/cli/thrift/TCloseOperationReq.java  |     2 +-
 .../service/cli/thrift/TCloseOperationResp.java |     2 +-
 .../service/cli/thrift/TCloseSessionReq.java    |     2 +-
 .../service/cli/thrift/TCloseSessionResp.java   |     2 +-
 .../hive/service/cli/thrift/TColumnDesc.java    |     2 +-
 .../hive/service/cli/thrift/TDoubleColumn.java  |     2 +-
 .../hive/service/cli/thrift/TDoubleValue.java   |     2 +-
 .../cli/thrift/TExecuteStatementReq.java        |     2 +-
 .../cli/thrift/TExecuteStatementResp.java       |     2 +-
 .../service/cli/thrift/TFetchResultsReq.java    |     2 +-
 .../service/cli/thrift/TFetchResultsResp.java   |     2 +-
 .../service/cli/thrift/TGetCatalogsReq.java     |     2 +-
 .../service/cli/thrift/TGetCatalogsResp.java    |     2 +-
 .../hive/service/cli/thrift/TGetColumnsReq.java |     2 +-
 .../service/cli/thrift/TGetColumnsResp.java     |     2 +-
 .../cli/thrift/TGetDelegationTokenReq.java      |     2 +-
 .../cli/thrift/TGetDelegationTokenResp.java     |     2 +-
 .../service/cli/thrift/TGetFunctionsReq.java    |     2 +-
 .../service/cli/thrift/TGetFunctionsResp.java   |     2 +-
 .../hive/service/cli/thrift/TGetInfoReq.java    |     2 +-
 .../hive/service/cli/thrift/TGetInfoResp.java   |     2 +-
 .../cli/thrift/TGetOperationStatusReq.java      |     2 +-
 .../cli/thrift/TGetOperationStatusResp.java     |     2 +-
 .../cli/thrift/TGetResultSetMetadataReq.java    |     2 +-
 .../cli/thrift/TGetResultSetMetadataResp.java   |     2 +-
 .../hive/service/cli/thrift/TGetSchemasReq.java |     2 +-
 .../service/cli/thrift/TGetSchemasResp.java     |     2 +-
 .../service/cli/thrift/TGetTableTypesReq.java   |     2 +-
 .../service/cli/thrift/TGetTableTypesResp.java  |     2 +-
 .../hive/service/cli/thrift/TGetTablesReq.java  |     2 +-
 .../hive/service/cli/thrift/TGetTablesResp.java |     2 +-
 .../service/cli/thrift/TGetTypeInfoReq.java     |     2 +-
 .../service/cli/thrift/TGetTypeInfoResp.java    |     2 +-
 .../service/cli/thrift/THandleIdentifier.java   |     2 +-
 .../hive/service/cli/thrift/TI16Column.java     |     2 +-
 .../hive/service/cli/thrift/TI16Value.java      |     2 +-
 .../hive/service/cli/thrift/TI32Column.java     |     2 +-
 .../hive/service/cli/thrift/TI32Value.java      |     2 +-
 .../hive/service/cli/thrift/TI64Column.java     |     2 +-
 .../hive/service/cli/thrift/TI64Value.java      |     2 +-
 .../hive/service/cli/thrift/TMapTypeEntry.java  |     2 +-
 .../service/cli/thrift/TOpenSessionReq.java     |     2 +-
 .../service/cli/thrift/TOpenSessionResp.java    |     2 +-
 .../service/cli/thrift/TOperationHandle.java    |     2 +-
 .../service/cli/thrift/TPrimitiveTypeEntry.java |     2 +-
 .../cli/thrift/TRenewDelegationTokenReq.java    |     2 +-
 .../cli/thrift/TRenewDelegationTokenResp.java   |     2 +-
 .../apache/hive/service/cli/thrift/TRow.java    |     2 +-
 .../apache/hive/service/cli/thrift/TRowSet.java |     2 +-
 .../hive/service/cli/thrift/TSessionHandle.java |     2 +-
 .../apache/hive/service/cli/thrift/TStatus.java |     2 +-
 .../hive/service/cli/thrift/TStringColumn.java  |     2 +-
 .../hive/service/cli/thrift/TStringValue.java   |     2 +-
 .../service/cli/thrift/TStructTypeEntry.java    |     2 +-
 .../hive/service/cli/thrift/TTableSchema.java   |     2 +-
 .../hive/service/cli/thrift/TTypeDesc.java      |     2 +-
 .../service/cli/thrift/TTypeQualifiers.java     |     2 +-
 .../service/cli/thrift/TUnionTypeEntry.java     |     2 +-
 .../cli/thrift/TUserDefinedTypeEntry.java       |     2 +-
 .../gen-py/hive_service/ThriftHive-remote       |    49 +-
 .../auth/LdapAuthenticationProviderImpl.java    |    82 +-
 .../org/apache/hive/service/cli/Column.java     |     2 +-
 .../thrift/RetryingThriftCLIServiceClient.java  |   331 +
 .../org/apache/hive/service/cli/TestColumn.java |   129 +
 .../cli/TestRetryingThriftCLIServiceClient.java |   133 +
 .../hadoop/hive/shims/Hadoop20SShims.java       |     5 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |     4 +-
 .../hive/thrift/HadoopThriftAuthBridge.java     |     3 +
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |     5 +-
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |     7 +-
 testutils/ptest2/src/main/resources/log4j2.xml  |     5 +-
 721 files changed, 90748 insertions(+), 16405 deletions(-)
----------------------------------------------------------------------



[52/52] [abbrv] hive git commit: HIVE-11921 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-11921 : merge master into branch (Sergey Shelukhin)


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

Branch: refs/heads/llap
Commit: f84eefa526346b7b48adebfd48044df7aff04e29
Parents: 4718761 451381c
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Sep 22 14:31:29 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Sep 22 14:31:29 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ant/QTestGenTask.java    |    11 +
 bin/ext/hbaseimport.cmd                         |    35 +
 bin/ext/hbaseimport.sh                          |    27 +
 bin/ext/hbaseschematool.sh                      |    27 +
 .../apache/hadoop/hive/common/ObjectPair.java   |     5 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    98 +-
 .../apache/hive/common/util/BloomFilter.java    |    20 +-
 data/conf/tez/hive-site.xml                     |    10 +
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |    70 +-
 .../main/java/org/apache/hive/hplsql/Cmp.java   |   314 +
 .../java/org/apache/hive/hplsql/Column.java     |    29 +-
 .../main/java/org/apache/hive/hplsql/Conn.java  |    21 +
 .../main/java/org/apache/hive/hplsql/Copy.java  |    50 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |    66 +-
 .../java/org/apache/hive/hplsql/Expression.java |    33 +-
 .../main/java/org/apache/hive/hplsql/File.java  |    18 +-
 .../main/java/org/apache/hive/hplsql/Meta.java  |    28 +-
 .../main/java/org/apache/hive/hplsql/Query.java |    18 +
 .../java/org/apache/hive/hplsql/Select.java     |    23 +-
 .../main/java/org/apache/hive/hplsql/Stmt.java  |     8 +-
 .../main/java/org/apache/hive/hplsql/Var.java   |   110 +-
 .../apache/hive/hplsql/functions/Function.java  |     6 +-
 .../hive/hplsql/functions/FunctionMisc.java     |   121 +
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |    18 +
 .../apache/hive/hplsql/TestHplsqlOffline.java   |     5 +
 hplsql/src/test/queries/db/cmp_row_count.sql    |     4 +
 hplsql/src/test/queries/db/cmp_sum.sql          |     3 +
 hplsql/src/test/queries/db/copy_to_file.sql     |     2 +
 hplsql/src/test/queries/db/copy_to_hdfs.sql     |     2 +
 hplsql/src/test/queries/db/copy_to_table.sql    |     2 +
 hplsql/src/test/queries/db/part_count.sql       |    17 +
 hplsql/src/test/queries/db/part_count_by.sql    |     4 +
 hplsql/src/test/queries/db/schema.sql           |    32 +
 hplsql/src/test/queries/db/select_into.sql      |    20 +-
 hplsql/src/test/queries/db/select_into2.sql     |    17 +
 .../test/queries/local/create_procedure2.sql    |    16 +
 hplsql/src/test/queries/local/if2.sql           |     5 +
 hplsql/src/test/queries/local/include.sql       |     2 +
 hplsql/src/test/queries/local/include_file.sql  |     1 +
 hplsql/src/test/queries/local/mult_div.sql      |     8 +
 hplsql/src/test/queries/offline/select_db2.sql  |     5 +
 .../src/test/results/db/cmp_row_count.out.txt   |    12 +
 hplsql/src/test/results/db/cmp_sum.out.txt      |   320 +
 hplsql/src/test/results/db/copy_to_file.out.txt |     6 +
 hplsql/src/test/results/db/copy_to_hdfs.out.txt |     4 +
 .../src/test/results/db/copy_to_table.out.txt   |     2 +
 hplsql/src/test/results/db/part_count.out.txt   |    15 +
 .../src/test/results/db/part_count_by.out.txt   |    13 +
 hplsql/src/test/results/db/select_into.out.txt  |    58 +-
 hplsql/src/test/results/db/select_into2.out.txt |    19 +
 .../results/local/create_procedure2.out.txt     |    10 +
 hplsql/src/test/results/local/if2.out.txt       |     4 +
 hplsql/src/test/results/local/include.out.txt   |     8 +
 hplsql/src/test/results/local/mult_div.out.txt  |     7 +
 .../src/test/results/offline/select_db2.out.txt |     6 +
 itests/hive-unit/pom.xml                        |    35 +
 .../hadoop/hive/metastore/TestAdminUser.java    |     4 +-
 .../hive/metastore/TestHiveMetaStore.java       |     3 +
 .../metastore/hbase/HBaseIntegrationTests.java  |   117 +
 .../TestHBaseAggrStatsCacheIntegration.java     |   691 +
 .../hive/metastore/hbase/TestHBaseImport.java   |   650 +
 .../metastore/hbase/TestHBaseMetastoreSql.java  |   223 +
 .../hbase/TestHBaseStoreIntegration.java        |  1794 +
 .../hbase/TestStorageDescriptorSharing.java     |   191 +
 itests/qtest/pom.xml                            |    10 +-
 itests/util/pom.xml                             |    32 +
 .../metastore/hbase/HBaseStoreTestUtil.java     |    45 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    53 +-
 .../hive/jdbc/ZooKeeperHiveClientHelper.java    |    32 +-
 metastore/if/hive_metastore.thrift              |    54 +
 metastore/pom.xml                               |    82 +
 .../metastore/hbase/HbaseMetastoreProto.java    | 34901 +++++++++++++++++
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  6919 ++--
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |   664 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    25 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  1294 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   371 +
 .../hive/metastore/api/AbortTxnRequest.java     |     2 +-
 .../metastore/api/AddDynamicPartitions.java     |     2 +-
 .../metastore/api/AddPartitionsRequest.java     |     2 +-
 .../hive/metastore/api/AddPartitionsResult.java |     2 +-
 .../hadoop/hive/metastore/api/AggrStats.java    |     2 +-
 .../metastore/api/AlreadyExistsException.java   |     2 +-
 .../metastore/api/BinaryColumnStatsData.java    |     2 +-
 .../metastore/api/BooleanColumnStatsData.java   |     2 +-
 .../hive/metastore/api/CheckLockRequest.java    |     2 +-
 .../metastore/api/ClearFileMetadataRequest.java |   438 +
 .../metastore/api/ClearFileMetadataResult.java  |   283 +
 .../hive/metastore/api/ColumnStatistics.java    |     2 +-
 .../metastore/api/ColumnStatisticsDesc.java     |     2 +-
 .../hive/metastore/api/ColumnStatisticsObj.java |     2 +-
 .../hive/metastore/api/CommitTxnRequest.java    |     2 +-
 .../hive/metastore/api/CompactionRequest.java   |     2 +-
 .../api/ConfigValSecurityException.java         |     2 +-
 .../api/CurrentNotificationEventId.java         |     2 +-
 .../hadoop/hive/metastore/api/Database.java     |     2 +-
 .../apache/hadoop/hive/metastore/api/Date.java  |     2 +-
 .../hive/metastore/api/DateColumnStatsData.java |     2 +-
 .../hadoop/hive/metastore/api/Decimal.java      |     2 +-
 .../metastore/api/DecimalColumnStatsData.java   |     2 +-
 .../metastore/api/DoubleColumnStatsData.java    |     2 +-
 .../hive/metastore/api/DropPartitionsExpr.java  |     2 +-
 .../metastore/api/DropPartitionsRequest.java    |     2 +-
 .../metastore/api/DropPartitionsResult.java     |     2 +-
 .../hive/metastore/api/EnvironmentContext.java  |     2 +-
 .../hadoop/hive/metastore/api/FieldSchema.java  |     2 +-
 .../hive/metastore/api/FireEventRequest.java    |     2 +-
 .../hive/metastore/api/FireEventResponse.java   |     2 +-
 .../hadoop/hive/metastore/api/Function.java     |     2 +-
 .../metastore/api/GetAllFunctionsResponse.java  |    38 +-
 .../api/GetFileMetadataByExprRequest.java       |   548 +
 .../api/GetFileMetadataByExprResult.java        |   703 +
 .../metastore/api/GetFileMetadataRequest.java   |   438 +
 .../metastore/api/GetFileMetadataResult.java    |   540 +
 .../metastore/api/GetOpenTxnsInfoResponse.java  |     2 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |     2 +-
 .../api/GetPrincipalsInRoleRequest.java         |     2 +-
 .../api/GetPrincipalsInRoleResponse.java        |     2 +-
 .../api/GetRoleGrantsForPrincipalRequest.java   |     2 +-
 .../api/GetRoleGrantsForPrincipalResponse.java  |     2 +-
 .../api/GrantRevokePrivilegeRequest.java        |     2 +-
 .../api/GrantRevokePrivilegeResponse.java       |     2 +-
 .../metastore/api/GrantRevokeRoleRequest.java   |     2 +-
 .../metastore/api/GrantRevokeRoleResponse.java  |     2 +-
 .../hive/metastore/api/HeartbeatRequest.java    |     2 +-
 .../metastore/api/HeartbeatTxnRangeRequest.java |     2 +-
 .../api/HeartbeatTxnRangeResponse.java          |     2 +-
 .../hive/metastore/api/HiveObjectPrivilege.java |     2 +-
 .../hive/metastore/api/HiveObjectRef.java       |     2 +-
 .../apache/hadoop/hive/metastore/api/Index.java |     2 +-
 .../api/IndexAlreadyExistsException.java        |     2 +-
 .../metastore/api/InsertEventRequestData.java   |     2 +-
 .../metastore/api/InvalidInputException.java    |     2 +-
 .../metastore/api/InvalidObjectException.java   |     2 +-
 .../api/InvalidOperationException.java          |     2 +-
 .../api/InvalidPartitionException.java          |     2 +-
 .../hive/metastore/api/LockComponent.java       |     2 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |     2 +-
 .../hadoop/hive/metastore/api/LockResponse.java |     2 +-
 .../hive/metastore/api/LongColumnStatsData.java |     2 +-
 .../hive/metastore/api/MetaException.java       |     2 +-
 .../hive/metastore/api/MetadataPpdResult.java   |   508 +
 .../hive/metastore/api/NoSuchLockException.java |     2 +-
 .../metastore/api/NoSuchObjectException.java    |     2 +-
 .../hive/metastore/api/NoSuchTxnException.java  |     2 +-
 .../hive/metastore/api/NotificationEvent.java   |     2 +-
 .../metastore/api/NotificationEventRequest.java |     2 +-
 .../api/NotificationEventResponse.java          |     2 +-
 .../hive/metastore/api/OpenTxnRequest.java      |     2 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Order.java |     2 +-
 .../hadoop/hive/metastore/api/Partition.java    |     2 +-
 .../api/PartitionListComposingSpec.java         |     2 +-
 .../hive/metastore/api/PartitionSpec.java       |     2 +-
 .../api/PartitionSpecWithSharedSD.java          |     2 +-
 .../hive/metastore/api/PartitionWithoutSD.java  |     2 +-
 .../metastore/api/PartitionsByExprRequest.java  |     2 +-
 .../metastore/api/PartitionsByExprResult.java   |     2 +-
 .../metastore/api/PartitionsStatsRequest.java   |     2 +-
 .../metastore/api/PartitionsStatsResult.java    |     2 +-
 .../metastore/api/PrincipalPrivilegeSet.java    |     2 +-
 .../hadoop/hive/metastore/api/PrivilegeBag.java |     2 +-
 .../hive/metastore/api/PrivilegeGrantInfo.java  |     2 +-
 .../metastore/api/PutFileMetadataRequest.java   |   588 +
 .../metastore/api/PutFileMetadataResult.java    |   283 +
 .../hadoop/hive/metastore/api/ResourceUri.java  |     2 +-
 .../apache/hadoop/hive/metastore/api/Role.java  |     2 +-
 .../hive/metastore/api/RolePrincipalGrant.java  |     2 +-
 .../hadoop/hive/metastore/api/Schema.java       |     2 +-
 .../hadoop/hive/metastore/api/SerDeInfo.java    |     2 +-
 .../api/SetPartitionsStatsRequest.java          |     2 +-
 .../hive/metastore/api/ShowCompactRequest.java  |     2 +-
 .../hive/metastore/api/ShowCompactResponse.java |     2 +-
 .../api/ShowCompactResponseElement.java         |     2 +-
 .../hive/metastore/api/ShowLocksRequest.java    |     2 +-
 .../hive/metastore/api/ShowLocksResponse.java   |     2 +-
 .../metastore/api/ShowLocksResponseElement.java |     2 +-
 .../hadoop/hive/metastore/api/SkewedInfo.java   |     2 +-
 .../hive/metastore/api/StorageDescriptor.java   |     2 +-
 .../metastore/api/StringColumnStatsData.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Table.java |     2 +-
 .../hive/metastore/api/TableStatsRequest.java   |     2 +-
 .../hive/metastore/api/TableStatsResult.java    |     2 +-
 .../hive/metastore/api/ThriftHiveMetastore.java |  8422 ++--
 .../hive/metastore/api/TxnAbortedException.java |     2 +-
 .../hadoop/hive/metastore/api/TxnInfo.java      |     2 +-
 .../hive/metastore/api/TxnOpenException.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Type.java  |     2 +-
 .../hive/metastore/api/UnknownDBException.java  |     2 +-
 .../api/UnknownPartitionException.java          |     2 +-
 .../metastore/api/UnknownTableException.java    |     2 +-
 .../hive/metastore/api/UnlockRequest.java       |     2 +-
 .../hadoop/hive/metastore/api/Version.java      |     2 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   |  2810 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |  1009 +-
 .../hive_metastore/ThriftHiveMetastore-remote   |    49 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  1563 +-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   734 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   167 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   267 +
 .../hadoop/hive/metastore/HiveAlterHandler.java |    38 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   266 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   112 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |     6 +
 .../hadoop/hive/metastore/ObjectStore.java      |   418 +-
 .../hive/metastore/PartFilterExprUtil.java      |   149 +
 .../apache/hadoop/hive/metastore/RawStore.java  |    66 +-
 .../hadoop/hive/metastore/RawStoreProxy.java    |     5 +-
 .../hbase/AggrStatsInvalidatorFilter.java       |   121 +
 .../hadoop/hive/metastore/hbase/Counter.java    |    53 +
 .../hive/metastore/hbase/HBaseConnection.java   |    96 +
 .../metastore/hbase/HBaseFilterPlanUtil.java    |   612 +
 .../hive/metastore/hbase/HBaseImport.java       |   535 +
 .../hive/metastore/hbase/HBaseReadWrite.java    |  2106 +
 .../hive/metastore/hbase/HBaseSchemaTool.java   |   239 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |  2387 ++
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |  1340 +
 .../hive/metastore/hbase/ObjectCache.java       |    81 +
 .../hive/metastore/hbase/PartitionCache.java    |   168 +
 .../metastore/hbase/PartitionKeyComparator.java |   292 +
 .../hbase/SharedStorageDescriptor.java          |   251 +
 .../hadoop/hive/metastore/hbase/StatsCache.java |   326 +
 .../metastore/hbase/TephraHBaseConnection.java  |   127 +
 .../metastore/hbase/VanillaHBaseConnection.java |   137 +
 .../stats/BinaryColumnStatsAggregator.java      |    35 +
 .../stats/BooleanColumnStatsAggregator.java     |    35 +
 .../hbase/stats/ColumnStatsAggregator.java      |    26 +
 .../stats/ColumnStatsAggregatorFactory.java     |    94 +
 .../stats/DecimalColumnStatsAggregator.java     |    43 +
 .../stats/DoubleColumnStatsAggregator.java      |    36 +
 .../hbase/stats/LongColumnStatsAggregator.java  |    36 +
 .../stats/StringColumnStatsAggregator.java      |    36 +
 .../hive/metastore/parser/ExpressionTree.java   |     9 +-
 .../metastore/hbase/hbase_metastore_proto.proto |   282 +
 .../DummyRawStoreControlledCommit.java          |    56 +-
 .../DummyRawStoreForJdoConnection.java          |    50 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |    43 +-
 .../hadoop/hive/metastore/hbase/MockUtils.java  |   199 +
 .../hbase/TestHBaseAggregateStatsCache.java     |   316 +
 .../hbase/TestHBaseFilterPlanUtil.java          |   483 +
 .../hive/metastore/hbase/TestHBaseStore.java    |  1307 +
 .../metastore/hbase/TestHBaseStoreCached.java   |   378 +
 .../hbase/TestSharedStorageDescriptor.java      |   153 +
 pom.xml                                         |     3 +-
 .../hadoop/hive/ql/plan/api/Adjacency.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Graph.java   |     2 +-
 .../hadoop/hive/ql/plan/api/Operator.java       |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Query.java   |     2 +-
 .../hadoop/hive/ql/plan/api/QueryPlan.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Stage.java   |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Task.java    |     2 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |     6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    64 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |     9 +-
 .../hadoop/hive/ql/metadata/Partition.java      |    29 +-
 .../AuthorizationPreEventListener.java          |     2 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |     3 +
 .../hive/metastore/TestMetastoreExpr.java       |     2 +-
 .../dynpart_sort_opt_vectorization.q            |     2 +
 .../clientpositive/dynpart_sort_optimization.q  |     2 +
 .../dynpart_sort_opt_vectorization.q.out        |    12 +-
 .../dynpart_sort_optimization.q.out             |    12 +-
 .../tez/dynpart_sort_opt_vectorization.q.out    |    12 +-
 .../tez/dynpart_sort_optimization.q.out         |    12 +-
 ql/src/test/templates/TestCliDriver.vm          |     4 +-
 .../hadoop/hive/serde/test/InnerStruct.java     |     2 +-
 .../hadoop/hive/serde/test/ThriftTestObj.java   |     2 +-
 .../hadoop/hive/serde2/thrift/test/Complex.java |     2 +-
 .../hive/serde2/thrift/test/IntString.java      |     2 +-
 .../hive/serde2/thrift/test/MegaStruct.java     |     2 +-
 .../hive/serde2/thrift/test/MiniStruct.java     |     2 +-
 .../hive/serde2/thrift/test/SetIntString.java   |     2 +-
 .../BinarySortableSerDeWithEndPrefix.java       |    41 +
 .../hadoop/hive/service/HiveClusterStatus.java  |     2 +-
 .../hive/service/HiveServerException.java       |     2 +-
 .../apache/hadoop/hive/service/ThriftHive.java  |     2 +-
 .../service/cli/thrift/TArrayTypeEntry.java     |     2 +-
 .../hive/service/cli/thrift/TBinaryColumn.java  |     2 +-
 .../hive/service/cli/thrift/TBoolColumn.java    |     2 +-
 .../hive/service/cli/thrift/TBoolValue.java     |     2 +-
 .../hive/service/cli/thrift/TByteColumn.java    |     2 +-
 .../hive/service/cli/thrift/TByteValue.java     |     2 +-
 .../hive/service/cli/thrift/TCLIService.java    |     2 +-
 .../cli/thrift/TCancelDelegationTokenReq.java   |     2 +-
 .../cli/thrift/TCancelDelegationTokenResp.java  |     2 +-
 .../service/cli/thrift/TCancelOperationReq.java |     2 +-
 .../cli/thrift/TCancelOperationResp.java        |     2 +-
 .../service/cli/thrift/TCloseOperationReq.java  |     2 +-
 .../service/cli/thrift/TCloseOperationResp.java |     2 +-
 .../service/cli/thrift/TCloseSessionReq.java    |     2 +-
 .../service/cli/thrift/TCloseSessionResp.java   |     2 +-
 .../hive/service/cli/thrift/TColumnDesc.java    |     2 +-
 .../hive/service/cli/thrift/TDoubleColumn.java  |     2 +-
 .../hive/service/cli/thrift/TDoubleValue.java   |     2 +-
 .../cli/thrift/TExecuteStatementReq.java        |     2 +-
 .../cli/thrift/TExecuteStatementResp.java       |     2 +-
 .../service/cli/thrift/TFetchResultsReq.java    |     2 +-
 .../service/cli/thrift/TFetchResultsResp.java   |     2 +-
 .../service/cli/thrift/TGetCatalogsReq.java     |     2 +-
 .../service/cli/thrift/TGetCatalogsResp.java    |     2 +-
 .../hive/service/cli/thrift/TGetColumnsReq.java |     2 +-
 .../service/cli/thrift/TGetColumnsResp.java     |     2 +-
 .../cli/thrift/TGetDelegationTokenReq.java      |     2 +-
 .../cli/thrift/TGetDelegationTokenResp.java     |     2 +-
 .../service/cli/thrift/TGetFunctionsReq.java    |     2 +-
 .../service/cli/thrift/TGetFunctionsResp.java   |     2 +-
 .../hive/service/cli/thrift/TGetInfoReq.java    |     2 +-
 .../hive/service/cli/thrift/TGetInfoResp.java   |     2 +-
 .../cli/thrift/TGetOperationStatusReq.java      |     2 +-
 .../cli/thrift/TGetOperationStatusResp.java     |     2 +-
 .../cli/thrift/TGetResultSetMetadataReq.java    |     2 +-
 .../cli/thrift/TGetResultSetMetadataResp.java   |     2 +-
 .../hive/service/cli/thrift/TGetSchemasReq.java |     2 +-
 .../service/cli/thrift/TGetSchemasResp.java     |     2 +-
 .../service/cli/thrift/TGetTableTypesReq.java   |     2 +-
 .../service/cli/thrift/TGetTableTypesResp.java  |     2 +-
 .../hive/service/cli/thrift/TGetTablesReq.java  |     2 +-
 .../hive/service/cli/thrift/TGetTablesResp.java |     2 +-
 .../service/cli/thrift/TGetTypeInfoReq.java     |     2 +-
 .../service/cli/thrift/TGetTypeInfoResp.java    |     2 +-
 .../service/cli/thrift/THandleIdentifier.java   |     2 +-
 .../hive/service/cli/thrift/TI16Column.java     |     2 +-
 .../hive/service/cli/thrift/TI16Value.java      |     2 +-
 .../hive/service/cli/thrift/TI32Column.java     |     2 +-
 .../hive/service/cli/thrift/TI32Value.java      |     2 +-
 .../hive/service/cli/thrift/TI64Column.java     |     2 +-
 .../hive/service/cli/thrift/TI64Value.java      |     2 +-
 .../hive/service/cli/thrift/TMapTypeEntry.java  |     2 +-
 .../service/cli/thrift/TOpenSessionReq.java     |     2 +-
 .../service/cli/thrift/TOpenSessionResp.java    |     2 +-
 .../service/cli/thrift/TOperationHandle.java    |     2 +-
 .../service/cli/thrift/TPrimitiveTypeEntry.java |     2 +-
 .../cli/thrift/TRenewDelegationTokenReq.java    |     2 +-
 .../cli/thrift/TRenewDelegationTokenResp.java   |     2 +-
 .../apache/hive/service/cli/thrift/TRow.java    |     2 +-
 .../apache/hive/service/cli/thrift/TRowSet.java |     2 +-
 .../hive/service/cli/thrift/TSessionHandle.java |     2 +-
 .../apache/hive/service/cli/thrift/TStatus.java |     2 +-
 .../hive/service/cli/thrift/TStringColumn.java  |     2 +-
 .../hive/service/cli/thrift/TStringValue.java   |     2 +-
 .../service/cli/thrift/TStructTypeEntry.java    |     2 +-
 .../hive/service/cli/thrift/TTableSchema.java   |     2 +-
 .../hive/service/cli/thrift/TTypeDesc.java      |     2 +-
 .../service/cli/thrift/TTypeQualifiers.java     |     2 +-
 .../service/cli/thrift/TUnionTypeEntry.java     |     2 +-
 .../cli/thrift/TUserDefinedTypeEntry.java       |     2 +-
 .../gen-py/hive_service/ThriftHive-remote       |    49 +-
 347 files changed, 76375 insertions(+), 7194 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f84eefa5/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f84eefa5/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f84eefa5/data/conf/tez/hive-site.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f84eefa5/itests/qtest/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f84eefa5/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --cc itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 0f17b1b,aab3aab..4cab161
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@@ -53,11 -54,15 +53,12 @@@ import java.util.concurrent.TimeUnit
  import java.util.regex.Matcher;
  import java.util.regex.Pattern;
  
 -import junit.framework.Assert;
 -import junit.framework.TestSuite;
 -
 -import org.apache.commons.lang.StringUtils;
  import org.apache.commons.io.FileUtils;
  import org.apache.commons.io.IOUtils;
 +import org.apache.commons.lang.StringUtils;
  import org.apache.commons.logging.Log;
  import org.apache.commons.logging.LogFactory;
+ import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.FileStatus;
  import org.apache.hadoop.fs.FileSystem;
  import org.apache.hadoop.fs.Path;
@@@ -347,18 -350,43 +353,44 @@@ public class QTestUtil 
      return "jceks://file" + new Path(keyDir, "test.jks").toUri();
    }
  
+   private void startMiniHBaseCluster() throws Exception {
+     Configuration hbaseConf = HBaseConfiguration.create();
+     hbaseConf.setInt("hbase.master.info.port", -1);
+     utility = new HBaseTestingUtility(hbaseConf);
+     utility.startMiniCluster();
+     conf = new HiveConf(utility.getConfiguration(), Driver.class);
+     HBaseAdmin admin = utility.getHBaseAdmin();
+     // Need to use reflection here to make compilation pass since HBaseIntegrationTests
+     // is not compiled in hadoop-1. All HBaseMetastore tests run under hadoop-2, so this
+     // guarantee HBaseIntegrationTests exist when we hitting this code path
+     java.lang.reflect.Method initHBaseMetastoreMethod = Class.forName(
+         "org.apache.hadoop.hive.metastore.hbase.HBaseStoreTestUtil")
+         .getMethod("initHBaseMetastore", HBaseAdmin.class, HiveConf.class);
+     initHBaseMetastoreMethod.invoke(null, admin, conf);
+   }
+ 
    public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
-        String confDir, String hadoopVer, String initScript,
-        String cleanupScript) throws Exception {
+       String confDir, String hadoopVer, String initScript, String cleanupScript)
+     throws Exception {
+     this(outDir, logDir, clusterType, confDir, hadoopVer, initScript, cleanupScript, false);
+   }
 +
+   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
+       String confDir, String hadoopVer, String initScript, String cleanupScript, boolean useHBaseMetastore)
+     throws Exception {
      this.outDir = outDir;
      this.logDir = logDir;
+     this.useHBaseMetastore = useHBaseMetastore;
  
+     Logger hadoopLog = Logger.getLogger("org.apache.hadoop");
+     hadoopLog.setLevel(Level.INFO);
      if (confDir != null && !confDir.isEmpty()) {
-       HiveConf.setHiveSiteLocation(new URL("file://"
-         + new File(confDir).toURI().getPath() + "/hive-site.xml"));
+       HiveConf.setHiveSiteLocation(new URL("file://"+ new File(confDir).toURI().getPath() + "/hive-site.xml"));
        System.out.println("Setting hive-site: "+HiveConf.getHiveSiteLocation());
      }
+     if (useHBaseMetastore) {
+       startMiniHBaseCluster();
+     }
      conf = new HiveConf(Driver.class);
      this.hadoopVer = getHadoopMainVersion(hadoopVer);
      qMap = new TreeMap<String, String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/f84eefa5/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f84eefa5/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f84eefa5/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------


[14/52] [abbrv] hive git commit: HIVE-11568 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index e58c146,cdbae95..72b2cc3
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@@ -31,6 -33,8 +33,8 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
  public class ThriftHiveMetastore {
  
    /**
@@@ -290,254 -294,254 +296,256 @@@
  
    public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
  
-     public void getMetaConf(String key, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getMetaConf_call> resultHandler) throws org.apache.thrift.TException;
+     public void getMetaConf(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+ 
+     public void setMetaConf(String key, String value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void setMetaConf(String key, String value, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.setMetaConf_call> resultHandler) throws org.apache.thrift.TException;
+     public void create_database(Database database, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void create_database(Database database, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.create_database_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_database(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_database(String name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_database_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_database(String name, boolean deleteData, boolean cascade, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_database(String name, boolean deleteData, boolean cascade, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_database_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_databases(String pattern, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_databases(String pattern, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_databases_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_all_databases(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_all_databases(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_all_databases_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_database(String dbname, Database db, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_database(String dbname, Database db, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_database_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_type(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_type(String name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_type_call> resultHandler) throws org.apache.thrift.TException;
+     public void create_type(Type type, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void create_type(Type type, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.create_type_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_type(String type, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_type(String type, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_type_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_type_all(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_type_all(String name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_type_all_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_fields(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_fields(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_fields_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_fields_with_environment_context(String db_name, String table_name, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_fields_with_environment_context(String db_name, String table_name, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_fields_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_schema(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_schema(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_schema_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_schema_with_environment_context(String db_name, String table_name, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_schema_with_environment_context(String db_name, String table_name, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_schema_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void create_table(Table tbl, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void create_table(Table tbl, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.create_table_call> resultHandler) throws org.apache.thrift.TException;
+     public void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.create_table_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_table(String dbname, String name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_table(String dbname, String name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_table_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_table_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_tables(String db_name, String pattern, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_tables(String db_name, String pattern, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_tables_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_all_tables(String db_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_all_tables(String db_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_all_tables_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_table(String dbname, String tbl_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_table(String dbname, String tbl_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_table_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_table_objects_by_name(String dbname, List<String> tbl_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_table_objects_by_name(String dbname, List<String> tbl_names, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_table_objects_by_name_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_table_names_by_filter(String dbname, String filter, short max_tables, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_table_names_by_filter(String dbname, String filter, short max_tables, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_table_names_by_filter_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_table(String dbname, String tbl_name, Table new_tbl, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_table(String dbname, String tbl_name, Table new_tbl, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_table_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_table_with_environment_context(String dbname, String tbl_name, Table new_tbl, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_table_with_environment_context(String dbname, String tbl_name, Table new_tbl, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_table_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_table_with_cascade(String dbname, String tbl_name, Table new_tbl, boolean cascade, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_table_with_cascade(String dbname, String tbl_name, Table new_tbl, boolean cascade, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_table_with_cascade_call> resultHandler) throws org.apache.thrift.TException;
+     public void add_partition(Partition new_part, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void add_partition(Partition new_part, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_partition_call> resultHandler) throws org.apache.thrift.TException;
+     public void add_partition_with_environment_context(Partition new_part, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void add_partition_with_environment_context(Partition new_part, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_partition_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void add_partitions(List<Partition> new_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void add_partitions(List<Partition> new_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_partitions_call> resultHandler) throws org.apache.thrift.TException;
+     public void add_partitions_pspec(List<PartitionSpec> new_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void add_partitions_pspec(List<PartitionSpec> new_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_partitions_pspec_call> resultHandler) throws org.apache.thrift.TException;
+     public void append_partition(String db_name, String tbl_name, List<String> part_vals, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void append_partition(String db_name, String tbl_name, List<String> part_vals, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.append_partition_call> resultHandler) throws org.apache.thrift.TException;
+     public void add_partitions_req(AddPartitionsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void add_partitions_req(AddPartitionsRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_partitions_req_call> resultHandler) throws org.apache.thrift.TException;
+     public void append_partition_with_environment_context(String db_name, String tbl_name, List<String> part_vals, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void append_partition_with_environment_context(String db_name, String tbl_name, List<String> part_vals, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.append_partition_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void append_partition_by_name(String db_name, String tbl_name, String part_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void append_partition_by_name(String db_name, String tbl_name, String part_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.append_partition_by_name_call> resultHandler) throws org.apache.thrift.TException;
+     public void append_partition_by_name_with_environment_context(String db_name, String tbl_name, String part_name, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void append_partition_by_name_with_environment_context(String db_name, String tbl_name, String part_name, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.append_partition_by_name_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_partition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_partition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_partition_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_partition_with_environment_context(String db_name, String tbl_name, List<String> part_vals, boolean deleteData, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_partition_with_environment_context(String db_name, String tbl_name, List<String> part_vals, boolean deleteData, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_partition_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_partition_by_name(String db_name, String tbl_name, String part_name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_partition_by_name(String db_name, String tbl_name, String part_name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_partition_by_name_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_partition_by_name_with_environment_context(String db_name, String tbl_name, String part_name, boolean deleteData, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_partition_by_name_with_environment_context(String db_name, String tbl_name, String part_name, boolean deleteData, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_partition_by_name_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_partitions_req(DropPartitionsRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_partitions_req(DropPartitionsRequest req, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_partitions_req_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partition(String db_name, String tbl_name, List<String> part_vals, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partition(String db_name, String tbl_name, List<String> part_vals, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partition_call> resultHandler) throws org.apache.thrift.TException;
+     public void exchange_partition(Map<String,String> partitionSpecs, String source_db, String source_table_name, String dest_db, String dest_table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void exchange_partition(Map<String,String> partitionSpecs, String source_db, String source_table_name, String dest_db, String dest_table_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.exchange_partition_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partition_with_auth(String db_name, String tbl_name, List<String> part_vals, String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partition_with_auth(String db_name, String tbl_name, List<String> part_vals, String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partition_with_auth_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partition_by_name(String db_name, String tbl_name, String part_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partition_by_name(String db_name, String tbl_name, String part_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partition_by_name_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions(String db_name, String tbl_name, short max_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions(String db_name, String tbl_name, short max_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions_with_auth(String db_name, String tbl_name, short max_parts, String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions_with_auth(String db_name, String tbl_name, short max_parts, String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_with_auth_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions_pspec(String db_name, String tbl_name, int max_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions_pspec(String db_name, String tbl_name, int max_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_pspec_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partition_names(String db_name, String tbl_name, short max_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partition_names(String db_name, String tbl_name, short max_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partition_names_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions_ps(String db_name, String tbl_name, List<String> part_vals, short max_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions_ps(String db_name, String tbl_name, List<String> part_vals, short max_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_ps_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions_ps_with_auth(String db_name, String tbl_name, List<String> part_vals, short max_parts, String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions_ps_with_auth(String db_name, String tbl_name, List<String> part_vals, short max_parts, String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_ps_with_auth_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partition_names_ps(String db_name, String tbl_name, List<String> part_vals, short max_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partition_names_ps(String db_name, String tbl_name, List<String> part_vals, short max_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partition_names_ps_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions_by_filter(String db_name, String tbl_name, String filter, short max_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions_by_filter(String db_name, String tbl_name, String filter, short max_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_by_filter_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_part_specs_by_filter(String db_name, String tbl_name, String filter, int max_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_part_specs_by_filter(String db_name, String tbl_name, String filter, int max_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_part_specs_by_filter_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions_by_expr(PartitionsByExprRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions_by_expr(PartitionsByExprRequest req, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_by_expr_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions_by_names(String db_name, String tbl_name, List<String> names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions_by_names(String db_name, String tbl_name, List<String> names, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_by_names_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_partition(String db_name, String tbl_name, Partition new_part, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_partition(String db_name, String tbl_name, Partition new_part, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_partition_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_partitions(String db_name, String tbl_name, List<Partition> new_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_partitions(String db_name, String tbl_name, List<Partition> new_parts, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_partitions_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_partition_with_environment_context(String db_name, String tbl_name, Partition new_part, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_partition_with_environment_context(String db_name, String tbl_name, Partition new_part, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_partition_with_environment_context_call> resultHandler) throws org.apache.thrift.TException;
+     public void rename_partition(String db_name, String tbl_name, List<String> part_vals, Partition new_part, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void rename_partition(String db_name, String tbl_name, List<String> part_vals, Partition new_part, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.rename_partition_call> resultHandler) throws org.apache.thrift.TException;
+     public void partition_name_has_valid_characters(List<String> part_vals, boolean throw_exception, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void partition_name_has_valid_characters(List<String> part_vals, boolean throw_exception, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.partition_name_has_valid_characters_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_config_value(String name, String defaultValue, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_config_value(String name, String defaultValue, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_config_value_call> resultHandler) throws org.apache.thrift.TException;
+     public void partition_name_to_vals(String part_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void partition_name_to_vals(String part_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.partition_name_to_vals_call> resultHandler) throws org.apache.thrift.TException;
+     public void partition_name_to_spec(String part_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void partition_name_to_spec(String part_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.partition_name_to_spec_call> resultHandler) throws org.apache.thrift.TException;
+     public void markPartitionForEvent(String db_name, String tbl_name, Map<String,String> part_vals, PartitionEventType eventType, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void markPartitionForEvent(String db_name, String tbl_name, Map<String,String> part_vals, PartitionEventType eventType, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.markPartitionForEvent_call> resultHandler) throws org.apache.thrift.TException;
+     public void isPartitionMarkedForEvent(String db_name, String tbl_name, Map<String,String> part_vals, PartitionEventType eventType, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void isPartitionMarkedForEvent(String db_name, String tbl_name, Map<String,String> part_vals, PartitionEventType eventType, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.isPartitionMarkedForEvent_call> resultHandler) throws org.apache.thrift.TException;
+     public void add_index(Index new_index, Table index_table, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void add_index(Index new_index, Table index_table, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_index_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_index(String dbname, String base_tbl_name, String idx_name, Index new_idx, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_index(String dbname, String base_tbl_name, String idx_name, Index new_idx, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_index_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_index_by_name(String db_name, String tbl_name, String index_name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_index_by_name(String db_name, String tbl_name, String index_name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_index_by_name_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_index_by_name(String db_name, String tbl_name, String index_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_index_by_name(String db_name, String tbl_name, String index_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_index_by_name_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_indexes(String db_name, String tbl_name, short max_indexes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_indexes(String db_name, String tbl_name, short max_indexes, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_indexes_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_index_names(String db_name, String tbl_name, short max_indexes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_index_names(String db_name, String tbl_name, short max_indexes, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_index_names_call> resultHandler) throws org.apache.thrift.TException;
+     public void update_table_column_statistics(ColumnStatistics stats_obj, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void update_table_column_statistics(ColumnStatistics stats_obj, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.update_table_column_statistics_call> resultHandler) throws org.apache.thrift.TException;
+     public void update_partition_column_statistics(ColumnStatistics stats_obj, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void update_partition_column_statistics(ColumnStatistics stats_obj, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.update_partition_column_statistics_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_table_column_statistics(String db_name, String tbl_name, String col_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_table_column_statistics(String db_name, String tbl_name, String col_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_table_column_statistics_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partition_column_statistics(String db_name, String tbl_name, String part_name, String col_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partition_column_statistics(String db_name, String tbl_name, String part_name, String col_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partition_column_statistics_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_table_statistics_req(TableStatsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_table_statistics_req(TableStatsRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_table_statistics_req_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_partitions_statistics_req(PartitionsStatsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_partitions_statistics_req(PartitionsStatsRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_partitions_statistics_req_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_aggr_stats_for(PartitionsStatsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_aggr_stats_for(PartitionsStatsRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_aggr_stats_for_call> resultHandler) throws org.apache.thrift.TException;
+     public void set_aggr_stats_for(SetPartitionsStatsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void set_aggr_stats_for(SetPartitionsStatsRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.set_aggr_stats_for_call> resultHandler) throws org.apache.thrift.TException;
+     public void delete_partition_column_statistics(String db_name, String tbl_name, String part_name, String col_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void delete_partition_column_statistics(String db_name, String tbl_name, String part_name, String col_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.delete_partition_column_statistics_call> resultHandler) throws org.apache.thrift.TException;
+     public void delete_table_column_statistics(String db_name, String tbl_name, String col_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void delete_table_column_statistics(String db_name, String tbl_name, String col_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.delete_table_column_statistics_call> resultHandler) throws org.apache.thrift.TException;
+     public void create_function(Function func, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void create_function(Function func, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.create_function_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_function(String dbName, String funcName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_function(String dbName, String funcName, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_function_call> resultHandler) throws org.apache.thrift.TException;
+     public void alter_function(String dbName, String funcName, Function newFunc, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void alter_function(String dbName, String funcName, Function newFunc, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_function_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_functions(String dbName, String pattern, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_functions(String dbName, String pattern, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_functions_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_function(String dbName, String funcName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_function(String dbName, String funcName, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_function_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_all_functions(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void create_role(Role role, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.create_role_call> resultHandler) throws org.apache.thrift.TException;
+     public void create_role(Role role, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void drop_role(String role_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_role_call> resultHandler) throws org.apache.thrift.TException;
+     public void drop_role(String role_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_role_names(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_role_names_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_role_names(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void grant_role(String role_name, String principal_name, PrincipalType principal_type, String grantor, PrincipalType grantorType, boolean grant_option, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grant_role_call> resultHandler) throws org.apache.thrift.TException;
+     public void grant_role(String role_name, String principal_name, PrincipalType principal_type, String grantor, PrincipalType grantorType, boolean grant_option, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void revoke_role(String role_name, String principal_name, PrincipalType principal_type, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revoke_role_call> resultHandler) throws org.apache.thrift.TException;
+     public void revoke_role(String role_name, String principal_name, PrincipalType principal_type, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void list_roles(String principal_name, PrincipalType principal_type, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.list_roles_call> resultHandler) throws org.apache.thrift.TException;
+     public void list_roles(String principal_name, PrincipalType principal_type, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void grant_revoke_role(GrantRevokeRoleRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grant_revoke_role_call> resultHandler) throws org.apache.thrift.TException;
+     public void grant_revoke_role(GrantRevokeRoleRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_principals_in_role(GetPrincipalsInRoleRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_principals_in_role_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_principals_in_role(GetPrincipalsInRoleRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_role_grants_for_principal(GetRoleGrantsForPrincipalRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_role_grants_for_principal_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_role_grants_for_principal(GetRoleGrantsForPrincipalRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_privilege_set(HiveObjectRef hiveObject, String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_privilege_set_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_privilege_set(HiveObjectRef hiveObject, String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void list_privileges(String principal_name, PrincipalType principal_type, HiveObjectRef hiveObject, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.list_privileges_call> resultHandler) throws org.apache.thrift.TException;
+     public void list_privileges(String principal_name, PrincipalType principal_type, HiveObjectRef hiveObject, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void grant_privileges(PrivilegeBag privileges, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grant_privileges_call> resultHandler) throws org.apache.thrift.TException;
+     public void grant_privileges(PrivilegeBag privileges, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void revoke_privileges(PrivilegeBag privileges, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revoke_privileges_call> resultHandler) throws org.apache.thrift.TException;
+     public void revoke_privileges(PrivilegeBag privileges, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void grant_revoke_privileges(GrantRevokePrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grant_revoke_privileges_call> resultHandler) throws org.apache.thrift.TException;
+     public void grant_revoke_privileges(GrantRevokePrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void set_ugi(String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.set_ugi_call> resultHandler) throws org.apache.thrift.TException;
+     public void set_ugi(String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_delegation_token(String token_owner, String renewer_kerberos_principal_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_delegation_token_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_delegation_token(String token_owner, String renewer_kerberos_principal_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void renew_delegation_token(String token_str_form, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.renew_delegation_token_call> resultHandler) throws org.apache.thrift.TException;
+     public void renew_delegation_token(String token_str_form, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void cancel_delegation_token(String token_str_form, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.cancel_delegation_token_call> resultHandler) throws org.apache.thrift.TException;
+     public void cancel_delegation_token(String token_str_form, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_open_txns(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_open_txns_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_open_txns(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_open_txns_info(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_open_txns_info_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_open_txns_info(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void open_txns(OpenTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.open_txns_call> resultHandler) throws org.apache.thrift.TException;
+     public void open_txns(OpenTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void abort_txn(AbortTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.abort_txn_call> resultHandler) throws org.apache.thrift.TException;
+     public void abort_txn(AbortTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void commit_txn(CommitTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.commit_txn_call> resultHandler) throws org.apache.thrift.TException;
+     public void commit_txn(CommitTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void lock(LockRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.lock_call> resultHandler) throws org.apache.thrift.TException;
+     public void lock(LockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void check_lock(CheckLockRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.check_lock_call> resultHandler) throws org.apache.thrift.TException;
+     public void check_lock(CheckLockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void unlock(UnlockRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.unlock_call> resultHandler) throws org.apache.thrift.TException;
+     public void unlock(UnlockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void show_locks(ShowLocksRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.show_locks_call> resultHandler) throws org.apache.thrift.TException;
+     public void show_locks(ShowLocksRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void heartbeat(HeartbeatRequest ids, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.heartbeat_call> resultHandler) throws org.apache.thrift.TException;
+     public void heartbeat(HeartbeatRequest ids, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void heartbeat_txn_range(HeartbeatTxnRangeRequest txns, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.heartbeat_txn_range_call> resultHandler) throws org.apache.thrift.TException;
+     public void heartbeat_txn_range(HeartbeatTxnRangeRequest txns, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void compact(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.compact_call> resultHandler) throws org.apache.thrift.TException;
+     public void compact(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void show_compact(ShowCompactRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.show_compact_call> resultHandler) throws org.apache.thrift.TException;
+     public void show_compact(ShowCompactRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void add_dynamic_partitions(AddDynamicPartitions rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_dynamic_partitions_call> resultHandler) throws org.apache.thrift.TException;
+     public void add_dynamic_partitions(AddDynamicPartitions rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_next_notification(NotificationEventRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_next_notification_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_next_notification(NotificationEventRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void get_current_notificationEventId(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_current_notificationEventId_call> resultHandler) throws org.apache.thrift.TException;
+     public void get_current_notificationEventId(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void fire_listener_event(FireEventRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.fire_listener_event_call> resultHandler) throws org.apache.thrift.TException;
+     public void fire_listener_event(FireEventRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
-     public void flushCache(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.flushCache_call> resultHandler) throws org.apache.thrift.TException;
++    public void flushCache(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 +
    }
  
    public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface {
@@@ -8613,35 -8652,6 +8675,35 @@@
        }
      }
  
-     public void flushCache(org.apache.thrift.async.AsyncMethodCallback<flushCache_call> resultHandler) throws org.apache.thrift.TException {
++    public void flushCache(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
 +      checkReady();
 +      flushCache_call method_call = new flushCache_call(resultHandler, this, ___protocolFactory, ___transport);
 +      this.___currentMethod = method_call;
 +      ___manager.call(method_call);
 +    }
 +
 +    public static class flushCache_call extends org.apache.thrift.async.TAsyncMethodCall {
-       public flushCache_call(org.apache.thrift.async.AsyncMethodCallback<flushCache_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
++      public flushCache_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
 +        super(client, protocolFactory, transport, resultHandler, false);
 +      }
 +
 +      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
 +        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("flushCache", org.apache.thrift.protocol.TMessageType.CALL, 0));
 +        flushCache_args args = new flushCache_args();
 +        args.write(prot);
 +        prot.writeMessageEnd();
 +      }
 +
 +      public void getResult() throws org.apache.thrift.TException {
 +        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
 +          throw new IllegalStateException("Method call not finished!");
 +        }
 +        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
 +        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
 +        (new Client(prot)).recv_flushCache();
 +      }
 +    }
 +
    }
  
    public static class Processor<I extends Iface> extends com.facebook.fb303.FacebookService.Processor<I> implements org.apache.thrift.TProcessor {
@@@ -11979,96 -12013,256 +12066,277 @@@
        }
      }
  
 +    public static class flushCache<I extends Iface> extends org.apache.thrift.ProcessFunction<I, flushCache_args> {
 +      public flushCache() {
 +        super("flushCache");
 +      }
 +
 +      public flushCache_args getEmptyArgsInstance() {
 +        return new flushCache_args();
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public flushCache_result getResult(I iface, flushCache_args args) throws org.apache.thrift.TException {
 +        flushCache_result result = new flushCache_result();
 +        iface.flushCache();
 +        return result;
 +      }
 +    }
 +
    }
  
-   public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable   {
-     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_args");
- 
-     private static final org.apache.thrift.protocol.TField KEY_FIELD_DESC = new org.apache.thrift.protocol.TField("key", org.apache.thrift.protocol.TType.STRING, (short)1);
+   public static class AsyncProcessor<I extends AsyncIface> extends com.facebook.fb303.FacebookService.AsyncProcessor<I> {
+     private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName());
+     public AsyncProcessor(I iface) {
+       super(iface, getProcessMap(new HashMap<String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+     }
  
-     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-     static {
-       schemes.put(StandardScheme.class, new getMetaConf_argsStandardSchemeFactory());
-       schemes.put(TupleScheme.class, new getMetaConf_argsTupleSchemeFactory());
+     protected AsyncProcessor(I iface, Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+       super(iface, getProcessMap(processMap));
      }
  
-     private String key; // required
+     private static <I extends AsyncIface> Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+       processMap.put("getMetaConf", new getMetaConf());
+       processMap.put("setMetaConf", new setMetaConf());
+       processMap.put("create_database", new create_database());
+       processMap.put("get_database", new get_database());
+       processMap.put("drop_database", new drop_database());
+       processMap.put("get_databases", new get_databases());
+       processMap.put("get_all_databases", new get_all_databases());
+       processMap.put("alter_database", new alter_database());
+       processMap.put("get_type", new get_type());
+       processMap.put("create_type", new create_type());
+       processMap.put("drop_type", new drop_type());
+       processMap.put("get_type_all", new get_type_all());
+       processMap.put("get_fields", new get_fields());
+       processMap.put("get_fields_with_environment_context", new get_fields_with_environment_context());
+       processMap.put("get_schema", new get_schema());
+       processMap.put("get_schema_with_environment_context", new get_schema_with_environment_context());
+       processMap.put("create_table", new create_table());
+       processMap.put("create_table_with_environment_context", new create_table_with_environment_context());
+       processMap.put("drop_table", new drop_table());
+       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
+       processMap.put("get_tables", new get_tables());
+       processMap.put("get_all_tables", new get_all_tables());
+       processMap.put("get_table", new get_table());
+       processMap.put("get_table_objects_by_name", new get_table_objects_by_name());
+       processMap.put("get_table_names_by_filter", new get_table_names_by_filter());
+       processMap.put("alter_table", new alter_table());
+       processMap.put("alter_table_with_environment_context", new alter_table_with_environment_context());
+       processMap.put("alter_table_with_cascade", new alter_table_with_cascade());
+       processMap.put("add_partition", new add_partition());
+       processMap.put("add_partition_with_environment_context", new add_partition_with_environment_context());
+       processMap.put("add_partitions", new add_partitions());
+       processMap.put("add_partitions_pspec", new add_partitions_pspec());
+       processMap.put("append_partition", new append_partition());
+       processMap.put("add_partitions_req", new add_partitions_req());
+       processMap.put("append_partition_with_environment_context", new append_partition_with_environment_context());
+       processMap.put("append_partition_by_name", new append_partition_by_name());
+       processMap.put("append_partition_by_name_with_environment_context", new append_partition_by_name_with_environment_context());
+       processMap.put("drop_partition", new drop_partition());
+       processMap.put("drop_partition_with_environment_context", new drop_partition_with_environment_context());
+       processMap.put("drop_partition_by_name", new drop_partition_by_name());
+       processMap.put("drop_partition_by_name_with_environment_context", new drop_partition_by_name_with_environment_context());
+       processMap.put("drop_partitions_req", new drop_partitions_req());
+       processMap.put("get_partition", new get_partition());
+       processMap.put("exchange_partition", new exchange_partition());
+       processMap.put("get_partition_with_auth", new get_partition_with_auth());
+       processMap.put("get_partition_by_name", new get_partition_by_name());
+       processMap.put("get_partitions", new get_partitions());
+       processMap.put("get_partitions_with_auth", new get_partitions_with_auth());
+       processMap.put("get_partitions_pspec", new get_partitions_pspec());
+       processMap.put("get_partition_names", new get_partition_names());
+       processMap.put("get_partitions_ps", new get_partitions_ps());
+       processMap.put("get_partitions_ps_with_auth", new get_partitions_ps_with_auth());
+       processMap.put("get_partition_names_ps", new get_partition_names_ps());
+       processMap.put("get_partitions_by_filter", new get_partitions_by_filter());
+       processMap.put("get_part_specs_by_filter", new get_part_specs_by_filter());
+       processMap.put("get_partitions_by_expr", new get_partitions_by_expr());
+       processMap.put("get_partitions_by_names", new get_partitions_by_names());
+       processMap.put("alter_partition", new alter_partition());
+       processMap.put("alter_partitions", new alter_partitions());
+       processMap.put("alter_partition_with_environment_context", new alter_partition_with_environment_context());
+       processMap.put("rename_partition", new rename_partition());
+       processMap.put("partition_name_has_valid_characters", new partition_name_has_valid_characters());
+       processMap.put("get_config_value", new get_config_value());
+       processMap.put("partition_name_to_vals", new partition_name_to_vals());
+       processMap.put("partition_name_to_spec", new partition_name_to_spec());
+       processMap.put("markPartitionForEvent", new markPartitionForEvent());
+       processMap.put("isPartitionMarkedForEvent", new isPartitionMarkedForEvent());
+       processMap.put("add_index", new add_index());
+       processMap.put("alter_index", new alter_index());
+       processMap.put("drop_index_by_name", new drop_index_by_name());
+       processMap.put("get_index_by_name", new get_index_by_name());
+       processMap.put("get_indexes", new get_indexes());
+       processMap.put("get_index_names", new get_index_names());
+       processMap.put("update_table_column_statistics", new update_table_column_statistics());
+       processMap.put("update_partition_column_statistics", new update_partition_column_statistics());
+       processMap.put("get_table_column_statistics", new get_table_column_statistics());
+       processMap.put("get_partition_column_statistics", new get_partition_column_statistics());
+       processMap.put("get_table_statistics_req", new get_table_statistics_req());
+       processMap.put("get_partitions_statistics_req", new get_partitions_statistics_req());
+       processMap.put("get_aggr_stats_for", new get_aggr_stats_for());
+       processMap.put("set_aggr_stats_for", new set_aggr_stats_for());
+       processMap.put("delete_partition_column_statistics", new delete_partition_column_statistics());
+       processMap.put("delete_table_column_statistics", new delete_table_column_statistics());
+       processMap.put("create_function", new create_function());
+       processMap.put("drop_function", new drop_function());
+       processMap.put("alter_function", new alter_function());
+       processMap.put("get_functions", new get_functions());
+       processMap.put("get_function", new get_function());
+       processMap.put("get_all_functions", new get_all_functions());
+       processMap.put("create_role", new create_role());
+       processMap.put("drop_role", new drop_role());
+       processMap.put("get_role_names", new get_role_names());
+       processMap.put("grant_role", new grant_role());
+       processMap.put("revoke_role", new revoke_role());
+       processMap.put("list_roles", new list_roles());
+       processMap.put("grant_revoke_role", new grant_revoke_role());
+       processMap.put("get_principals_in_role", new get_principals_in_role());
+       processMap.put("get_role_grants_for_principal", new get_role_grants_for_principal());
+       processMap.put("get_privilege_set", new get_privilege_set());
+       processMap.put("list_privileges", new list_privileges());
+       processMap.put("grant_privileges", new grant_privileges());
+       processMap.put("revoke_privileges", new revoke_privileges());
+       processMap.put("grant_revoke_privileges", new grant_revoke_privileges());
+       processMap.put("set_ugi", new set_ugi());
+       processMap.put("get_delegation_token", new get_delegation_token());
+       processMap.put("renew_delegation_token", new renew_delegation_token());
+       processMap.put("cancel_delegation_token", new cancel_delegation_token());
+       processMap.put("get_open_txns", new get_open_txns());
+       processMap.put("get_open_txns_info", new get_open_txns_info());
+       processMap.put("open_txns", new open_txns());
+       processMap.put("abort_txn", new abort_txn());
+       processMap.put("commit_txn", new commit_txn());
+       processMap.put("lock", new lock());
+       processMap.put("check_lock", new check_lock());
+       processMap.put("unlock", new unlock());
+       processMap.put("show_locks", new show_locks());
+       processMap.put("heartbeat", new heartbeat());
+       processMap.put("heartbeat_txn_range", new heartbeat_txn_range());
+       processMap.put("compact", new compact());
+       processMap.put("show_compact", new show_compact());
+       processMap.put("add_dynamic_partitions", new add_dynamic_partitions());
+       processMap.put("get_next_notification", new get_next_notification());
+       processMap.put("get_current_notificationEventId", new get_current_notificationEventId());
+       processMap.put("fire_listener_event", new fire_listener_event());
++      processMap.put("flushCache", new flushCache());
+       return processMap;
+     }
  
-     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-       KEY((short)1, "key");
+     public static class getMetaConf<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getMetaConf_args, String> {
+       public getMetaConf() {
+         super("getMetaConf");
+       }
  
-       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+       public getMetaConf_args getEmptyArgsInstance() {
+         return new getMetaConf_args();
+       }
  
-       static {
-         for (_Fields field : EnumSet.allOf(_Fields.class)) {
-           byName.put(field.getFieldName(), field);
-         }
+       public AsyncMethodCallback<String> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+         final org.apache.thrift.AsyncProcessFunction fcall = this;
+         return new AsyncMethodCallback<String>() { 
+           public void onComplete(String o) {
+             getMetaConf_result result = new getMetaConf_result();
+             result.success = o;
+             try {
+               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+               return;
+             } catch (Exception e) {
+               LOGGER.error("Exception writing to internal frame buffer", e);
+             }
+             fb.close();
+           }
+           public void onError(Exception e) {
+             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+             org.apache.thrift.TBase msg;
+             getMetaConf_result result = new getMetaConf_result();
+             if (e instanceof MetaException) {
+                         result.o1 = (MetaException) e;
+                         result.setO1IsSet(true);
+                         msg = result;
+             }
+              else 
+             {
+               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+               msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+             }
+             try {
+               fcall.sendResponse(fb,msg,msgType,seqid);
+               return;
+             } catch (Exception ex) {
+               LOGGER.error("Exception writing to internal frame buffer", ex);
+             }
+             fb.close();
+           }
+         };
        }
  
-       /**
-        * Find the _Fields constant that matches fieldId, or null if its not found.
-        */
-       public static _Fields findByThriftId(int fieldId) {
-         switch(fieldId) {
-           case 1: // KEY
-             return KEY;
-           default:
-             return null;
-         }
+       protected boolean isOneway() {
+         return false;
        }
  
-       /**
-        * Find the _Fields constant that matches fieldId, throwing an exception
-        * if it is not found.
-        */
-       public static _Fields findByThriftIdOrThrow(int fieldId) {
-         _Fields fields = findByThriftId(fieldId);
-         if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-         return fields;
+       public void start(I iface, getMetaConf_args args, org.apache.thrift.async.AsyncMethodCallback<String> resultHandler) throws TException {
+         iface.getMetaConf(args.key,resultHandler);
        }
+     }
  
-       /**
-        * Find the _Fields constant that matches name, or null if its not found.
-        */
-       public static _Fields findByName(String name) {
-         return byName.get(name);
+     public static class setMetaConf<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, setMetaConf_args, Void> {
+       public setMetaConf() {
+         super("setMetaConf");
        }
  
-       private final short _thriftId;
-       private final String _fieldName;
+       public setMetaConf_args getEmptyArgsInstance() {
+         return new setMetaConf_args();
+       }
  
-       _Fields(short thriftId, String fieldName) {
-         _thriftId = thriftId;
-         _fieldName = fieldName;
+       public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+         final org.apache.thrift.AsyncProcessFunction fcall = this;
+         return new AsyncMethodCallback<Void>() { 
+           public void onComplete(Void o) {
+             setMetaConf_result result = new setMetaConf_result();
+             try {
+               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+               return;
+             } catch (Exception e) {
+               LOGGER.error("Exception writing to internal frame buffer", e);
+             }
+             fb.close();
+           }
+           public void onError(Exception e) {
+             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+             org.apache.thrift.TBase msg;
+             setMetaConf_result result = new setMetaConf_result();
+             if (e instanceof MetaException) {
+                         result.o1 = (MetaException) e;
+                         result.setO1IsSet(true);
+                         msg = result;
+             }
+              else 
+             {
+               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+               msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+             }
+             try {
+               fcall.sendResponse(fb,msg,msgType,seqid);
+               return;
+             } catch (Exception ex) {
+               LOGGER.error("Exception writing to internal frame buffer", ex);
+             }
+             fb.close();
+           }
+         };
        }
  
-       public short getThriftFieldId() {
-         return _thriftId;
+       protected boolean isOneway() {
+         return false;
        }
  
-       public String getFieldName() {
-         return _fieldName;
+       public void start(I iface, setMetaConf_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+         iface.setMetaConf(args.key, args.value,resultHandler);
        }
      }
  
@@@ -12548,133 -13295,7199 +13369,7249 @@@
        }
      }
  
-     public Object getFieldValue(_Fields field) {
-       switch (field) {
-       case SUCCESS:
-         return getSuccess();
+     public static class drop_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_args, Void> {
+       public drop_table() {
+         super("drop_table");
+       }
  
-       case O1:
-         return getO1();
+       public drop_table_args getEmptyArgsInstance() {
+         return new drop_table_args();
+       }
  
+       public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+         final org.apache.thrift.AsyncProcessFunction fcall = this;
+         return new AsyncMethodCallback<Void>() { 
+           public void onComplete(Void o) {
+             drop_table_result result = new drop_table_result();
+             try {
+               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+               return;
+             } catch (Exception e) {
+               LOGGER.error("Exception writing to internal frame buffer", e);
+             }
+             fb.close();
+           }
+           public void onError(Exception e) {
+             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+             org.apache.thrift.TBase msg;
+             drop_table_result result = new drop_table_result();
+             if (e instanceof NoSuchObjectException) {
+                         result.o1 = (NoSuchObjectException) e;
+                         result.setO1IsSet(true);
+                         msg = result;
+             }
+             else             if (e instanceof MetaException) {
+                         result.o3 = (MetaException) e;
+                         result.setO3IsSet(true);
+                         msg = result;
+             }
+              else 
+             {
+               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+               msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+             }
+             try {
+               fcall.sendResponse(fb,msg,msgType,seqid);
+               return;
+             } catch (Exception ex) {
+               LOGGER.error("Exception writing to internal frame buffer", ex);
+             }
+             fb.close();
+           }
+         };
        }
-       throw new IllegalStateException();
-     }
  
-     /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-     public boolean isSet(_Fields field) {
-       if (field == null) {
-         throw new IllegalArgumentException();
+       protected boolean isOneway() {
+         return false;
        }
  
-       switch (field) {
-       case SUCCESS:
-         return isSetSuccess();
-       case O1:
-         return isSetO1();
+       public void start(I iface, drop_table_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+         iface.drop_table(args.dbname, args.name, args.deleteData,resultHandler);
        }
-       throw new IllegalStateException();
      }
  
-     @Override
-     public boolean equals(Object that) {
-       if (that == null)
-         return false;
-       if (that instanceof getMetaConf_result)
-         return this.equals((getMetaConf_result)that);
-       return false;
-     }
+     public static class drop_table_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_with_environment_context_args, Void> {
+       public drop_table_with_environment_context() {
+         super("drop_table_with_environment_context");
+       }
  
-     public boolean equals(getMetaConf_result that) {
-       if (that == null)
-         return false;
+       public drop_table_with_environment_context_args getEmptyArgsInstance() {
+         return new drop_table_with_environment_context_args();
+       }
  
-       boolean this_present_success = true && this.isSetSuccess();
-       boolean that_present_success = true && that.isSetSuccess();
-       if (this_present_success || that_present_success) {
-         if (!(this_present_success && that_present_success))
-           return false;
-         if (!this.success.equals(that.success))
-           return false;
+       public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+         final org.apache.thrift.AsyncProcessFunction fcall = this;
+         return new AsyncMethodCallback<Void>() { 
+           public void onComplete(Void o) {
+             drop_table_with_environment_context_result result = new drop_table_with_environment_context_result();
+             try {
+               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+               return;
+             } catch (Exception e) {
+               LOGGER.error("Exception writing to internal frame buffer", e);
+             }
+             fb.close();
+           }
+           public void onError(Exception e) {
+             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+             org.apache.thrift.TBase msg;
+             drop_table_with_environment_context_result result = new drop_table_with_environment_context_result();
+             if (e instanceof NoSuchObjectException) {
+                         result.o1 = (NoSuchObjectException) e;
+                         result.setO1IsSet(true);
+                         msg = result;
+             }
+             else             if (e instanceof MetaException) {
+                         result.o3 = (MetaException) e;
+                         result.setO3IsSet(true);
+                         msg = result;
+             }
+              else 
+             {
+               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+               msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+             }
+             try {
+               fcall.sendResponse(fb,msg,msgType,seqid);
+               return;
+             } catch (Exception ex) {
+               LOGGER.error("Exception writing to internal frame buffer", ex);
+             }
+             fb.close();
+           }
+         };
        }
  
-       boolean this_present_o1 = true && this.isSetO1();
-       boolean that_present_o1 = true && that.isSetO1();
-       if (this_present_o1 || that_present_o1) {
-         if (!(this_present_o1 && that_present_o1))
-           return false;
-         if (!this.o1.equals(that.o1))
-           return false;
+       protected boolean isOneway() {
+         return false;
        }
  
-       return true;
+       public void start(I iface, drop_table_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+         iface.drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context,resultHandler);
+       }
      }
  
-     @Override
-     public int hashCode() {
-       HashCodeBuilder builder = new HashCodeBuilder();
- 
-       boolean present_success = true && (isSetSuccess());
-       builder.append(present_success);
-       if (present_success)
-         builder.append(success);
- 
-       boolean present_o1 = true && (isSetO1());
-       builder.append(present_o1);
-       if (present_o1)
-         builder.append(o1);
+     public static class get_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_tables_args, List<String>> {
+       public get_tables() {
+         super("get_tables");
+       }
  
-       return builder.toHashCode();
-     }
+       public get_tables_args getEmptyArgsInstance() {
+         return new get_tables_args();
+       }
  
-     public int compareTo(getMetaConf_result other) {
-       if (!getClass().equals(other.getClass())) {
-         return getClass().getName().compareTo(other.getClass().getName());
+       public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+         final org.apache.thrift.AsyncProcessFunction fcall = this;
+         return new AsyncMethodCallback<List<String>>() { 
+           public void onComplete(List<String> o) {
+             get_tables_result result = new get_tables_result();
+             result.success = o;
+             try {
+               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+               return;
+             } catch (Exception e) {
+               LOGGER.error("Exception writing to internal frame buffer", e);
+             }
+             fb.close();
+           }
+           public void onError(Exception e) {
+             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+             org.apache.thrift.TBase msg;
+             get_tables_result result = new get_tables_result();
+             if (e instanceof MetaException) {
+                         result.o1 = (MetaException) e;
+                         result.setO1IsSet(true);
+                         msg = result;
+             }
+              else 
+             {
+               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+               msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+             }
+             try {
+               fcall.sendResponse(fb,msg,msgType,seqid);
+               return;
+             } catch (Exception ex) {
+               LOGGER.error("Exception writing to internal frame buffer", ex);
+             }
+             fb.close();
+           }
+         };
        }
  
-       int lastComparison = 0;
-       getMetaConf_result typedOther = (getMetaConf_result)other;
+       protected boolean isOneway() {
+         return false;
+       }
  
-       lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
-       if (lastComparison != 0) {
-         return lastComparison;
+       public void start(I iface, get_tables_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
+         iface.get_tables(args.db_name, args.pattern,resultHandler);
        }
-       if (isSetSuccess()) {
-         lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
-         if (lastComparison != 0) {
-           return lastComparison;
-         }
+     }
+ 
+     public static class get_all_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_all_tables_args, List<String>> {
+       public get_all_tables() {
+         super("get_all_tables");
        }
-       lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1());
-       if (lastComparison != 0) {
-         return lastComparison;
+ 
+       public get_all_tables_args getEmptyArgsInstance() {
+         return new get_all_tables_args();
        }
-       if (isSetO1()) {
-         lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1);
-         if (lastComparison != 0) {
-           return lastComparison;
-         }
+ 
+       public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+         final org.apache.thrift.AsyncProcessFunction fcall = this;
+         return new AsyncMethodCallback<List<String>>() { 
+           public void onComplete(List<String> o) {
+             get_all_tables_result result = new get_all_tables_result();
+             result.success = o;
+             try {
+               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+               return;
+             } catch (Exception e) {
+               LOGGER.error("Exception writing to internal frame buffer", e);
+             }
+             fb.close();
+           }
+           public void onError(Exception e) {
+             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+             org.apache.thrift.TBase msg;
+             get_all_tables_result result = new get_all_tables_result();
+             if (e instanceof MetaException) {
+                         result.o1 = (MetaException) e;
+                         result.setO1IsSet(true);
+                         msg = result;
+             }
+              else 
+             {
+               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+               msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+             }
+             try {
+               fcall.sendResponse(fb,msg,msgType,seqid);
+               return;
+             } catch (Exception ex) {
+               LOGGER.error("Exception writing to internal frame buffer", ex);
+             }
+             fb.close();
+           }
+         };
        }
-       return 0;
-     }
  
-     public _Fields fieldForId(int fieldId) {
-       return _Fields.findByThriftId(fieldId);
-     }
+       protected boolean isOneway() {
+         return false;
+       }
  
-     public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-       schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+       public void start(I iface, get_all_tables_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
+         iface.get_all_tables(args.db_name,resultHandler);
+       }
      }
  
-     public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-       schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+     public static class get_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_args, Table> {
+       public get_table() {
+         super("get_table");
        }
  
-     @Override
-     public String toString() {
-       StringBuilder sb = new StringBuilder("getMetaConf_result(");
-       boolean first = true;
+       public get_table_args getEmptyArgsInstance() {
+         return new get_table_args();
+       }
  
-       sb.append("success:");
+       public AsyncMethodCallback<Table> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+         final org.apache.thrift.AsyncProcessFunction fcall = this;
+         return new AsyncMethodCallback<Table>() { 
+           public void onComplete(Table o) {
+             get_table_result result = new get_table_result();
+             result.success = o;
+             try {
+               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+               return;
+             } catch (Exception e) {
+               LOGGER.error("Exception writing to internal frame buffer", e);
+             }
+             fb.close();
+           }
+           public void onError(Exception e) {
+             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+             org.apache.thrift.TBase msg;
+             get_table_result result = new get_table_result();
+             if (e instanceof MetaException) {
+                         result.o1 = (MetaException) e;
+                         result.setO1IsSet(true);
+                         msg = result;
+             }
+             else             if (e instanceof NoSuchObjectException) {
+                         result.o2 = (NoSuchObjectException) e;
+                         result.setO2IsSet(true);
+                         msg = result;
+             }
+              else 
+             {
+               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+               msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+             }
+             try {
+               fcall.sendResponse(fb,msg,msgType,seqid);
+               return;
+             } catch (Exception ex) {
+               LOGGER.error("Exception writing to internal frame buffer", ex);
+             }
+             fb.close();
+           }
+         };
+       }
+ 
+       protected boolean isOneway() {
+         return false;
+       }
+ 
+       public void start(I iface, get_table_args args, org.apache.thrift.async.AsyncMethodCallback<Table> resultHandler) throws TException {
+         iface.get_table(args.dbname, args.tbl_name,resultHandler);
+       }
+     }
+ 
+     public static class get_table_objects_by_name<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_objects_by_name_args, List<Table>> {
+       public get_table_objects_by_name() {
+         super("get_table_objects_by_name");
+       }
+ 
+       public get_table_objects_by_name_args getEmptyArgsInstance() {
+         return new get_table_objects_by_name_args();
+       }
+ 
+       public AsyncMethodCallback<List<Table>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+         final org.apache.thrift.AsyncProcessFunction fcall = this;
+         return new AsyncMethodCallback<List<Table>>() { 
+           public void onComplete(List<Table> o) {
+             get_table_objects_by_name_result result = new get_table_objects_by_name_result();
+             result.success = o;
+             try {
+               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+               return;
+             } catch (Exception e) {
+               LOGGER.error("Exception writing to internal frame buffer", e);
+             }
+             fb.close();
+           }
+           public void onError(Exception e) {
+             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+             org.apache.thrift.TBase msg;
+             get_table_objects_by_name_result result = new get_table_objects_by_name_result();
+             if (e instanceof MetaException) {
+               

<TRUNCATED>

[45/52] [abbrv] hive git commit: HIVE-11743: HBase Port conflict for MiniHBaseCluster

Posted by se...@apache.org.
HIVE-11743: HBase Port conflict for MiniHBaseCluster


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

Branch: refs/heads/llap
Commit: 76828e0ade2e55d30b0680c7312737a8212a158f
Parents: 757553e
Author: Daniel Dai <da...@hortonworks.com>
Authored: Sat Sep 5 11:12:21 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Sat Sep 5 11:12:21 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/hbase/HBaseIntegrationTests.java   | 8 +++++++-
 .../src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java   | 6 +++++-
 2 files changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/76828e0a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
index 02e481a..5b82579 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
@@ -19,8 +19,11 @@
 package org.apache.hadoop.hive.metastore.hbase;
 
 import co.cask.tephra.hbase10.coprocessor.TransactionProcessor;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -32,6 +35,7 @@ import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 import org.apache.hadoop.hive.ql.session.SessionState;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
@@ -60,7 +64,9 @@ public class HBaseIntegrationTests {
     if (testingTephra) {
       LOG.info("Testing with Tephra");
     }
-    utility = new HBaseTestingUtility();
+    Configuration hbaseConf = HBaseConfiguration.create();
+    hbaseConf.setInt("hbase.master.info.port", -1);
+    utility = new HBaseTestingUtility(hbaseConf);
     utility.startMiniCluster();
     conf = new HiveConf(utility.getConfiguration(), HBaseIntegrationTests.class);
     admin = utility.getHBaseAdmin();

http://git-wip-us.apache.org/repos/asf/hive/blob/76828e0a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 24a85cf..16e73c9 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -62,9 +62,11 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
@@ -348,7 +350,9 @@ public class QTestUtil {
   }
 
   private void startMiniHBaseCluster() throws Exception {
-    utility = new HBaseTestingUtility();
+    Configuration hbaseConf = HBaseConfiguration.create();
+    hbaseConf.setInt("hbase.master.info.port", -1);
+    utility = new HBaseTestingUtility(hbaseConf);
     utility.startMiniCluster();
     conf = new HiveConf(utility.getConfiguration(), Driver.class);
     HBaseAdmin admin = utility.getHBaseAdmin();


[21/52] [abbrv] hive git commit: HIVE-10289: Support filter on non-first partition key and non-string partition key (Daniel Dai reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
index 5c5818a..39a7278 100644
--- a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
+++ b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
@@ -30617,6 +30617,3675 @@ public final class HbaseMetastoreProto {
     // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.Table)
   }
 
+  public interface PartitionKeyComparatorOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string names = 1;
+    /**
+     * <code>required string names = 1;</code>
+     */
+    boolean hasNames();
+    /**
+     * <code>required string names = 1;</code>
+     */
+    java.lang.String getNames();
+    /**
+     * <code>required string names = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getNamesBytes();
+
+    // required string types = 2;
+    /**
+     * <code>required string types = 2;</code>
+     */
+    boolean hasTypes();
+    /**
+     * <code>required string types = 2;</code>
+     */
+    java.lang.String getTypes();
+    /**
+     * <code>required string types = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getTypesBytes();
+
+    // repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator op = 3;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator op = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator> 
+        getOpList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator op = 3;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator getOp(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator op = 3;</code>
+     */
+    int getOpCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator op = 3;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.OperatorOrBuilder> 
+        getOpOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator op = 3;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.OperatorOrBuilder getOpOrBuilder(
+        int index);
+
+    // repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range range = 4;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range range = 4;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range> 
+        getRangeList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range range = 4;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range getRange(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range range = 4;</code>
+     */
+    int getRangeCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range range = 4;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.RangeOrBuilder> 
+        getRangeOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range range = 4;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.RangeOrBuilder getRangeOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator}
+   */
+  public static final class PartitionKeyComparator extends
+      com.google.protobuf.GeneratedMessage
+      implements PartitionKeyComparatorOrBuilder {
+    // Use PartitionKeyComparator.newBuilder() to construct.
+    private PartitionKeyComparator(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private PartitionKeyComparator(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final PartitionKeyComparator defaultInstance;
+    public static PartitionKeyComparator getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public PartitionKeyComparator getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private PartitionKeyComparator(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              names_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              types_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                op_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              op_.add(input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.PARSER, extensionRegistry));
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+                range_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range>();
+                mutable_bitField0_ |= 0x00000008;
+              }
+              range_.add(input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          op_ = java.util.Collections.unmodifiableList(op_);
+        }
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+          range_ = java.util.Collections.unmodifiableList(range_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<PartitionKeyComparator> PARSER =
+        new com.google.protobuf.AbstractParser<PartitionKeyComparator>() {
+      public PartitionKeyComparator parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new PartitionKeyComparator(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<PartitionKeyComparator> getParserForType() {
+      return PARSER;
+    }
+
+    public interface MarkOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string value = 1;
+      /**
+       * <code>required string value = 1;</code>
+       */
+      boolean hasValue();
+      /**
+       * <code>required string value = 1;</code>
+       */
+      java.lang.String getValue();
+      /**
+       * <code>required string value = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getValueBytes();
+
+      // required bool inclusive = 2;
+      /**
+       * <code>required bool inclusive = 2;</code>
+       */
+      boolean hasInclusive();
+      /**
+       * <code>required bool inclusive = 2;</code>
+       */
+      boolean getInclusive();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark}
+     */
+    public static final class Mark extends
+        com.google.protobuf.GeneratedMessage
+        implements MarkOrBuilder {
+      // Use Mark.newBuilder() to construct.
+      private Mark(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private Mark(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final Mark defaultInstance;
+      public static Mark getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public Mark getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private Mark(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                value_ = input.readBytes();
+                break;
+              }
+              case 16: {
+                bitField0_ |= 0x00000002;
+                inclusive_ = input.readBool();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Mark_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Mark_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<Mark> PARSER =
+          new com.google.protobuf.AbstractParser<Mark>() {
+        public Mark parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new Mark(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<Mark> getParserForType() {
+        return PARSER;
+      }
+
+      private int bitField0_;
+      // required string value = 1;
+      public static final int VALUE_FIELD_NUMBER = 1;
+      private java.lang.Object value_;
+      /**
+       * <code>required string value = 1;</code>
+       */
+      public boolean hasValue() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string value = 1;</code>
+       */
+      public java.lang.String getValue() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            value_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string value = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getValueBytes() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          value_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // required bool inclusive = 2;
+      public static final int INCLUSIVE_FIELD_NUMBER = 2;
+      private boolean inclusive_;
+      /**
+       * <code>required bool inclusive = 2;</code>
+       */
+      public boolean hasInclusive() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required bool inclusive = 2;</code>
+       */
+      public boolean getInclusive() {
+        return inclusive_;
+      }
+
+      private void initFields() {
+        value_ = "";
+        inclusive_ = false;
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasValue()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasInclusive()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeBytes(1, getValueBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeBool(2, inclusive_);
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(1, getValueBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBoolSize(2, inclusive_);
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Mark_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Mark_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          }
+        }
+        private static Builder create() {
+          return new Builder();
+        }
+
+        public Builder clear() {
+          super.clear();
+          value_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          inclusive_ = false;
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Mark_descriptor;
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark getDefaultInstanceForType() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark build() {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark buildPartial() {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.value_ = value_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.inclusive_ = inclusive_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark) {
+            return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark other) {
+          if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance()) return this;
+          if (other.hasValue()) {
+            bitField0_ |= 0x00000001;
+            value_ = other.value_;
+            onChanged();
+          }
+          if (other.hasInclusive()) {
+            setInclusive(other.getInclusive());
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasValue()) {
+            
+            return false;
+          }
+          if (!hasInclusive()) {
+            
+            return false;
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required string value = 1;
+        private java.lang.Object value_ = "";
+        /**
+         * <code>required string value = 1;</code>
+         */
+        public boolean hasValue() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string value = 1;</code>
+         */
+        public java.lang.String getValue() {
+          java.lang.Object ref = value_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            value_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string value = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getValueBytes() {
+          java.lang.Object ref = value_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            value_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string value = 1;</code>
+         */
+        public Builder setValue(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 1;</code>
+         */
+        public Builder clearValue() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          value_ = getDefaultInstance().getValue();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 1;</code>
+         */
+        public Builder setValueBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+
+        // required bool inclusive = 2;
+        private boolean inclusive_ ;
+        /**
+         * <code>required bool inclusive = 2;</code>
+         */
+        public boolean hasInclusive() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>required bool inclusive = 2;</code>
+         */
+        public boolean getInclusive() {
+          return inclusive_;
+        }
+        /**
+         * <code>required bool inclusive = 2;</code>
+         */
+        public Builder setInclusive(boolean value) {
+          bitField0_ |= 0x00000002;
+          inclusive_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required bool inclusive = 2;</code>
+         */
+        public Builder clearInclusive() {
+          bitField0_ = (bitField0_ & ~0x00000002);
+          inclusive_ = false;
+          onChanged();
+          return this;
+        }
+
+        // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark)
+      }
+
+      static {
+        defaultInstance = new Mark(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark)
+    }
+
+    public interface RangeOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string key = 1;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      boolean hasKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      java.lang.String getKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getKeyBytes();
+
+      // optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+       */
+      boolean hasStart();
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark getStart();
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder getStartOrBuilder();
+
+      // optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+       */
+      boolean hasEnd();
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark getEnd();
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder getEndOrBuilder();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range}
+     */
+    public static final class Range extends
+        com.google.protobuf.GeneratedMessage
+        implements RangeOrBuilder {
+      // Use Range.newBuilder() to construct.
+      private Range(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private Range(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final Range defaultInstance;
+      public static Range getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public Range getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private Range(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                key_ = input.readBytes();
+                break;
+              }
+              case 18: {
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder subBuilder = null;
+                if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                  subBuilder = start_.toBuilder();
+                }
+                start_ = input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.PARSER, extensionRegistry);
+                if (subBuilder != null) {
+                  subBuilder.mergeFrom(start_);
+                  start_ = subBuilder.buildPartial();
+                }
+                bitField0_ |= 0x00000002;
+                break;
+              }
+              case 26: {
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder subBuilder = null;
+                if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                  subBuilder = end_.toBuilder();
+                }
+                end_ = input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.PARSER, extensionRegistry);
+                if (subBuilder != null) {
+                  subBuilder.mergeFrom(end_);
+                  end_ = subBuilder.buildPartial();
+                }
+                bitField0_ |= 0x00000004;
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Range_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Range_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<Range> PARSER =
+          new com.google.protobuf.AbstractParser<Range>() {
+        public Range parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new Range(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<Range> getParserForType() {
+        return PARSER;
+      }
+
+      private int bitField0_;
+      // required string key = 1;
+      public static final int KEY_FIELD_NUMBER = 1;
+      private java.lang.Object key_;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public boolean hasKey() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public java.lang.String getKey() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            key_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getKeyBytes() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          key_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;
+      public static final int START_FIELD_NUMBER = 2;
+      private org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark start_;
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+       */
+      public boolean hasStart() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark getStart() {
+        return start_;
+      }
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder getStartOrBuilder() {
+        return start_;
+      }
+
+      // optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;
+      public static final int END_FIELD_NUMBER = 3;
+      private org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark end_;
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+       */
+      public boolean hasEnd() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark getEnd() {
+        return end_;
+      }
+      /**
+       * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder getEndOrBuilder() {
+        return end_;
+      }
+
+      private void initFields() {
+        key_ = "";
+        start_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+        end_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasKey()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (hasStart()) {
+          if (!getStart().isInitialized()) {
+            memoizedIsInitialized = 0;
+            return false;
+          }
+        }
+        if (hasEnd()) {
+          if (!getEnd().isInitialized()) {
+            memoizedIsInitialized = 0;
+            return false;
+          }
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeBytes(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeMessage(2, start_);
+        }
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          output.writeMessage(3, end_);
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeMessageSize(2, start_);
+        }
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeMessageSize(3, end_);
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.RangeOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Range_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Range_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+            getStartFieldBuilder();
+            getEndFieldBuilder();
+          }
+        }
+        private static Builder create() {
+          return new Builder();
+        }
+
+        public Builder clear() {
+          super.clear();
+          key_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          if (startBuilder_ == null) {
+            start_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+          } else {
+            startBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000002);
+          if (endBuilder_ == null) {
+            end_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+          } else {
+            endBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000004);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Range_descriptor;
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range getDefaultInstanceForType() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range build() {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range buildPartial() {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.key_ = key_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          if (startBuilder_ == null) {
+            result.start_ = start_;
+          } else {
+            result.start_ = startBuilder_.build();
+          }
+          if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+            to_bitField0_ |= 0x00000004;
+          }
+          if (endBuilder_ == null) {
+            result.end_ = end_;
+          } else {
+            result.end_ = endBuilder_.build();
+          }
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range) {
+            return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range other) {
+          if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range.getDefaultInstance()) return this;
+          if (other.hasKey()) {
+            bitField0_ |= 0x00000001;
+            key_ = other.key_;
+            onChanged();
+          }
+          if (other.hasStart()) {
+            mergeStart(other.getStart());
+          }
+          if (other.hasEnd()) {
+            mergeEnd(other.getEnd());
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasKey()) {
+            
+            return false;
+          }
+          if (hasStart()) {
+            if (!getStart().isInitialized()) {
+              
+              return false;
+            }
+          }
+          if (hasEnd()) {
+            if (!getEnd().isInitialized()) {
+              
+              return false;
+            }
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Range) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required string key = 1;
+        private java.lang.Object key_ = "";
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public boolean hasKey() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public java.lang.String getKey() {
+          java.lang.Object ref = key_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            key_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getKeyBytes() {
+          java.lang.Object ref = key_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            key_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKey(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder clearKey() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          key_ = getDefaultInstance().getKey();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKeyBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+
+        // optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;
+        private org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark start_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+        private com.google.protobuf.SingleFieldBuilder<
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder> startBuilder_;
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        public boolean hasStart() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark getStart() {
+          if (startBuilder_ == null) {
+            return start_;
+          } else {
+            return startBuilder_.getMessage();
+          }
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        public Builder setStart(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark value) {
+          if (startBuilder_ == null) {
+            if (value == null) {
+              throw new NullPointerException();
+            }
+            start_ = value;
+            onChanged();
+          } else {
+            startBuilder_.setMessage(value);
+          }
+          bitField0_ |= 0x00000002;
+          return this;
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        public Builder setStart(
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder builderForValue) {
+          if (startBuilder_ == null) {
+            start_ = builderForValue.build();
+            onChanged();
+          } else {
+            startBuilder_.setMessage(builderForValue.build());
+          }
+          bitField0_ |= 0x00000002;
+          return this;
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        public Builder mergeStart(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark value) {
+          if (startBuilder_ == null) {
+            if (((bitField0_ & 0x00000002) == 0x00000002) &&
+                start_ != org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance()) {
+              start_ =
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.newBuilder(start_).mergeFrom(value).buildPartial();
+            } else {
+              start_ = value;
+            }
+            onChanged();
+          } else {
+            startBuilder_.mergeFrom(value);
+          }
+          bitField0_ |= 0x00000002;
+          return this;
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        public Builder clearStart() {
+          if (startBuilder_ == null) {
+            start_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+            onChanged();
+          } else {
+            startBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder getStartBuilder() {
+          bitField0_ |= 0x00000002;
+          onChanged();
+          return getStartFieldBuilder().getBuilder();
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder getStartOrBuilder() {
+          if (startBuilder_ != null) {
+            return startBuilder_.getMessageOrBuilder();
+          } else {
+            return start_;
+          }
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark start = 2;</code>
+         */
+        private com.google.protobuf.SingleFieldBuilder<
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder> 
+            getStartFieldBuilder() {
+          if (startBuilder_ == null) {
+            startBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder>(
+                    start_,
+                    getParentForChildren(),
+                    isClean());
+            start_ = null;
+          }
+          return startBuilder_;
+        }
+
+        // optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;
+        private org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark end_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+        private com.google.protobuf.SingleFieldBuilder<
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder> endBuilder_;
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        public boolean hasEnd() {
+          return ((bitField0_ & 0x00000004) == 0x00000004);
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark getEnd() {
+          if (endBuilder_ == null) {
+            return end_;
+          } else {
+            return endBuilder_.getMessage();
+          }
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        public Builder setEnd(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark value) {
+          if (endBuilder_ == null) {
+            if (value == null) {
+              throw new NullPointerException();
+            }
+            end_ = value;
+            onChanged();
+          } else {
+            endBuilder_.setMessage(value);
+          }
+          bitField0_ |= 0x00000004;
+          return this;
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        public Builder setEnd(
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder builderForValue) {
+          if (endBuilder_ == null) {
+            end_ = builderForValue.build();
+            onChanged();
+          } else {
+            endBuilder_.setMessage(builderForValue.build());
+          }
+          bitField0_ |= 0x00000004;
+          return this;
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        public Builder mergeEnd(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark value) {
+          if (endBuilder_ == null) {
+            if (((bitField0_ & 0x00000004) == 0x00000004) &&
+                end_ != org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance()) {
+              end_ =
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.newBuilder(end_).mergeFrom(value).buildPartial();
+            } else {
+              end_ = value;
+            }
+            onChanged();
+          } else {
+            endBuilder_.mergeFrom(value);
+          }
+          bitField0_ |= 0x00000004;
+          return this;
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        public Builder clearEnd() {
+          if (endBuilder_ == null) {
+            end_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.getDefaultInstance();
+            onChanged();
+          } else {
+            endBuilder_.clear();
+          }
+          bitField0_ = (bitField0_ & ~0x00000004);
+          return this;
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder getEndBuilder() {
+          bitField0_ |= 0x00000004;
+          onChanged();
+          return getEndFieldBuilder().getBuilder();
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder getEndOrBuilder() {
+          if (endBuilder_ != null) {
+            return endBuilder_.getMessageOrBuilder();
+          } else {
+            return end_;
+          }
+        }
+        /**
+         * <code>optional .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Mark end = 3;</code>
+         */
+        private com.google.protobuf.SingleFieldBuilder<
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder> 
+            getEndFieldBuilder() {
+          if (endBuilder_ == null) {
+            endBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Mark.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.MarkOrBuilder>(
+                    end_,
+                    getParentForChildren(),
+                    isClean());
+            end_ = null;
+          }
+          return endBuilder_;
+        }
+
+        // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range)
+      }
+
+      static {
+        defaultInstance = new Range(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Range)
+    }
+
+    public interface OperatorOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator.Type type = 1;
+      /**
+       * <code>required .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator.Type type = 1;</code>
+       */
+      boolean hasType();
+      /**
+       * <code>required .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator.Type type = 1;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.Type getType();
+
+      // required string key = 2;
+      /**
+       * <code>required string key = 2;</code>
+       */
+      boolean hasKey();
+      /**
+       * <code>required string key = 2;</code>
+       */
+      java.lang.String getKey();
+      /**
+       * <code>required string key = 2;</code>
+       */
+      com.google.protobuf.ByteString
+          getKeyBytes();
+
+      // required string val = 3;
+      /**
+       * <code>required string val = 3;</code>
+       */
+      boolean hasVal();
+      /**
+       * <code>required string val = 3;</code>
+       */
+      java.lang.String getVal();
+      /**
+       * <code>required string val = 3;</code>
+       */
+      com.google.protobuf.ByteString
+          getValBytes();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator}
+     */
+    public static final class Operator extends
+        com.google.protobuf.GeneratedMessage
+        implements OperatorOrBuilder {
+      // Use Operator.newBuilder() to construct.
+      private Operator(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private Operator(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final Operator defaultInstance;
+      public static Operator getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public Operator getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private Operator(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 8: {
+                int rawValue = input.readEnum();
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.Type value = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.Type.valueOf(rawValue);
+                if (value == null) {
+                  unknownFields.mergeVarintField(1, rawValue);
+                } else {
+                  bitField0_ |= 0x00000001;
+                  type_ = value;
+                }
+                break;
+              }
+              case 18: {
+                bitField0_ |= 0x00000002;
+                key_ = input.readBytes();
+                break;
+              }
+              case 26: {
+                bitField0_ |= 0x00000004;
+                val_ = input.readBytes();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Operator_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PartitionKeyComparator_Operator_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<Operator> PARSER =
+          new com.google.protobuf.AbstractParser<Operator>() {
+        public Operator parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new Operator(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<Operator> getParserForType() {
+        return PARSER;
+      }
+
+      /**
+       * Protobuf enum {@code org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator.Type}
+       */
+      public enum Type
+          implements com.google.protobuf.ProtocolMessageEnum {
+        /**
+         * <code>LIKE = 0;</code>
+         */
+        LIKE(0, 0),
+        /**
+         * <code>NOTEQUALS = 1;</code>
+         */
+        NOTEQUALS(1, 1),
+        ;
+
+        /**
+         * <code>LIKE = 0;</code>
+         */
+        public static final int LIKE_VALUE = 0;
+        /**
+         * <code>NOTEQUALS = 1;</code>
+         */
+        public static final int NOTEQUALS_VALUE = 1;
+
+
+        public final int getNumber() { return value; }
+
+        public static Type valueOf(int value) {
+          switch (value) {
+            case 0: return LIKE;
+            case 1: return NOTEQUALS;
+            default: return null;
+          }
+        }
+
+        public static com.google.protobuf.Internal.EnumLiteMap<Type>
+            internalGetValueMap() {
+          return internalValueMap;
+        }
+        private static com.google.protobuf.Internal.EnumLiteMap<Type>
+            internalValueMap =
+              new com.google.protobuf.Internal.EnumLiteMap<Type>() {
+                public Type findValueByNumber(int number) {
+                  return Type.valueOf(number);
+                }
+              };
+
+        public final com.google.protobuf.Descriptors.EnumValueDescriptor
+            getValueDescriptor() {
+          return getDescriptor().getValues().get(index);
+        }
+        public final com.google.protobuf.Descriptors.EnumDescriptor
+            getDescriptorForType() {
+          return getDescriptor();
+        }
+        public static final com.google.protobuf.Descriptors.EnumDescriptor
+            getDescriptor() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.getDescriptor().getEnumTypes().get(0);
+        }
+
+        private static final Type[] VALUES = values();
+
+        public static Type valueOf(
+            com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          if (desc.getType() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "EnumValueDescriptor is not for this type.");
+          }
+          return VALUES[desc.getIndex()];
+        }
+
+        private final int index;
+        private final int value;
+
+        private Type(int index, int value) {
+          this.index = index;
+          this.value = value;
+        }
+
+        // @@protoc_insertion_point(enum_scope:org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator.Type)
+      }
+
+      private int bitField0_;
+      // required .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator.Type type = 1;
+      public static final int TYPE_FIELD_NUMBER = 1;
+      private org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.Type type_;
+      /**
+       * <code>required .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator.Type type = 1;</code>
+       */
+      public boolean hasType() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator.Type type = 1;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.Type getType() {
+        return type_;
+      }
+
+      // required string key = 2;
+      public static final int KEY_FIELD_NUMBER = 2;
+      private java.lang.Object key_;
+      /**
+       * <code>required string key = 2;</code>
+       */
+      public boolean hasKey() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required string key = 2;</code>
+       */
+      public java.lang.String getKey() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            key_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string key = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getKeyBytes() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          key_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // required string val = 3;
+      public static final int VAL_FIELD_NUMBER = 3;
+      private java.lang.Object val_;
+      /**
+       * <code>required string val = 3;</code>
+       */
+      public boolean hasVal() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required string val = 3;</code>
+       */
+      public java.lang.String getVal() {
+        java.lang.Object ref = val_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            val_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string val = 3;</code>
+       */
+      public com.google.protobuf.ByteString
+          getValBytes() {
+        java.lang.Object ref = val_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          val_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      private void initFields() {
+        type_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.Type.LIKE;
+        key_ = "";
+        val_ = "";
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasType()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasKey()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasVal()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeEnum(1, type_.getNumber());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeBytes(2, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          output.writeBytes(3, getValBytes());
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeEnumSize(1, type_.getNumber());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(2, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(3, getValBytes());
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+      

<TRUNCATED>

[44/52] [abbrv] hive git commit: HIVE-11731: Exclude hbase-metastore in itests for hadoop-1

Posted by se...@apache.org.
HIVE-11731: Exclude hbase-metastore in itests for hadoop-1


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

Branch: refs/heads/llap
Commit: 757553e64280088bb2fc1546ac1259a519d064a6
Parents: 848b977
Author: Daniel Dai <da...@hortonworks.com>
Authored: Thu Sep 3 21:57:42 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Thu Sep 3 21:57:42 2015 -0700

----------------------------------------------------------------------
 .../metastore/hbase/HBaseStoreTestUtil.java     | 45 ++++++++++++++++++++
 1 file changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/757553e6/itests/util/src/main/java/org/apache/hadoop/hive/metastore/hbase/HBaseStoreTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/metastore/hbase/HBaseStoreTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/metastore/hbase/HBaseStoreTestUtil.java
new file mode 100644
index 0000000..1f42007
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/metastore/hbase/HBaseStoreTestUtil.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hive.metastore.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+public class HBaseStoreTestUtil {
+  public static void initHBaseMetastore(HBaseAdmin admin, HiveConf conf) throws Exception {
+    for (String tableName : HBaseReadWrite.tableNames) {
+      List<byte[]> families = HBaseReadWrite.columnFamilies.get(tableName);
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+      for (byte[] family : families) {
+        HColumnDescriptor columnDesc = new HColumnDescriptor(family);
+        desc.addFamily(columnDesc);
+      }
+      admin.createTable(desc);
+    }
+    admin.close();
+    if (conf != null) {
+      HBaseReadWrite.getInstance(conf);
+    }
+  }
+}
\ No newline at end of file


[23/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java
index 5469108..f821459 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCancelDelegationTokenResp implements org.apache.thrift.TBase<TCancelDelegationTokenResp, TCancelDelegationTokenResp._Fields>, java.io.Serializable, Cloneable, Comparable<TCancelDelegationTokenResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationReq.java
index 83d191e..e63145a 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCancelOperationReq implements org.apache.thrift.TBase<TCancelOperationReq, TCancelOperationReq._Fields>, java.io.Serializable, Cloneable, Comparable<TCancelOperationReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationResp.java
index b8d96df..56c9e76 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelOperationResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCancelOperationResp implements org.apache.thrift.TBase<TCancelOperationResp, TCancelOperationResp._Fields>, java.io.Serializable, Cloneable, Comparable<TCancelOperationResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationReq.java
index ca68866..6ad5446 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCloseOperationReq implements org.apache.thrift.TBase<TCloseOperationReq, TCloseOperationReq._Fields>, java.io.Serializable, Cloneable, Comparable<TCloseOperationReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationResp.java
index 092e7e4..3cd3643 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseOperationResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCloseOperationResp implements org.apache.thrift.TBase<TCloseOperationResp, TCloseOperationResp._Fields>, java.io.Serializable, Cloneable, Comparable<TCloseOperationResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionReq.java
index a5d910f..7bca565 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCloseSessionReq implements org.apache.thrift.TBase<TCloseSessionReq, TCloseSessionReq._Fields>, java.io.Serializable, Cloneable, Comparable<TCloseSessionReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionResp.java
index ae4b554..2ee0551 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCloseSessionResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCloseSessionResp implements org.apache.thrift.TBase<TCloseSessionResp, TCloseSessionResp._Fields>, java.io.Serializable, Cloneable, Comparable<TCloseSessionResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TColumnDesc.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TColumnDesc.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TColumnDesc.java
index e31aa81..ad2444e 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TColumnDesc.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TColumnDesc.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TColumnDesc implements org.apache.thrift.TBase<TColumnDesc, TColumnDesc._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnDesc> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnDesc");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleColumn.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleColumn.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleColumn.java
index 7236d90..1f3b77e 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleColumn.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleColumn.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TDoubleColumn implements org.apache.thrift.TBase<TDoubleColumn, TDoubleColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TDoubleColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleColumn");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleValue.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleValue.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleValue.java
index 4b0811e..59203b5 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleValue.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleValue.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TDoubleValue implements org.apache.thrift.TBase<TDoubleValue, TDoubleValue._Fields>, java.io.Serializable, Cloneable, Comparable<TDoubleValue> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleValue");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java
index feaed34..ee6ed29 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteStatementReq, TExecuteStatementReq._Fields>, java.io.Serializable, Cloneable, Comparable<TExecuteStatementReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java
index daf7b5c..074023c 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TExecuteStatementResp implements org.apache.thrift.TBase<TExecuteStatementResp, TExecuteStatementResp._Fields>, java.io.Serializable, Cloneable, Comparable<TExecuteStatementResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsReq.java
index 47d9a0b..6893eb9 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TFetchResultsReq implements org.apache.thrift.TBase<TFetchResultsReq, TFetchResultsReq._Fields>, java.io.Serializable, Cloneable, Comparable<TFetchResultsReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsResp.java
index b9f3ef6..66116ea 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TFetchResultsResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TFetchResultsResp implements org.apache.thrift.TBase<TFetchResultsResp, TFetchResultsResp._Fields>, java.io.Serializable, Cloneable, Comparable<TFetchResultsResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java
index 0e63b9a..ad7ffa5 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetCatalogsReq implements org.apache.thrift.TBase<TGetCatalogsReq, TGetCatalogsReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetCatalogsReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java
index 7067ff7..651b1b0 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetCatalogsResp implements org.apache.thrift.TBase<TGetCatalogsResp, TGetCatalogsResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetCatalogsResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsReq.java
index a153968..a883ab8 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetColumnsReq implements org.apache.thrift.TBase<TGetColumnsReq, TGetColumnsReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetColumnsReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsResp.java
index 49ecb98..0503062 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetColumnsResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetColumnsResp implements org.apache.thrift.TBase<TGetColumnsResp, TGetColumnsResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetColumnsResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java
index 2c2fa0f..5778ea0 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetDelegationTokenReq implements org.apache.thrift.TBase<TGetDelegationTokenReq, TGetDelegationTokenReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetDelegationTokenReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java
index 4222005..dc8ef44 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetDelegationTokenResp implements org.apache.thrift.TBase<TGetDelegationTokenResp, TGetDelegationTokenResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetDelegationTokenResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java
index 5bc0540..8fd9690 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetFunctionsReq implements org.apache.thrift.TBase<TGetFunctionsReq, TGetFunctionsReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetFunctionsReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java
index a64c948..f24183e 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetFunctionsResp implements org.apache.thrift.TBase<TGetFunctionsResp, TGetFunctionsResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetFunctionsResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoReq.java
index 7dd6a14..fac38c8 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetInfoReq implements org.apache.thrift.TBase<TGetInfoReq, TGetInfoReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetInfoReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoResp.java
index 3e16318..c54b6a9 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetInfoResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetInfoResp implements org.apache.thrift.TBase<TGetInfoResp, TGetInfoResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetInfoResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java
index f4d5fae..4cc87d7 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetOperationStatusReq implements org.apache.thrift.TBase<TGetOperationStatusReq, TGetOperationStatusReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetOperationStatusReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java
index 897f7f5..b77148c 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetOperationStatusResp implements org.apache.thrift.TBase<TGetOperationStatusResp, TGetOperationStatusResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetOperationStatusResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java
index 405ffc6..c69bbed 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetResultSetMetadataReq implements org.apache.thrift.TBase<TGetResultSetMetadataReq, TGetResultSetMetadataReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetResultSetMetadataReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java
index 3735310..d308d4c 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetResultSetMetadataResp implements org.apache.thrift.TBase<TGetResultSetMetadataResp, TGetResultSetMetadataResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetResultSetMetadataResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasReq.java
index 33088b4..9f45078 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetSchemasReq implements org.apache.thrift.TBase<TGetSchemasReq, TGetSchemasReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetSchemasReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasResp.java
index ec598a3..6e85540 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetSchemasResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetSchemasResp implements org.apache.thrift.TBase<TGetSchemasResp, TGetSchemasResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetSchemasResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java
index 3e09a64..8321ce1 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetTableTypesReq implements org.apache.thrift.TBase<TGetTableTypesReq, TGetTableTypesReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetTableTypesReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java
index cdba9b5..d7d9dc3 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetTableTypesResp implements org.apache.thrift.TBase<TGetTableTypesResp, TGetTableTypesResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetTableTypesResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java
index 805e69f..d9e9e40 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetTablesReq implements org.apache.thrift.TBase<TGetTablesReq, TGetTablesReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetTablesReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesResp.java
index 0d03dd8..65513a0 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetTablesResp implements org.apache.thrift.TBase<TGetTablesResp, TGetTablesResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetTablesResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java
index 0628e21..47b8b38 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetTypeInfoReq implements org.apache.thrift.TBase<TGetTypeInfoReq, TGetTypeInfoReq._Fields>, java.io.Serializable, Cloneable, Comparable<TGetTypeInfoReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java
index 6058826..1ef8dc5 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TGetTypeInfoResp implements org.apache.thrift.TBase<TGetTypeInfoResp, TGetTypeInfoResp._Fields>, java.io.Serializable, Cloneable, Comparable<TGetTypeInfoResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/THandleIdentifier.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/THandleIdentifier.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/THandleIdentifier.java
index f20c4d0..fec1b78 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/THandleIdentifier.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/THandleIdentifier.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class THandleIdentifier implements org.apache.thrift.TBase<THandleIdentifier, THandleIdentifier._Fields>, java.io.Serializable, Cloneable, Comparable<THandleIdentifier> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THandleIdentifier");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Column.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Column.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Column.java
index 267d109..2634ef9 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Column.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Column.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TI16Column implements org.apache.thrift.TBase<TI16Column, TI16Column._Fields>, java.io.Serializable, Cloneable, Comparable<TI16Column> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Column");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Value.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Value.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Value.java
index ddaf9ef..afdc29f 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Value.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Value.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TI16Value implements org.apache.thrift.TBase<TI16Value, TI16Value._Fields>, java.io.Serializable, Cloneable, Comparable<TI16Value> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Value");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Column.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Column.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Column.java
index c79df4f..cd59dc3 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Column.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Column.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TI32Column implements org.apache.thrift.TBase<TI32Column, TI32Column._Fields>, java.io.Serializable, Cloneable, Comparable<TI32Column> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Column");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Value.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Value.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Value.java
index d521373..2886d4c 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Value.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Value.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TI32Value implements org.apache.thrift.TBase<TI32Value, TI32Value._Fields>, java.io.Serializable, Cloneable, Comparable<TI32Value> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Value");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Column.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Column.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Column.java
index 746ade2..fc28197 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Column.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Column.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TI64Column implements org.apache.thrift.TBase<TI64Column, TI64Column._Fields>, java.io.Serializable, Cloneable, Comparable<TI64Column> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Column");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Value.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Value.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Value.java
index b419b80..c628896 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Value.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Value.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TI64Value implements org.apache.thrift.TBase<TI64Value, TI64Value._Fields>, java.io.Serializable, Cloneable, Comparable<TI64Value> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Value");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TMapTypeEntry.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TMapTypeEntry.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TMapTypeEntry.java
index 91a85ef..7a43c4d 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TMapTypeEntry.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TMapTypeEntry.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TMapTypeEntry implements org.apache.thrift.TBase<TMapTypeEntry, TMapTypeEntry._Fields>, java.io.Serializable, Cloneable, Comparable<TMapTypeEntry> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMapTypeEntry");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java
index 657f868..a2f6530 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TOpenSessionReq implements org.apache.thrift.TBase<TOpenSessionReq, TOpenSessionReq._Fields>, java.io.Serializable, Cloneable, Comparable<TOpenSessionReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java
index 48f4b45..607847c 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TOpenSessionResp implements org.apache.thrift.TBase<TOpenSessionResp, TOpenSessionResp._Fields>, java.io.Serializable, Cloneable, Comparable<TOpenSessionResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOperationHandle.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOperationHandle.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOperationHandle.java
index db41117..45a53f6 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOperationHandle.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOperationHandle.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TOperationHandle implements org.apache.thrift.TBase<TOperationHandle, TOperationHandle._Fields>, java.io.Serializable, Cloneable, Comparable<TOperationHandle> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOperationHandle");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java
index f1c8d58..6f246c1 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TPrimitiveTypeEntry implements org.apache.thrift.TBase<TPrimitiveTypeEntry, TPrimitiveTypeEntry._Fields>, java.io.Serializable, Cloneable, Comparable<TPrimitiveTypeEntry> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPrimitiveTypeEntry");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java
index 91f8b00..c7708e5 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TRenewDelegationTokenReq implements org.apache.thrift.TBase<TRenewDelegationTokenReq, TRenewDelegationTokenReq._Fields>, java.io.Serializable, Cloneable, Comparable<TRenewDelegationTokenReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenReq");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java
index c01cc3f..38cc331 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TRenewDelegationTokenResp implements org.apache.thrift.TBase<TRenewDelegationTokenResp, TRenewDelegationTokenResp._Fields>, java.io.Serializable, Cloneable, Comparable<TRenewDelegationTokenResp> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenResp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRow.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRow.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRow.java
index 197bab6..bbab399 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRow.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRow.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TRow implements org.apache.thrift.TBase<TRow, TRow._Fields>, java.io.Serializable, Cloneable, Comparable<TRow> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRow");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java
index cc1a148..dc93ff9 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TRowSet implements org.apache.thrift.TBase<TRowSet, TRowSet._Fields>, java.io.Serializable, Cloneable, Comparable<TRowSet> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowSet");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TSessionHandle.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TSessionHandle.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TSessionHandle.java
index 264e155..4ab6a3e 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TSessionHandle.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TSessionHandle.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TSessionHandle implements org.apache.thrift.TBase<TSessionHandle, TSessionHandle._Fields>, java.io.Serializable, Cloneable, Comparable<TSessionHandle> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSessionHandle");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java
index 1cd7980..1ce3ac7 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TStatus implements org.apache.thrift.TBase<TStatus, TStatus._Fields>, java.io.Serializable, Cloneable, Comparable<TStatus> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStatus");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringColumn.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringColumn.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringColumn.java
index d996529..6883c1a 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringColumn.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringColumn.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TStringColumn implements org.apache.thrift.TBase<TStringColumn, TStringColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TStringColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringColumn");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringValue.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringValue.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringValue.java
index 9655f38..2378060 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringValue.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringValue.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TStringValue implements org.apache.thrift.TBase<TStringValue, TStringValue._Fields>, java.io.Serializable, Cloneable, Comparable<TStringValue> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringValue");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStructTypeEntry.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStructTypeEntry.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStructTypeEntry.java
index d58184c..828b43a 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStructTypeEntry.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStructTypeEntry.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TStructTypeEntry implements org.apache.thrift.TBase<TStructTypeEntry, TStructTypeEntry._Fields>, java.io.Serializable, Cloneable, Comparable<TStructTypeEntry> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStructTypeEntry");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTableSchema.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTableSchema.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTableSchema.java
index 796b73b..f2ef9a4 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTableSchema.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTableSchema.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TTableSchema implements org.apache.thrift.TBase<TTableSchema, TTableSchema._Fields>, java.io.Serializable, Cloneable, Comparable<TTableSchema> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableSchema");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeDesc.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeDesc.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeDesc.java
index 95b4466..9aa071d 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeDesc.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeDesc.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TTypeDesc implements org.apache.thrift.TBase<TTypeDesc, TTypeDesc._Fields>, java.io.Serializable, Cloneable, Comparable<TTypeDesc> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeDesc");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeQualifiers.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeQualifiers.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeQualifiers.java
index 15ac5a9..9480984 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeQualifiers.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeQualifiers.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TTypeQualifiers implements org.apache.thrift.TBase<TTypeQualifiers, TTypeQualifiers._Fields>, java.io.Serializable, Cloneable, Comparable<TTypeQualifiers> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifiers");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java
index eccf303..8ff0766 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TUnionTypeEntry implements org.apache.thrift.TBase<TUnionTypeEntry, TUnionTypeEntry._Fields>, java.io.Serializable, Cloneable, Comparable<TUnionTypeEntry> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUnionTypeEntry");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java
index 21da61b..7ccc1e8 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TUserDefinedTypeEntry implements org.apache.thrift.TBase<TUserDefinedTypeEntry, TUserDefinedTypeEntry._Fields>, java.io.Serializable, Cloneable, Comparable<TUserDefinedTypeEntry> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUserDefinedTypeEntry");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote b/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
index 747c2be..54d59a8 100755
--- a/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
+++ b/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
@@ -157,6 +157,11 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  NotificationEventResponse get_next_notification(NotificationEventRequest rqst)')
   print('  CurrentNotificationEventId get_current_notificationEventId()')
   print('  FireEventResponse fire_listener_event(FireEventRequest rqst)')
+  print('  void flushCache()')
+  print('  GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req)')
+  print('  GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req)')
+  print('  PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req)')
+  print('  ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req)')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -166,8 +171,8 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void setOption(string key, string value)')
   print('  string getOption(string key)')
   print('   getOptions()')
+  print('  string getCpuProfile(i32 profileDurationInSec)')
   print('  i64 aliveSince()')
-  print('  reflection_limited.Service getLimitedReflection()')
   print('  void reinitialize()')
   print('  void shutdown()')
   print('')
@@ -1024,6 +1029,36 @@ elif cmd == 'fire_listener_event':
     sys.exit(1)
   pp.pprint(client.fire_listener_event(eval(args[0]),))
 
+elif cmd == 'flushCache':
+  if len(args) != 0:
+    print('flushCache requires 0 args')
+    sys.exit(1)
+  pp.pprint(client.flushCache())
+
+elif cmd == 'get_file_metadata_by_expr':
+  if len(args) != 1:
+    print('get_file_metadata_by_expr requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_file_metadata_by_expr(eval(args[0]),))
+
+elif cmd == 'get_file_metadata':
+  if len(args) != 1:
+    print('get_file_metadata requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_file_metadata(eval(args[0]),))
+
+elif cmd == 'put_file_metadata':
+  if len(args) != 1:
+    print('put_file_metadata requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.put_file_metadata(eval(args[0]),))
+
+elif cmd == 'clear_file_metadata':
+  if len(args) != 1:
+    print('clear_file_metadata requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.clear_file_metadata(eval(args[0]),))
+
 elif cmd == 'getName':
   if len(args) != 0:
     print('getName requires 0 args')
@@ -1078,18 +1113,18 @@ elif cmd == 'getOptions':
     sys.exit(1)
   pp.pprint(client.getOptions())
 
+elif cmd == 'getCpuProfile':
+  if len(args) != 1:
+    print('getCpuProfile requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.getCpuProfile(eval(args[0]),))
+
 elif cmd == 'aliveSince':
   if len(args) != 0:
     print('aliveSince requires 0 args')
     sys.exit(1)
   pp.pprint(client.aliveSince())
 
-elif cmd == 'getLimitedReflection':
-  if len(args) != 0:
-    print('getLimitedReflection requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getLimitedReflection())
-
 elif cmd == 'reinitialize':
   if len(args) != 0:
     print('reinitialize requires 0 args')


[39/52] [abbrv] hive git commit: HIVE-11654 After HIVE-10289, HBase metastore tests failing (Daniel Dai via gates)

Posted by se...@apache.org.
HIVE-11654 After HIVE-10289, HBase metastore tests failing (Daniel Dai via gates)


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

Branch: refs/heads/llap
Commit: e150af9457079c87c267094f3861528286e951ea
Parents: f014f0d
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Aug 28 10:48:35 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Aug 28 10:48:35 2015 -0700

----------------------------------------------------------------------
 .../TestHBaseAggrStatsCacheIntegration.java     |  4 +-
 .../hive/metastore/hbase/HBaseReadWrite.java    | 36 ++++-----
 .../hadoop/hive/metastore/hbase/HBaseStore.java | 79 ++++++++++++++------
 .../hadoop/hive/metastore/hbase/HBaseUtils.java | 36 ++++-----
 .../hive/metastore/hbase/TestHBaseStore.java    | 73 +++++++++++++-----
 5 files changed, 144 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e150af94/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java
index ad76b2e..899fee1 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java
@@ -140,7 +140,7 @@ public class TestHBaseAggrStatsCacheIntegration extends HBaseIntegrationTests {
     Checker statChecker = new Checker() {
       @Override
       public void checkStats(AggrStats aggrStats) throws Exception {
-        Assert.assertEquals(4, aggrStats.getPartsFound());
+        Assert.assertEquals(2, aggrStats.getPartsFound());
         Assert.assertEquals(2, aggrStats.getColStatsSize());
         ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
         Assert.assertEquals("col1", cso.getColName());
@@ -152,7 +152,7 @@ public class TestHBaseAggrStatsCacheIntegration extends HBaseIntegrationTests {
 
         cso = aggrStats.getColStats().get(1);
         Assert.assertEquals("col2", cso.getColName());
-        Assert.assertEquals("string", cso.getColType());
+        Assert.assertEquals("varchar", cso.getColType());
         StringColumnStatsData scsd = cso.getStatsData().getStringStats();
         Assert.assertEquals(10.3, scsd.getAvgColLen(), 0.1);
         Assert.assertEquals(2000, scsd.getMaxColLen());

http://git-wip-us.apache.org/repos/asf/hive/blob/e150af94/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index 8a1448c..d38c561 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -550,7 +550,7 @@ public class HBaseReadWrite {
    * @param newPart partitiion to replace it with
    * @throws IOException
    */
-  void replacePartition(Partition oldPart, Partition newPart) throws IOException {
+  void replacePartition(Partition oldPart, Partition newPart, List<String> partTypes) throws IOException {
     byte[] hash;
     byte[] oldHash = HBaseUtils.hashStorageDescriptor(oldPart.getSd(), md);
     byte[] newHash = HBaseUtils.hashStorageDescriptor(newPart.getSd(), md);
@@ -565,7 +565,7 @@ public class HBaseReadWrite {
     store(PART_TABLE, serialized[0], CATALOG_CF, CATALOG_COL, serialized[1]);
     partCache.put(newPart.getDbName(), newPart.getTableName(), newPart);
     if (!oldPart.getTableName().equals(newPart.getTableName())) {
-      deletePartition(oldPart.getDbName(), oldPart.getTableName(), oldPart.getValues());
+      deletePartition(oldPart.getDbName(), oldPart.getTableName(), partTypes, oldPart.getValues());
     }
   }
 
@@ -592,7 +592,7 @@ public class HBaseReadWrite {
     conn.flush(htab);
   }
 
-  void replacePartitions(List<Partition> oldParts, List<Partition> newParts) throws IOException {
+  void replacePartitions(List<Partition> oldParts, List<Partition> newParts, List<String> oldPartTypes) throws IOException {
     if (oldParts.size() != newParts.size()) {
       throw new RuntimeException("Number of old and new partitions must match.");
     }
@@ -616,7 +616,7 @@ public class HBaseReadWrite {
       partCache.put(newParts.get(i).getDbName(), newParts.get(i).getTableName(), newParts.get(i));
       if (!newParts.get(i).getTableName().equals(oldParts.get(i).getTableName())) {
         // We need to remove the old record as well.
-        deletePartition(oldParts.get(i).getDbName(), oldParts.get(i).getTableName(),
+        deletePartition(oldParts.get(i).getDbName(), oldParts.get(i).getTableName(), oldPartTypes,
             oldParts.get(i).getValues(), false);
       }
     }
@@ -710,16 +710,17 @@ public class HBaseReadWrite {
            i < table.getPartitionKeys().size() && i < partVals.size(); i++) {
 
         if ("*".equals(partVals.get(i))) {
+          PartitionKeyComparator.Operator op = new PartitionKeyComparator.Operator(
+              PartitionKeyComparator.Operator.Type.LIKE,
+              table.getPartitionKeys().get(i).getName(),
+              ".*");
+          ops.add(op);
+        } else {
           PartitionKeyComparator.Range range = new PartitionKeyComparator.Range(
               table.getPartitionKeys().get(i).getName(),
               new PartitionKeyComparator.Mark(partVals.get(i), true),
               new PartitionKeyComparator.Mark(partVals.get(i), true));
           ranges.add(range);
-        } else {
-          PartitionKeyComparator.Operator op = new PartitionKeyComparator.Operator(
-              PartitionKeyComparator.Operator.Type.LIKE,
-              table.getPartitionKeys().get(i).getName(),
-              ".*");
         }
       }
     }
@@ -771,20 +772,20 @@ public class HBaseReadWrite {
    *                 partition columns they are values for
    * @throws IOException
    */
-  void deletePartition(String dbName, String tableName, List<String> partVals) throws IOException {
-    deletePartition(dbName, tableName, partVals, true);
+  void deletePartition(String dbName, String tableName, List<String> partTypes,
+      List<String> partVals) throws IOException {
+    deletePartition(dbName, tableName, partTypes, partVals, true);
   }
 
-  private void deletePartition(String dbName, String tableName, List<String> partVals,
-                               boolean decrementRefCnt) throws IOException {
+  private void deletePartition(String dbName, String tableName, List<String> partTypes,
+      List<String> partVals, boolean decrementRefCnt) throws IOException {
     // Find the partition so I can get the storage descriptor and drop it
     partCache.remove(dbName, tableName, partVals);
     if (decrementRefCnt) {
       Partition p = getPartition(dbName, tableName, partVals, false);
       decrementStorageDescriptorRefCount(p.getSd());
     }
-    byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName,
-        HBaseUtils.getPartitionKeyTypes(getTable(dbName, tableName).getPartitionKeys()), partVals);
+    byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName, partTypes, partVals);
     delete(PART_TABLE, key, null, null);
   }
 
@@ -1610,10 +1611,9 @@ public class HBaseReadWrite {
             // recontruct the key.  We have to pull the dbName and tableName out of the key to
             // find the partition values.
             byte[] key = results[i].getRow();
-            List<String> reconstructedKey = HBaseUtils.parseKey(key, HBaseUtils.getPartitionNames(getTable(dbName, tblName).getPartitionKeys()),
-                HBaseUtils.getPartitionKeyTypes(getTable(dbName, tblName).getPartitionKeys()));
             List<String> reconstructedPartVals =
-                reconstructedKey.subList(2, reconstructedKey.size());
+                HBaseUtils.deserializePartitionKey(getTable(dbName, tblName).getPartitionKeys(), key,
+                    staticConf);
             String partName = valToPartMap.get(reconstructedPartVals);
             assert partName != null;
             csd.setIsTblLevel(false);

http://git-wip-us.apache.org/repos/asf/hive/blob/e150af94/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index 568a347..4cda9cc 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -139,9 +139,10 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-
+      Database dbCopy = db.deepCopy();
+      dbCopy.setName(HiveStringUtils.normalizeIdentifier(dbCopy.getName()));
       // HiveMetaStore already checks for existence of the database, don't recheck
-      getHBase().putDb(db);
+      getHBase().putDb(dbCopy);
       commit = true;
     } catch (IOException e) {
       LOG.error("Unable to create database ", e);
@@ -195,7 +196,9 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().putDb(db);
+      Database dbCopy = db.deepCopy();
+      dbCopy.setName(HiveStringUtils.normalizeIdentifier(dbCopy.getName()));
+      getHBase().putDb(dbCopy);
       commit = true;
       return true;
     } catch (IOException e) {
@@ -211,7 +214,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Database> dbs = getHBase().scanDatabases(likeToRegex(pattern));
+      List<Database> dbs = getHBase().scanDatabases(
+          pattern==null?null:HiveStringUtils.normalizeIdentifier(likeToRegex(pattern)));
       List<String> dbNames = new ArrayList<String>(dbs.size());
       for (Database db : dbs) dbNames.add(db.getName());
       commit = true;
@@ -250,7 +254,10 @@ public class HBaseStore implements RawStore {
     openTransaction();
     // HiveMetaStore above us checks if the table already exists, so we can blindly store it here.
     try {
-      getHBase().putTable(tbl);
+      Table tblCopy = tbl.deepCopy();
+      tblCopy.setDbName(HiveStringUtils.normalizeIdentifier(tblCopy.getDbName()));
+      tblCopy.setTableName(HiveStringUtils.normalizeIdentifier(tblCopy.getTableName()));
+      getHBase().putTable(tblCopy);
       commit = true;
     } catch (IOException e) {
       LOG.error("Unable to create table ", e);
@@ -303,7 +310,10 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().putPartition(part);
+      Partition partCopy = part.deepCopy();
+      partCopy.setDbName(HiveStringUtils.normalizeIdentifier(part.getDbName()));
+      partCopy.setTableName(HiveStringUtils.normalizeIdentifier(part.getTableName()));
+      getHBase().putPartition(partCopy);
       commit = true;
       return true;
     } catch (IOException e) {
@@ -320,7 +330,14 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().putPartitions(parts);
+      List<Partition> partsCopy = new ArrayList<Partition>();
+      for (int i=0;i<parts.size();i++) {
+        Partition partCopy = parts.get(i).deepCopy();
+        partCopy.setDbName(HiveStringUtils.normalizeIdentifier(partCopy.getDbName()));
+        partCopy.setTableName(HiveStringUtils.normalizeIdentifier(partCopy.getTableName()));
+        partsCopy.add(i, partCopy);
+      }
+      getHBase().putPartitions(partsCopy);
       commit = true;
       return true;
     } catch (IOException e) {
@@ -383,11 +400,12 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().deletePartition(HiveStringUtils.normalizeIdentifier(dbName),
-          HiveStringUtils.normalizeIdentifier(tableName), part_vals);
+      dbName = HiveStringUtils.normalizeIdentifier(dbName);
+      tableName = HiveStringUtils.normalizeIdentifier(tableName);
+      getHBase().deletePartition(dbName, tableName, HBaseUtils.getPartitionKeyTypes(
+          getTable(dbName, tableName).getPartitionKeys()), part_vals);
       // Drop any cached stats that reference this partitions
-      getHBase().getStatsCache().invalidate(HiveStringUtils.normalizeIdentifier(dbName),
-          HiveStringUtils.normalizeIdentifier(tableName),
+      getHBase().getStatsCache().invalidate(dbName, tableName,
           buildExternalPartName(dbName, tableName, part_vals));
       commit = true;
       return true;
@@ -419,26 +437,31 @@ public class HBaseStore implements RawStore {
   }
 
   @Override
-  public void alterTable(String dbname, String name, Table newTable) throws InvalidObjectException,
+  public void alterTable(String dbName, String tableName, Table newTable) throws InvalidObjectException,
       MetaException {
     boolean commit = false;
     openTransaction();
     try {
-      getHBase().replaceTable(getHBase().getTable(HiveStringUtils.normalizeIdentifier(dbname),
-          HiveStringUtils.normalizeIdentifier(name)), newTable);
+      Table newTableCopy = newTable.deepCopy();
+      newTableCopy.setDbName(HiveStringUtils.normalizeIdentifier(newTableCopy.getDbName()));
+      List<String> oldPartTypes = getTable(dbName, tableName).getPartitionKeys()==null?
+          null:HBaseUtils.getPartitionKeyTypes(getTable(dbName, tableName).getPartitionKeys());
+      newTableCopy.setTableName(HiveStringUtils.normalizeIdentifier(newTableCopy.getTableName()));
+      getHBase().replaceTable(getHBase().getTable(HiveStringUtils.normalizeIdentifier(dbName),
+          HiveStringUtils.normalizeIdentifier(tableName)), newTableCopy);
       if (newTable.getPartitionKeys() != null && newTable.getPartitionKeys().size() > 0
-          && !name.equals(newTable.getTableName())) {
+          && !tableName.equals(newTable.getTableName())) {
         // They renamed the table, so we need to change each partition as well, since it changes
         // the key.
         try {
-          List<Partition> oldParts = getPartitions(dbname, name, -1);
+          List<Partition> oldParts = getPartitions(dbName, tableName, -1);
           List<Partition> newParts = new ArrayList<>(oldParts.size());
           for (Partition oldPart : oldParts) {
             Partition newPart = oldPart.deepCopy();
             newPart.setTableName(newTable.getTableName());
             newParts.add(newPart);
           }
-          getHBase().replacePartitions(oldParts, newParts);
+          getHBase().replacePartitions(oldParts, newParts, oldPartTypes);
         } catch (NoSuchObjectException e) {
           LOG.debug("No partitions found for old table so not worrying about it");
         }
@@ -446,8 +469,8 @@ public class HBaseStore implements RawStore {
       }
       commit = true;
     } catch (IOException e) {
-      LOG.error("Unable to alter table " + tableNameForErrorMsg(dbname, name), e);
-      throw new MetaException("Unable to alter table " + tableNameForErrorMsg(dbname, name));
+      LOG.error("Unable to alter table " + tableNameForErrorMsg(dbName, tableName), e);
+      throw new MetaException("Unable to alter table " + tableNameForErrorMsg(dbName, tableName));
     } finally {
       commitOrRoleBack(commit);
     }
@@ -459,7 +482,7 @@ public class HBaseStore implements RawStore {
     openTransaction();
     try {
       List<Table> tables = getHBase().scanTables(HiveStringUtils.normalizeIdentifier(dbName),
-          likeToRegex(pattern));
+          pattern==null?null:HiveStringUtils.normalizeIdentifier(likeToRegex(pattern)));
       List<String> tableNames = new ArrayList<String>(tables.size());
       for (Table table : tables) tableNames.add(table.getTableName());
       commit = true;
@@ -544,9 +567,13 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
+      Partition new_partCopy = new_part.deepCopy();
+      new_partCopy.setDbName(HiveStringUtils.normalizeIdentifier(new_partCopy.getDbName()));
+      new_partCopy.setTableName(HiveStringUtils.normalizeIdentifier(new_partCopy.getTableName()));
       Partition oldPart = getHBase().getPartition(HiveStringUtils.normalizeIdentifier(db_name),
           HiveStringUtils.normalizeIdentifier(tbl_name), part_vals);
-      getHBase().replacePartition(oldPart, new_part);
+      getHBase().replacePartition(oldPart, new_partCopy, HBaseUtils.getPartitionKeyTypes(
+          getTable(db_name, tbl_name).getPartitionKeys()));
       // Drop any cached stats that reference this partitions
       getHBase().getStatsCache().invalidate(HiveStringUtils.normalizeIdentifier(db_name),
           HiveStringUtils.normalizeIdentifier(tbl_name),
@@ -567,11 +594,19 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
+      List<Partition> new_partsCopy = new ArrayList<Partition>();
+      for (int i=0;i<new_parts.size();i++) {
+        Partition newPartCopy = new_parts.get(i).deepCopy();
+        newPartCopy.setDbName(HiveStringUtils.normalizeIdentifier(newPartCopy.getDbName()));
+        newPartCopy.setTableName(HiveStringUtils.normalizeIdentifier(newPartCopy.getTableName()));
+        new_partsCopy.add(i, newPartCopy);
+      }
       List<Partition> oldParts = getHBase().getPartitions(HiveStringUtils.normalizeIdentifier(db_name),
           HiveStringUtils.normalizeIdentifier(tbl_name),
           HBaseUtils.getPartitionKeyTypes(getTable(HiveStringUtils.normalizeIdentifier(db_name),
           HiveStringUtils.normalizeIdentifier(tbl_name)).getPartitionKeys()), part_vals_list);
-      getHBase().replacePartitions(oldParts, new_parts);
+      getHBase().replacePartitions(oldParts, new_partsCopy, HBaseUtils.getPartitionKeyTypes(
+          getTable(db_name, tbl_name).getPartitionKeys()));
       for (List<String> part_vals : part_vals_list) {
         getHBase().getStatsCache().invalidate(HiveStringUtils.normalizeIdentifier(db_name),
             HiveStringUtils.normalizeIdentifier(tbl_name),

http://git-wip-us.apache.org/repos/asf/hive/blob/e150af94/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index cc90a76..1885089 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -117,24 +117,6 @@ class HBaseUtils {
     return protoKey.getBytes(ENCODING);
   }
 
-  static List<String> parseKey(byte[] serialized, List<String> partNames, List<String> partTypes) {
-    BinarySortableSerDe serDe = new BinarySortableSerDe();
-    Properties props = new Properties();
-    props.setProperty(serdeConstants.LIST_COLUMNS, "dbName,tableName," + StringUtils.join(partNames, ","));
-    props.setProperty(serdeConstants.LIST_COLUMN_TYPES, "string,string," + StringUtils.join(partTypes, ","));
-    List<String> partVals = null;
-    try {
-      serDe.initialize(new Configuration(), props);
-      List deserializedkeys = ((List)serDe.deserialize(new BytesWritable(serialized)));
-      partVals = new ArrayList<String>();
-      for (Object deserializedkey : deserializedkeys) {
-        partVals.add(deserializedkey.toString());
-      }
-    } catch (SerDeException e) {
-    }
-    return partVals;
-  }
-
   private static HbaseMetastoreProto.Parameters buildParameters(Map<String, String> params) {
     List<HbaseMetastoreProto.ParameterEntry> entries = new ArrayList<>();
     for (Map.Entry<String, String> e : params.entrySet()) {
@@ -910,7 +892,7 @@ class HBaseUtils {
     return k.split(KEY_SEPARATOR_STR);
   }
 
-  private static List<String> deserializePartitionKey(List<FieldSchema> partitions, byte[] key,
+  static List<String> deserializePartitionKey(List<FieldSchema> partitions, byte[] key,
       Configuration conf) {
     StringBuffer names = new StringBuffer();
     names.append("dbName,tableName,");
@@ -932,9 +914,19 @@ class HBaseUtils {
       serDe.initialize(new Configuration(), props);
       List deserializedkeys = ((List)serDe.deserialize(new BytesWritable(key))).subList(2, partitions.size()+2);
       List<String> partitionKeys = new ArrayList<String>();
-      for (Object deserializedKey : deserializedkeys) {
-        partitionKeys.add(deserializedKey!=null?deserializedKey.toString():
-          HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME));
+      for (int i=0;i<deserializedkeys.size();i++) {
+        Object deserializedKey = deserializedkeys.get(i);
+        if (deserializedKey==null) {
+          partitionKeys.add(HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME));
+        } else {
+          TypeInfo inputType =
+              TypeInfoUtils.getTypeInfoFromTypeString(partitions.get(i).getType());
+          ObjectInspector inputOI =
+              TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(inputType);
+          Converter converter = ObjectInspectorConverters.getConverter(inputOI,
+              PrimitiveObjectInspectorFactory.javaStringObjectInspector);
+          partitionKeys.add((String)converter.convert(deserializedKey));
+        }
       }
       return partitionKeys;
     } catch (SerDeException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e150af94/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
index fac7dcc..b6dfcf3 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
@@ -96,18 +96,26 @@ public class TestHBaseStore {
     }
   }
   static final long DEFAULT_TIME = System.currentTimeMillis();
+  static final String PART_KEY = "part";
   static final String BOOLEAN_COL = "boolCol";
   static final String BOOLEAN_TYPE = "boolean";
+  static final String BOOLEAN_VAL = "true";
   static final String LONG_COL = "longCol";
   static final String LONG_TYPE = "long";
+  static final String INT_TYPE = "int";
+  static final String INT_VAL = "1234";
   static final String DOUBLE_COL = "doubleCol";
   static final String DOUBLE_TYPE = "double";
+  static final String DOUBLE_VAL = "3.1415";
   static final String STRING_COL = "stringCol";
   static final String STRING_TYPE = "string";
+  static final String STRING_VAL = "stringval";
   static final String BINARY_COL = "binaryCol";
   static final String BINARY_TYPE = "binary";
+  static final String BINARY_VAL = "1";
   static final String DECIMAL_COL = "decimalCol";
   static final String DECIMAL_TYPE = "decimal(5,3)";
+  static final String DECIMAL_VAL = "12.123";
   static List<ColumnStatisticsObj> booleanColStatsObjs = new ArrayList<ColumnStatisticsObj>(
       NUM_PARTITIONS);
   static List<ColumnStatisticsObj> longColStatsObjs = new ArrayList<ColumnStatisticsObj>(
@@ -991,12 +999,13 @@ public class TestHBaseStore {
 
   @Test
   public void booleanPartitionStatistics() throws Exception {
-    // Add partition stats for: BOOLEAN_COL and partition: {PART_KEYS(0), PART_VALS(0)} to DB
+    createMockTableAndPartition(BOOLEAN_TYPE, BOOLEAN_VAL);
+    // Add partition stats for: BOOLEAN_COL and partition: {PART_KEY, BOOLEAN_VAL} to DB
     // Because of the way our mock implementation works we actually need to not create the table
     // before we set statistics on it.
     ColumnStatistics stats = new ColumnStatistics();
     // Get a default ColumnStatisticsDesc for partition level stats
-    ColumnStatisticsDesc desc = getMockPartColStatsDesc(0, 0);
+    ColumnStatisticsDesc desc = getMockPartColStatsDesc(PART_KEY, BOOLEAN_VAL);
     stats.setStatsDesc(desc);
     // Get one of the pre-created ColumnStatisticsObj
     ColumnStatisticsObj obj = booleanColStatsObjs.get(0);
@@ -1004,7 +1013,7 @@ public class TestHBaseStore {
     // Add to DB
     stats.addToStatsObj(obj);
     List<String> parVals = new ArrayList<String>();
-    parVals.add(PART_VALS.get(0));
+    parVals.add(BOOLEAN_VAL);
     store.updatePartitionColumnStatistics(stats, parVals);
     // Get from DB
     List<String> partNames = new ArrayList<String>();
@@ -1033,12 +1042,13 @@ public class TestHBaseStore {
 
   @Test
   public void longPartitionStatistics() throws Exception {
-    // Add partition stats for: LONG_COL and partition: {PART_KEYS(0), PART_VALS(0)} to DB
+    createMockTableAndPartition(INT_TYPE, INT_VAL);
+    // Add partition stats for: LONG_COL and partition: {PART_KEY, INT_VAL} to DB
     // Because of the way our mock implementation works we actually need to not create the table
     // before we set statistics on it.
     ColumnStatistics stats = new ColumnStatistics();
     // Get a default ColumnStatisticsDesc for partition level stats
-    ColumnStatisticsDesc desc = getMockPartColStatsDesc(0, 0);
+    ColumnStatisticsDesc desc = getMockPartColStatsDesc(PART_KEY, INT_VAL);
     stats.setStatsDesc(desc);
     // Get one of the pre-created ColumnStatisticsObj
     ColumnStatisticsObj obj = longColStatsObjs.get(0);
@@ -1046,7 +1056,7 @@ public class TestHBaseStore {
     // Add to DB
     stats.addToStatsObj(obj);
     List<String> parVals = new ArrayList<String>();
-    parVals.add(PART_VALS.get(0));
+    parVals.add(INT_VAL);
     store.updatePartitionColumnStatistics(stats, parVals);
     // Get from DB
     List<String> partNames = new ArrayList<String>();
@@ -1076,12 +1086,13 @@ public class TestHBaseStore {
 
   @Test
   public void doublePartitionStatistics() throws Exception {
-    // Add partition stats for: DOUBLE_COL and partition: {PART_KEYS(0), PART_VALS(0)} to DB
+    createMockTableAndPartition(DOUBLE_TYPE, DOUBLE_VAL);
+    // Add partition stats for: DOUBLE_COL and partition: {PART_KEY, DOUBLE_VAL} to DB
     // Because of the way our mock implementation works we actually need to not create the table
     // before we set statistics on it.
     ColumnStatistics stats = new ColumnStatistics();
     // Get a default ColumnStatisticsDesc for partition level stats
-    ColumnStatisticsDesc desc = getMockPartColStatsDesc(0, 0);
+    ColumnStatisticsDesc desc = getMockPartColStatsDesc(PART_KEY, DOUBLE_VAL);
     stats.setStatsDesc(desc);
     // Get one of the pre-created ColumnStatisticsObj
     ColumnStatisticsObj obj = doubleColStatsObjs.get(0);
@@ -1089,7 +1100,7 @@ public class TestHBaseStore {
     // Add to DB
     stats.addToStatsObj(obj);
     List<String> parVals = new ArrayList<String>();
-    parVals.add(PART_VALS.get(0));
+    parVals.add(DOUBLE_VAL);
     store.updatePartitionColumnStatistics(stats, parVals);
     // Get from DB
     List<String> partNames = new ArrayList<String>();
@@ -1119,12 +1130,13 @@ public class TestHBaseStore {
 
   @Test
   public void stringPartitionStatistics() throws Exception {
-    // Add partition stats for: STRING_COL and partition: {PART_KEYS(0), PART_VALS(0)} to DB
+    createMockTableAndPartition(STRING_TYPE, STRING_VAL);
+    // Add partition stats for: STRING_COL and partition: {PART_KEY, STRING_VAL} to DB
     // Because of the way our mock implementation works we actually need to not create the table
     // before we set statistics on it.
     ColumnStatistics stats = new ColumnStatistics();
     // Get a default ColumnStatisticsDesc for partition level stats
-    ColumnStatisticsDesc desc = getMockPartColStatsDesc(0, 0);
+    ColumnStatisticsDesc desc = getMockPartColStatsDesc(PART_KEY, STRING_VAL);
     stats.setStatsDesc(desc);
     // Get one of the pre-created ColumnStatisticsObj
     ColumnStatisticsObj obj = stringColStatsObjs.get(0);
@@ -1132,7 +1144,7 @@ public class TestHBaseStore {
     // Add to DB
     stats.addToStatsObj(obj);
     List<String> parVals = new ArrayList<String>();
-    parVals.add(PART_VALS.get(0));
+    parVals.add(STRING_VAL);
     store.updatePartitionColumnStatistics(stats, parVals);
     // Get from DB
     List<String> partNames = new ArrayList<String>();
@@ -1162,12 +1174,13 @@ public class TestHBaseStore {
 
   @Test
   public void binaryPartitionStatistics() throws Exception {
-    // Add partition stats for: BINARY_COL and partition: {PART_KEYS(0), PART_VALS(0)} to DB
+    createMockTableAndPartition(BINARY_TYPE, BINARY_VAL);
+    // Add partition stats for: BINARY_COL and partition: {PART_KEY, BINARY_VAL} to DB
     // Because of the way our mock implementation works we actually need to not create the table
     // before we set statistics on it.
     ColumnStatistics stats = new ColumnStatistics();
     // Get a default ColumnStatisticsDesc for partition level stats
-    ColumnStatisticsDesc desc = getMockPartColStatsDesc(0, 0);
+    ColumnStatisticsDesc desc = getMockPartColStatsDesc(PART_KEY, BINARY_VAL);
     stats.setStatsDesc(desc);
     // Get one of the pre-created ColumnStatisticsObj
     ColumnStatisticsObj obj = binaryColStatsObjs.get(0);
@@ -1175,7 +1188,7 @@ public class TestHBaseStore {
     // Add to DB
     stats.addToStatsObj(obj);
     List<String> parVals = new ArrayList<String>();
-    parVals.add(PART_VALS.get(0));
+    parVals.add(BINARY_VAL);
     store.updatePartitionColumnStatistics(stats, parVals);
     // Get from DB
     List<String> partNames = new ArrayList<String>();
@@ -1204,12 +1217,13 @@ public class TestHBaseStore {
 
   @Test
   public void decimalPartitionStatistics() throws Exception {
-    // Add partition stats for: DECIMAL_COL and partition: {PART_KEYS(0), PART_VALS(0)} to DB
+    createMockTableAndPartition(DECIMAL_TYPE, DECIMAL_VAL);
+    // Add partition stats for: DECIMAL_COL and partition: {PART_KEY, DECIMAL_VAL} to DB
     // Because of the way our mock implementation works we actually need to not create the table
     // before we set statistics on it.
     ColumnStatistics stats = new ColumnStatistics();
     // Get a default ColumnStatisticsDesc for partition level stats
-    ColumnStatisticsDesc desc = getMockPartColStatsDesc(0, 0);
+    ColumnStatisticsDesc desc = getMockPartColStatsDesc(PART_KEY, DECIMAL_VAL);
     stats.setStatsDesc(desc);
     // Get one of the pre-created ColumnStatisticsObj
     ColumnStatisticsObj obj = decimalColStatsObjs.get(0);
@@ -1217,7 +1231,7 @@ public class TestHBaseStore {
     // Add to DB
     stats.addToStatsObj(obj);
     List<String> parVals = new ArrayList<String>();
-    parVals.add(PART_VALS.get(0));
+    parVals.add(DECIMAL_VAL);
     store.updatePartitionColumnStatistics(stats, parVals);
     // Get from DB
     List<String> partNames = new ArrayList<String>();
@@ -1245,6 +1259,25 @@ public class TestHBaseStore {
     Assert.assertEquals(decimalData.getNumDVs(), decimalDataFromDB.getNumDVs());
   }
 
+  private Table createMockTableAndPartition(String partType, String partVal) throws Exception {
+    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    cols.add(new FieldSchema("col1", partType, ""));
+    List<String> vals = new ArrayList<String>();
+    vals.add(partVal);
+    SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+    Map<String, String> params = new HashMap<String, String>();
+    params.put("key", "value");
+    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+    int currentTime = (int)(System.currentTimeMillis() / 1000);
+    Table table = new Table(TBL, DB, "me", currentTime, currentTime, 0, sd, cols,
+        emptyParameters, null, null, null);
+    store.createTable(table);
+    Partition part = new Partition(vals, DB, TBL, currentTime, currentTime, sd,
+        emptyParameters);
+    store.addPartition(part);
+    return table;
+  }
   /**
    * Returns a dummy table level ColumnStatisticsDesc with default values
    */
@@ -1260,13 +1293,13 @@ public class TestHBaseStore {
   /**
    * Returns a dummy partition level ColumnStatisticsDesc
    */
-  private ColumnStatisticsDesc getMockPartColStatsDesc(int partKeyIndex, int partValIndex) {
+  private ColumnStatisticsDesc getMockPartColStatsDesc(String partKey, String partVal) {
     ColumnStatisticsDesc desc = new ColumnStatisticsDesc();
     desc.setLastAnalyzed(DEFAULT_TIME);
     desc.setDbName(DB);
     desc.setTableName(TBL);
     // part1=val1
-    desc.setPartName(PART_KEYS.get(partKeyIndex) + PART_KV_SEPARATOR + PART_VALS.get(partValIndex));
+    desc.setPartName(partKey + PART_KV_SEPARATOR + partVal);
     desc.setIsTblLevel(false);
     return desc;
   }


[29/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 72b2cc3..9d72cd0 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ThriftHiveMetastore {
 
   /**
@@ -292,6 +292,14 @@ public class ThriftHiveMetastore {
 
     public void flushCache() throws org.apache.thrift.TException;
 
+    public GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req) throws org.apache.thrift.TException;
+
+    public GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req) throws org.apache.thrift.TException;
+
+    public PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req) throws org.apache.thrift.TException;
+
+    public ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req) throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
@@ -546,6 +554,14 @@ public class ThriftHiveMetastore {
 
     public void flushCache(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_file_metadata_by_expr(GetFileMetadataByExprRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_file_metadata(GetFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void put_file_metadata(PutFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void clear_file_metadata(ClearFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface {
@@ -4242,6 +4258,98 @@ public class ThriftHiveMetastore {
       return;
     }
 
+    public GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req) throws org.apache.thrift.TException
+    {
+      send_get_file_metadata_by_expr(req);
+      return recv_get_file_metadata_by_expr();
+    }
+
+    public void send_get_file_metadata_by_expr(GetFileMetadataByExprRequest req) throws org.apache.thrift.TException
+    {
+      get_file_metadata_by_expr_args args = new get_file_metadata_by_expr_args();
+      args.setReq(req);
+      sendBase("get_file_metadata_by_expr", args);
+    }
+
+    public GetFileMetadataByExprResult recv_get_file_metadata_by_expr() throws org.apache.thrift.TException
+    {
+      get_file_metadata_by_expr_result result = new get_file_metadata_by_expr_result();
+      receiveBase(result, "get_file_metadata_by_expr");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_file_metadata_by_expr failed: unknown result");
+    }
+
+    public GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req) throws org.apache.thrift.TException
+    {
+      send_get_file_metadata(req);
+      return recv_get_file_metadata();
+    }
+
+    public void send_get_file_metadata(GetFileMetadataRequest req) throws org.apache.thrift.TException
+    {
+      get_file_metadata_args args = new get_file_metadata_args();
+      args.setReq(req);
+      sendBase("get_file_metadata", args);
+    }
+
+    public GetFileMetadataResult recv_get_file_metadata() throws org.apache.thrift.TException
+    {
+      get_file_metadata_result result = new get_file_metadata_result();
+      receiveBase(result, "get_file_metadata");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_file_metadata failed: unknown result");
+    }
+
+    public PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req) throws org.apache.thrift.TException
+    {
+      send_put_file_metadata(req);
+      return recv_put_file_metadata();
+    }
+
+    public void send_put_file_metadata(PutFileMetadataRequest req) throws org.apache.thrift.TException
+    {
+      put_file_metadata_args args = new put_file_metadata_args();
+      args.setReq(req);
+      sendBase("put_file_metadata", args);
+    }
+
+    public PutFileMetadataResult recv_put_file_metadata() throws org.apache.thrift.TException
+    {
+      put_file_metadata_result result = new put_file_metadata_result();
+      receiveBase(result, "put_file_metadata");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "put_file_metadata failed: unknown result");
+    }
+
+    public ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req) throws org.apache.thrift.TException
+    {
+      send_clear_file_metadata(req);
+      return recv_clear_file_metadata();
+    }
+
+    public void send_clear_file_metadata(ClearFileMetadataRequest req) throws org.apache.thrift.TException
+    {
+      clear_file_metadata_args args = new clear_file_metadata_args();
+      args.setReq(req);
+      sendBase("clear_file_metadata", args);
+    }
+
+    public ClearFileMetadataResult recv_clear_file_metadata() throws org.apache.thrift.TException
+    {
+      clear_file_metadata_result result = new clear_file_metadata_result();
+      receiveBase(result, "clear_file_metadata");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "clear_file_metadata failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends com.facebook.fb303.FacebookService.AsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -8704,6 +8812,134 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_file_metadata_by_expr(GetFileMetadataByExprRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_file_metadata_by_expr_call method_call = new get_file_metadata_by_expr_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_file_metadata_by_expr_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private GetFileMetadataByExprRequest req;
+      public get_file_metadata_by_expr_call(GetFileMetadataByExprRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.req = req;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_file_metadata_by_expr", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_file_metadata_by_expr_args args = new get_file_metadata_by_expr_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public GetFileMetadataByExprResult getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_file_metadata_by_expr();
+      }
+    }
+
+    public void get_file_metadata(GetFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_file_metadata_call method_call = new get_file_metadata_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_file_metadata_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private GetFileMetadataRequest req;
+      public get_file_metadata_call(GetFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.req = req;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_file_metadata", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_file_metadata_args args = new get_file_metadata_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public GetFileMetadataResult getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_file_metadata();
+      }
+    }
+
+    public void put_file_metadata(PutFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      put_file_metadata_call method_call = new put_file_metadata_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class put_file_metadata_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private PutFileMetadataRequest req;
+      public put_file_metadata_call(PutFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.req = req;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("put_file_metadata", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        put_file_metadata_args args = new put_file_metadata_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public PutFileMetadataResult getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_put_file_metadata();
+      }
+    }
+
+    public void clear_file_metadata(ClearFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      clear_file_metadata_call method_call = new clear_file_metadata_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class clear_file_metadata_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private ClearFileMetadataRequest req;
+      public clear_file_metadata_call(ClearFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.req = req;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("clear_file_metadata", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        clear_file_metadata_args args = new clear_file_metadata_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public ClearFileMetadataResult getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_clear_file_metadata();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends com.facebook.fb303.FacebookService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -8842,6 +9078,10 @@ public class ThriftHiveMetastore {
       processMap.put("get_current_notificationEventId", new get_current_notificationEventId());
       processMap.put("fire_listener_event", new fire_listener_event());
       processMap.put("flushCache", new flushCache());
+      processMap.put("get_file_metadata_by_expr", new get_file_metadata_by_expr());
+      processMap.put("get_file_metadata", new get_file_metadata());
+      processMap.put("put_file_metadata", new put_file_metadata());
+      processMap.put("clear_file_metadata", new clear_file_metadata());
       return processMap;
     }
 
@@ -12086,6 +12326,86 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_file_metadata_by_expr<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_file_metadata_by_expr_args> {
+      public get_file_metadata_by_expr() {
+        super("get_file_metadata_by_expr");
+      }
+
+      public get_file_metadata_by_expr_args getEmptyArgsInstance() {
+        return new get_file_metadata_by_expr_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_file_metadata_by_expr_result getResult(I iface, get_file_metadata_by_expr_args args) throws org.apache.thrift.TException {
+        get_file_metadata_by_expr_result result = new get_file_metadata_by_expr_result();
+        result.success = iface.get_file_metadata_by_expr(args.req);
+        return result;
+      }
+    }
+
+    public static class get_file_metadata<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_file_metadata_args> {
+      public get_file_metadata() {
+        super("get_file_metadata");
+      }
+
+      public get_file_metadata_args getEmptyArgsInstance() {
+        return new get_file_metadata_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_file_metadata_result getResult(I iface, get_file_metadata_args args) throws org.apache.thrift.TException {
+        get_file_metadata_result result = new get_file_metadata_result();
+        result.success = iface.get_file_metadata(args.req);
+        return result;
+      }
+    }
+
+    public static class put_file_metadata<I extends Iface> extends org.apache.thrift.ProcessFunction<I, put_file_metadata_args> {
+      public put_file_metadata() {
+        super("put_file_metadata");
+      }
+
+      public put_file_metadata_args getEmptyArgsInstance() {
+        return new put_file_metadata_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public put_file_metadata_result getResult(I iface, put_file_metadata_args args) throws org.apache.thrift.TException {
+        put_file_metadata_result result = new put_file_metadata_result();
+        result.success = iface.put_file_metadata(args.req);
+        return result;
+      }
+    }
+
+    public static class clear_file_metadata<I extends Iface> extends org.apache.thrift.ProcessFunction<I, clear_file_metadata_args> {
+      public clear_file_metadata() {
+        super("clear_file_metadata");
+      }
+
+      public clear_file_metadata_args getEmptyArgsInstance() {
+        return new clear_file_metadata_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public clear_file_metadata_result getResult(I iface, clear_file_metadata_args args) throws org.apache.thrift.TException {
+        clear_file_metadata_result result = new clear_file_metadata_result();
+        result.success = iface.clear_file_metadata(args.req);
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends com.facebook.fb303.FacebookService.AsyncProcessor<I> {
@@ -12224,6 +12544,10 @@ public class ThriftHiveMetastore {
       processMap.put("get_current_notificationEventId", new get_current_notificationEventId());
       processMap.put("fire_listener_event", new fire_listener_event());
       processMap.put("flushCache", new flushCache());
+      processMap.put("get_file_metadata_by_expr", new get_file_metadata_by_expr());
+      processMap.put("get_file_metadata", new get_file_metadata());
+      processMap.put("put_file_metadata", new put_file_metadata());
+      processMap.put("clear_file_metadata", new clear_file_metadata());
       return processMap;
     }
 
@@ -19936,385 +20260,228 @@ public class ThriftHiveMetastore {
       }
     }
 
-  }
-
-  public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_args");
-
-    private static final org.apache.thrift.protocol.TField KEY_FIELD_DESC = new org.apache.thrift.protocol.TField("key", org.apache.thrift.protocol.TType.STRING, (short)1);
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new getMetaConf_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new getMetaConf_argsTupleSchemeFactory());
-    }
-
-    private String key; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      KEY((short)1, "key");
-
-      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-      static {
-        for (_Fields field : EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
+    public static class get_file_metadata_by_expr<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_file_metadata_by_expr_args, GetFileMetadataByExprResult> {
+      public get_file_metadata_by_expr() {
+        super("get_file_metadata_by_expr");
       }
 
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 1: // KEY
-            return KEY;
-          default:
-            return null;
-        }
+      public get_file_metadata_by_expr_args getEmptyArgsInstance() {
+        return new get_file_metadata_by_expr_args();
       }
 
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      public static _Fields findByName(String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final String _fieldName;
-
-      _Fields(short thriftId, String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
+      public AsyncMethodCallback<GetFileMetadataByExprResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<GetFileMetadataByExprResult>() { 
+          public void onComplete(GetFileMetadataByExprResult o) {
+            get_file_metadata_by_expr_result result = new get_file_metadata_by_expr_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_file_metadata_by_expr_result result = new get_file_metadata_by_expr_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
       }
 
-      public short getThriftFieldId() {
-        return _thriftId;
+      protected boolean isOneway() {
+        return false;
       }
 
-      public String getFieldName() {
-        return _fieldName;
+      public void start(I iface, get_file_metadata_by_expr_args args, org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataByExprResult> resultHandler) throws TException {
+        iface.get_file_metadata_by_expr(args.req,resultHandler);
       }
     }
 
-    // isset id assignments
-    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getMetaConf_args.class, metaDataMap);
-    }
-
-    public getMetaConf_args() {
-    }
-
-    public getMetaConf_args(
-      String key)
-    {
-      this();
-      this.key = key;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getMetaConf_args(getMetaConf_args other) {
-      if (other.isSetKey()) {
-        this.key = other.key;
+    public static class get_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_file_metadata_args, GetFileMetadataResult> {
+      public get_file_metadata() {
+        super("get_file_metadata");
       }
-    }
-
-    public getMetaConf_args deepCopy() {
-      return new getMetaConf_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.key = null;
-    }
-
-    public String getKey() {
-      return this.key;
-    }
-
-    public void setKey(String key) {
-      this.key = key;
-    }
-
-    public void unsetKey() {
-      this.key = null;
-    }
-
-    /** Returns true if field key is set (has been assigned a value) and false otherwise */
-    public boolean isSetKey() {
-      return this.key != null;
-    }
-
-    public void setKeyIsSet(boolean value) {
-      if (!value) {
-        this.key = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case KEY:
-        if (value == null) {
-          unsetKey();
-        } else {
-          setKey((String)value);
-        }
-        break;
 
+      public get_file_metadata_args getEmptyArgsInstance() {
+        return new get_file_metadata_args();
       }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case KEY:
-        return getKey();
 
+      public AsyncMethodCallback<GetFileMetadataResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<GetFileMetadataResult>() { 
+          public void onComplete(GetFileMetadataResult o) {
+            get_file_metadata_result result = new get_file_metadata_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_file_metadata_result result = new get_file_metadata_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
       }
-      throw new IllegalStateException();
-    }
 
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new IllegalArgumentException();
+      protected boolean isOneway() {
+        return false;
       }
 
-      switch (field) {
-      case KEY:
-        return isSetKey();
+      public void start(I iface, get_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataResult> resultHandler) throws TException {
+        iface.get_file_metadata(args.req,resultHandler);
       }
-      throw new IllegalStateException();
     }
 
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof getMetaConf_args)
-        return this.equals((getMetaConf_args)that);
-      return false;
-    }
-
-    public boolean equals(getMetaConf_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_key = true && this.isSetKey();
-      boolean that_present_key = true && that.isSetKey();
-      if (this_present_key || that_present_key) {
-        if (!(this_present_key && that_present_key))
-          return false;
-        if (!this.key.equals(that.key))
-          return false;
+    public static class put_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, put_file_metadata_args, PutFileMetadataResult> {
+      public put_file_metadata() {
+        super("put_file_metadata");
       }
 
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      boolean present_key = true && (isSetKey());
-      list.add(present_key);
-      if (present_key)
-        list.add(key);
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(getMetaConf_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
+      public put_file_metadata_args getEmptyArgsInstance() {
+        return new put_file_metadata_args();
       }
 
-      int lastComparison = 0;
-
-      lastComparison = Boolean.valueOf(isSetKey()).compareTo(other.isSetKey());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetKey()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.key, other.key);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
+      public AsyncMethodCallback<PutFileMetadataResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<PutFileMetadataResult>() { 
+          public void onComplete(PutFileMetadataResult o) {
+            put_file_metadata_result result = new put_file_metadata_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            put_file_metadata_result result = new put_file_metadata_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
       }
-      return 0;
-    }
-
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-    }
 
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder("getMetaConf_args(");
-      boolean first = true;
-
-      sb.append("key:");
-      if (this.key == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.key);
+      protected boolean isOneway() {
+        return false;
       }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
 
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
+      public void start(I iface, put_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<PutFileMetadataResult> resultHandler) throws TException {
+        iface.put_file_metadata(args.req,resultHandler);
       }
     }
 
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
+    public static class clear_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, clear_file_metadata_args, ClearFileMetadataResult> {
+      public clear_file_metadata() {
+        super("clear_file_metadata");
       }
-    }
 
-    private static class getMetaConf_argsStandardSchemeFactory implements SchemeFactory {
-      public getMetaConf_argsStandardScheme getScheme() {
-        return new getMetaConf_argsStandardScheme();
+      public clear_file_metadata_args getEmptyArgsInstance() {
+        return new clear_file_metadata_args();
       }
-    }
-
-    private static class getMetaConf_argsStandardScheme extends StandardScheme<getMetaConf_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getMetaConf_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
+      public AsyncMethodCallback<ClearFileMetadataResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<ClearFileMetadataResult>() { 
+          public void onComplete(ClearFileMetadataResult o) {
+            clear_file_metadata_result result = new clear_file_metadata_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
           }
-          switch (schemeField.id) {
-            case 1: // KEY
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.key = iprot.readString();
-                struct.setKeyIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            clear_file_metadata_result result = new clear_file_metadata_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
           }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-        struct.validate();
-      }
-
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getMetaConf_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.key != null) {
-          oprot.writeFieldBegin(KEY_FIELD_DESC);
-          oprot.writeString(struct.key);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getMetaConf_argsTupleSchemeFactory implements SchemeFactory {
-      public getMetaConf_argsTupleScheme getScheme() {
-        return new getMetaConf_argsTupleScheme();
+        };
       }
-    }
-
-    private static class getMetaConf_argsTupleScheme extends TupleScheme<getMetaConf_args> {
 
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getMetaConf_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetKey()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetKey()) {
-          oprot.writeString(struct.key);
-        }
+      protected boolean isOneway() {
+        return false;
       }
 
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getMetaConf_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.key = iprot.readString();
-          struct.setKeyIsSet(true);
-        }
+      public void start(I iface, clear_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<ClearFileMetadataResult> resultHandler) throws TException {
+        iface.clear_file_metadata(args.req,resultHandler);
       }
     }
 
   }
 
-  public static class getMetaConf_result implements org.apache.thrift.TBase<getMetaConf_result, getMetaConf_result._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_result");
+  public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_args");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
-    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField KEY_FIELD_DESC = new org.apache.thrift.protocol.TField("key", org.apache.thrift.protocol.TType.STRING, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new getMetaConf_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new getMetaConf_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new getMetaConf_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getMetaConf_argsTupleSchemeFactory());
     }
 
-    private String success; // required
-    private MetaException o1; // required
+    private String key; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success"),
-      O1((short)1, "o1");
+      KEY((short)1, "key");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -20329,10 +20496,371 @@ public class ThriftHiveMetastore {
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          case 1: // O1
-            return O1;
+          case 1: // KEY
+            return KEY;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getMetaConf_args.class, metaDataMap);
+    }
+
+    public getMetaConf_args() {
+    }
+
+    public getMetaConf_args(
+      String key)
+    {
+      this();
+      this.key = key;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getMetaConf_args(getMetaConf_args other) {
+      if (other.isSetKey()) {
+        this.key = other.key;
+      }
+    }
+
+    public getMetaConf_args deepCopy() {
+      return new getMetaConf_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.key = null;
+    }
+
+    public String getKey() {
+      return this.key;
+    }
+
+    public void setKey(String key) {
+      this.key = key;
+    }
+
+    public void unsetKey() {
+      this.key = null;
+    }
+
+    /** Returns true if field key is set (has been assigned a value) and false otherwise */
+    public boolean isSetKey() {
+      return this.key != null;
+    }
+
+    public void setKeyIsSet(boolean value) {
+      if (!value) {
+        this.key = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case KEY:
+        if (value == null) {
+          unsetKey();
+        } else {
+          setKey((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case KEY:
+        return getKey();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case KEY:
+        return isSetKey();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getMetaConf_args)
+        return this.equals((getMetaConf_args)that);
+      return false;
+    }
+
+    public boolean equals(getMetaConf_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_key = true && this.isSetKey();
+      boolean that_present_key = true && that.isSetKey();
+      if (this_present_key || that_present_key) {
+        if (!(this_present_key && that_present_key))
+          return false;
+        if (!this.key.equals(that.key))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_key = true && (isSetKey());
+      list.add(present_key);
+      if (present_key)
+        list.add(key);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getMetaConf_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetKey()).compareTo(other.isSetKey());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetKey()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.key, other.key);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getMetaConf_args(");
+      boolean first = true;
+
+      sb.append("key:");
+      if (this.key == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.key);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getMetaConf_argsStandardSchemeFactory implements SchemeFactory {
+      public getMetaConf_argsStandardScheme getScheme() {
+        return new getMetaConf_argsStandardScheme();
+      }
+    }
+
+    private static class getMetaConf_argsStandardScheme extends StandardScheme<getMetaConf_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getMetaConf_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // KEY
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.key = iprot.readString();
+                struct.setKeyIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getMetaConf_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.key != null) {
+          oprot.writeFieldBegin(KEY_FIELD_DESC);
+          oprot.writeString(struct.key);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getMetaConf_argsTupleSchemeFactory implements SchemeFactory {
+      public getMetaConf_argsTupleScheme getScheme() {
+        return new getMetaConf_argsTupleScheme();
+      }
+    }
+
+    private static class getMetaConf_argsTupleScheme extends TupleScheme<getMetaConf_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getMetaConf_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetKey()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetKey()) {
+          oprot.writeString(struct.key);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getMetaConf_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.key = iprot.readString();
+          struct.setKeyIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class getMetaConf_result implements org.apache.thrift.TBase<getMetaConf_result, getMetaConf_result._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getMetaConf_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getMetaConf_resultTupleSchemeFactory());
+    }
+
+    private String success; // required
+    private MetaException o1; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      O1((short)1, "o1");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // O1
+            return O1;
           default:
             return null;
         }
@@ -25331,13 +25859,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list532.size);
-                  String _elem533;
-                  for (int _i534 = 0; _i534 < _list532.size; ++_i534)
+                  org.apache.thrift.protocol.TList _list600 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list600.size);
+                  String _elem601;
+                  for (int _i602 = 0; _i602 < _list600.size; ++_i602)
                   {
-                    _elem533 = iprot.readString();
-                    struct.success.add(_elem533);
+                    _elem601 = iprot.readString();
+                    struct.success.add(_elem601);
                   }
                   iprot.readListEnd();
                 }
@@ -25372,9 +25900,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter535 : struct.success)
+            for (String _iter603 : struct.success)
             {
-              oprot.writeString(_iter535);
+              oprot.writeString(_iter603);
             }
             oprot.writeListEnd();
           }
@@ -25413,9 +25941,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter536 : struct.success)
+            for (String _iter604 : struct.success)
             {
-              oprot.writeString(_iter536);
+              oprot.writeString(_iter604);
             }
           }
         }
@@ -25430,13 +25958,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list537.size);
-            String _elem538;
-            for (int _i539 = 0; _i539 < _list537.size; ++_i539)
+            org.apache.thrift.protocol.TList _list605 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list605.size);
+            String _elem606;
+            for (int _i607 = 0; _i607 < _list605.size; ++_i607)
             {
-              _elem538 = iprot.readString();
-              struct.success.add(_elem538);
+              _elem606 = iprot.readString();
+              struct.success.add(_elem606);
             }
           }
           struct.setSuccessIsSet(true);
@@ -26090,13 +26618,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list540.size);
-                  String _elem541;
-                  for (int _i542 = 0; _i542 < _list540.size; ++_i542)
+                  org.apache.thrift.protocol.TList _list608 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list608.size);
+                  String _elem609;
+                  for (int _i610 = 0; _i610 < _list608.size; ++_i610)
                   {
-                    _elem541 = iprot.readString();
-                    struct.success.add(_elem541);
+                    _elem609 = iprot.readString();
+                    struct.success.add(_elem609);
                   }
                   iprot.readListEnd();
                 }
@@ -26131,9 +26659,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter543 : struct.success)
+            for (String _iter611 : struct.success)
             {
-              oprot.writeString(_iter543);
+              oprot.writeString(_iter611);
             }
             oprot.writeListEnd();
           }
@@ -26172,9 +26700,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter544 : struct.success)
+            for (String _iter612 : struct.success)
             {
-              oprot.writeString(_iter544);
+              oprot.writeString(_iter612);
             }
           }
         }
@@ -26189,13 +26717,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list545 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list545.size);
-            String _elem546;
-            for (int _i547 = 0; _i547 < _list545.size; ++_i547)
+            org.apache.thrift.protocol.TList _list613 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list613.size);
+            String _elem614;
+            for (int _i615 = 0; _i615 < _list613.size; ++_i615)
             {
-              _elem546 = iprot.readString();
-              struct.success.add(_elem546);
+              _elem614 = iprot.readString();
+              struct.success.add(_elem614);
             }
           }
           struct.setSuccessIsSet(true);
@@ -30802,16 +31330,16 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map548 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map548.size);
-                  String _key549;
-                  Type _val550;
-                  for (int _i551 = 0; _i551 < _map548.size; ++_i551)
+                  org.apache.thrift.protocol.TMap _map616 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map616.size);
+                  String _key617;
+                  Type _val618;
+                  for (int _i619 = 0; _i619 < _map616.size; ++_i619)
                   {
-                    _key549 = iprot.readString();
-                    _val550 = new Type();
-                    _val550.read(iprot);
-                    struct.success.put(_key549, _val550);
+                    _key617 = iprot.readString();
+                    _val618 = new Type();
+                    _val618.read(iprot);
+                    struct.success.put(_key617, _val618);
                   }
                   iprot.readMapEnd();
                 }
@@ -30846,10 +31374,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter552 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter620 : struct.success.entrySet())
             {
-              oprot.writeString(_iter552.getKey());
-              _iter552.getValue().write(oprot);
+              oprot.writeString(_iter620.getKey());
+              _iter620.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -30888,10 +31416,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter553 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter621 : struct.success.entrySet())
             {
-              oprot.writeString(_iter553.getKey());
-              _iter553.getValue().write(oprot);
+              oprot.writeString(_iter621.getKey());
+              _iter621.getValue().write(oprot);
             }
           }
         }
@@ -30906,16 +31434,16 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map554 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new HashMap<String,Type>(2*_map554.size);
-            String _key555;
-            Type _val556;
-            for (int _i557 = 0; _i557 < _map554.size; ++_i557)
+            org.apache.thrift.protocol.TMap _map622 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Type>(2*_map622.size);
+            String _key623;
+            Type _val624;
+            for (int _i625 = 0; _i625 < _map622.size; ++_i625)
             {
-              _key555 = iprot.readString();
-              _val556 = new Type();
-              _val556.read(iprot);
-              struct.success.put(_key555, _val556);
+              _key623 = iprot.readString();
+              _val624 = new Type();
+              _val624.read(iprot);
+              struct.success.put(_key623, _val624);
             }
           }
           struct.setSuccessIsSet(true);
@@ -31950,14 +32478,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list558 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list558.size);
-                  FieldSchema _elem559;
-                  for (int _i560 = 0; _i560 < _list558.size; ++_i560)
+                  org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list626.size);
+                  FieldSchema _elem627;
+                  for (int _i628 = 0; _i628 < _list626.size; ++_i628)
                   {
-                    _elem559 = new FieldSchema();
-                    _elem559.read(iprot);
-                    struct.success.add(_elem559);
+                    _elem627 = new FieldSchema();
+                    _elem627.read(iprot);
+                    struct.success.add(_elem627);
                   }
                   iprot.readListEnd();
                 }
@@ -32010,9 +32538,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter561 : struct.success)
+            for (FieldSchema _iter629 : struct.success)
             {
-              _iter561.write(oprot);
+              _iter629.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -32067,9 +32595,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter562 : struct.success)
+            for (FieldSchema _iter630 : struct.success)
             {
-              _iter562.write(oprot);
+              _iter630.write(oprot);
             }
           }
         }
@@ -32090,14 +32618,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list563.size);
-            FieldSchema _elem564;
-            for (int _i565 = 0; _i565 < _list563.size; ++_i565)
+            org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list631.size);
+            FieldSchema _elem632;
+            for (int _i633 = 0; _i633 < _list631.size; ++_i633)
             {
-              _elem564 = new FieldSchema();
-              _elem564.read(iprot);
-              struct.success.add(_elem564);
+              _elem632 = new FieldSchema();
+              _elem632.read(iprot);
+              struct.success.add(_elem632);
             }
           }
           struct.setSuccessIsSet(true);
@@ -33251,14 +33779,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list566 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list566.size);
-                  FieldSchema _elem567;
-                  for (int _i568 = 0; _i568 < _list566.size; ++_i568)
+                  org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list634.size);
+                  FieldSchema _elem635;
+                  for (int _i636 = 0; _i636 < _list634.size; ++_i636)
                   {
-                    _elem567 = new FieldSchema();
-                    _elem567.read(iprot);
-                    struct.success.add(_elem567);
+                    _elem635 = new FieldSchema();
+                    _elem635.read(iprot);
+                    struct.success.add(_elem635);
                   }
                   iprot.readListEnd();
                 }
@@ -33311,9 +33839,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter569 : struct.success)
+            for (FieldSchema _iter637 : struct.success)
             {
-              _iter569.write(oprot);
+              _iter637.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -33368,9 +33896,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter570 : struct.success)
+            for (FieldSchema _iter638 : struct.success)
             {
-              _iter570.write(oprot);
+              _iter638.write(oprot);
             }
           }
         }
@@ -33391,14 +33919,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list571.size);
-            FieldSchema _elem572;
-            for (int _i573 = 0; _i573 < _list571.size; ++_i573)
+            org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list639.size);
+            FieldSchema _elem640;
+            for (int _i641 = 0; _i641 < _list639.size; ++_i641)
             {
-              _elem572 = new FieldSchema();
-              _elem572.read(iprot);
-              struct.success.add(_elem572);
+              _elem640 = new FieldSchema();
+              _elem640.read(iprot);
+              struct.success.add(_elem640);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34443,14 +34971,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list574.size);
-                  FieldSchema _elem575;
-                  for (int _i576 = 0; _i576 < _list574.size; ++_i576)
+                  org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list642.size);
+                  FieldSchema _elem643;
+                  for (int _i644 = 0; _i644 < _list642.size; ++_i644)
                   {
-                    _elem575 = new FieldSchema();
-                    _elem575.read(iprot);
-                    struct.success.add(_elem575);
+                    _elem643 = new FieldSchema();
+                    _elem643.read(iprot);
+                    struct.success.add(_elem643);
                   }
                   iprot.readListEnd();
                 }
@@ -34503,9 +35031,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter577 : struct.success)
+            for (FieldSchema _iter645 : struct.success)
             {
-              _iter577.write(oprot);
+              _iter645.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -34560,9 +35088,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter578 : struct.success)
+            for (FieldSchema _iter646 : struct.success)
             {
-              _iter578.write(oprot);
+              _iter646.write(oprot);
             }
           }
         }
@@ -34583,14 +35111,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list579.size);
-            FieldSchema _elem580;
-            for (int _i581 = 0; _i581 < _list579.size; ++_i581)
+            org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list647.size);
+            FieldSchema _elem648;
+            for (int _i649 = 0; _i649 < _list647.size; ++_i649)
             {
-              _elem580 = new FieldSchema();
-              _elem580.read(iprot);
-              struct.success.add(_elem580);
+              _elem648 = new FieldSchema();
+              _elem648.read(iprot);
+              struct.success.add(_elem648);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35744,14 +36272,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list582 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list582.size);
-                  FieldSchema _elem583;
-                  for (int _i584 = 0; _i584 < _list582.size; ++_i584)
+                  org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list650.size);
+                  FieldSchema _elem651;
+                  for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                   {
-                    _elem583 = new FieldSchema();
-                    _elem583.read(iprot);
-                    struct.success.add(_elem583);
+                    _elem651 = new FieldSchema();
+                    _elem651.read(iprot);
+                    struct.success.add(_elem651);
                   }
                   iprot.readListEnd();
                 }
@@ -35804,9 +36332,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter585 : struct.success)
+            for (FieldSchema _iter653 : struct.success)
             {
-              _iter585.write(oprot);
+              _iter653.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -35861,9 +36389,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter586 : struct.success)
+            for (FieldSchema _iter654 : struct.success)
             {
-              _iter586.write(oprot);
+              _iter654.write(oprot);
             }
           }
         }
@@ -35884,14 +36412,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list587.size);
-            FieldSchema _elem588;
-            for (int _i589 = 0; _i589 < _list587.size; ++_i589)
+            org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list655.size);
+            FieldSchema _elem656;
+            for (int _i657 = 0; _i657 < _list655.size; ++_i657)
             {
-              _elem588 = new FieldSchema();
-              _elem588.read(iprot);
-              struct.success.add(_elem588);
+              _elem656 = new FieldSchema();
+              _elem656.read(iprot);
+              struct.success.add(_elem656);
             }
           }
           struct.setSuccessIsSet(true);
@@ -41131,13 +41659,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list590 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list590.size);
-                  String _elem591;
-                  for (int _i592 = 0; _i592 < _list590.size; ++_i592)
+                  org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list658.size);
+                  String _elem659;
+                  for (int _i660 = 0; _i660 < _list658.size; ++_i660)
                   {
-                    _elem591 = iprot.readString();
-                    struct.success.add(_elem591);
+                    _elem659 = iprot.readString();
+                    struct.success.add(_elem659);
                   }
                   iprot.readListEnd();
                 }
@@ -41172,9 +41700,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter593 : struct.success)
+            for (String _iter661 : struct.success)
             {
-              oprot.writeString(_iter593);
+              oprot.writeString(_iter661);
             }
             oprot.writeListEnd();
           }
@@ -41213,9 +41741,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter594 : struct.success)
+            for (String _iter662 : struct.success)
             {
-              oprot.writeString(_iter594);
+              oprot.writeString(_iter662);
             }
           }
         }
@@ -41230,13 +41758,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list595.size);
-            String _elem596;
-            for (int _i597 = 0; _i597 < _list595.size; ++_i597)
+            org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list663.size);
+            String _elem664;
+            for (int _i665 = 0; _i665 < _list663.size; ++_i665)
             {
-              _elem596 = iprot.readString();
-              struct.success.add(_elem596);
+              _elem664 = iprot.readString();
+              struct.success.add(_elem664);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42002,13 +42530,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list598 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list598.size);
-                  String _elem599;
-                  for (int _i600 = 0; _i600 < _list598.size; ++_i600)
+                  org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list666.size);
+                  String _elem667;
+                  for (int _i668 = 0; _i668 < _list666.size; ++_i668)
                   {
-                    _elem599 = iprot.readString();
-                    struct.success.add(_elem599);
+                    _elem667 = iprot.readString();
+                    struct.success.add(_elem667);
                   }
                   iprot.readListEnd();
                 }
@@ -42043,9 +42571,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter601 : struct.success)
+            for (String _iter669 : struct.success)
             {
-              oprot.writeString(_iter601);
+              oprot.writeString(_iter669);
             }
             oprot.writeListEnd();
           }
@@ -42084,9 +42612,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter602 : struct.success)
+            for (String _iter670 : struct.success)
             {
-              oprot.writeString(_iter602);
+              oprot.writeString(_iter670);
             }
           }
         }
@@ -42101,13 +42629,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list603.size);
-            String _elem604;
-            for (int _i605 = 0; _i605 < _list603.size; ++_i605)
+            org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list671.size);
+            String _elem672;
+            for (int _i673 = 0; _i673 < _list671.size; ++_i673)
             {
-              _elem604 = iprot.readString();
-              struct.success.add(_elem604);
+              _elem672 = iprot.readString();
+              struct.success.add(_elem672);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43560,13 +44088,13 @@ public class ThriftHiveMetastore {
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list606 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list606.size);
-                  String _elem607;
-                  for (int _i608 = 0; _i608 < _list606.size; ++_i608)
+                  org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list674.size);
+                  String _elem675;
+                  for (int _i676 = 0; _i676 < _list674.size; ++_i676)
                   {
-                    _elem607 = iprot.readString();
-                    struct.tbl_names.add(_elem607);
+                    _elem675 = iprot.readString();
+                    struct.tbl_names.add(_elem675);
                   }
                   iprot.readListEnd();
                 }
@@ -43597,9 +44125,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter609 : struct.tbl_names)
+            for (String _iter677 : struct.tbl_names)
             {
-              oprot.writeString(_iter609);
+              oprot.writeString(_iter677);
             }
             oprot.writeListEnd();
           }
@@ -43636,9 +44164,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter610 : struct.tbl_names)
+            for (String _iter678 : struct.tbl_names)
             {
-              oprot.writeString(_iter610);
+              oprot.writeString(_iter678);
             }
           }
         }
@@ -43654,13 +44182,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list611.size);
-            String _elem612;
-            for (int _i613 = 0; _i613 < _list611.size; ++_i613)
+            org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list679.size);
+            String _elem680;
+            for (int _i681 = 0; _i681 < _list679.size; ++_i681)
             {
-              _elem612 = iprot.readString();
-              struct.tbl_names.add(_elem612);
+              _elem680 = iprot.readString();
+              struct.tbl_names.add(_elem680);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -44228,14 +44756,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list614 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list614.size);
-                  Table _elem615;
-                  for (int _i616 = 0; _i616 < _list614.size; ++_i616)
+                  org.apache.thrift.protocol.TList _list682 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list682.size);
+                  Table _elem683;
+                  for (int _i684 = 0; _i684 < _list682.size; ++_i684)
                   {
-                    _elem615 = new Table();
-                    _elem615.read(iprot);
-                    struct.success.add(_elem615);
+                    _elem683 = new Table();
+                    _elem683.read(iprot);
+                    struct.success.add(_elem683);
                   }
                   iprot.readListEnd();
                 }
@@ -44288,9 +44816,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter617 : struct.success)
+            for (Table _iter685 : struct.success)
             {
-              _iter617.write(oprot);
+              _iter685.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -44345,9 +44873,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter618 : struct.success)
+            for (Table _iter686 : struct.success)
             {
-              _iter618.write(oprot);
+              _iter686.write(oprot);
             }
           }
         }
@@ -44368,14 +44896,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list619.size);
-            Table _elem620;
-            for (int _i621 = 0; _i621 < _list619.size; ++_i621)
+            org.apache.thrift.protocol.TList _list687 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list687.size);
+            Table _elem688;
+            for (int _i689 = 0; _i689 < _list687.size; ++_i689)
             {
-              _elem620 = new Table();
-              _elem620.read(iprot);
-              struct.success.add(_elem620);
+              _elem688 = new Table();
+              _elem688.read(iprot);
+              struct.success.add(_elem688);
             }
           }
           struct.setSuccessIsSet(true);
@@ -45521,13 +46049,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _li

<TRUNCATED>

[19/52] [abbrv] hive git commit: HIVE-11588 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-11588 : merge master into branch (Sergey Shelukhin)


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

Branch: refs/heads/llap
Commit: 9d9dd72a06ee2db379dbbae3561d172223d7c96d
Parents: 2fe6086 3b6825b
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Aug 17 15:20:25 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Aug 17 15:20:25 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   6 +
 data/conf/hive-log4j2.xml                       |   5 +-
 data/conf/hive-site.xml                         |   6 -
 .../deployers/config/hive/hive-site.mysql.xml   |  22 +++
 .../hive/hcatalog/streaming/TestStreaming.java  |  54 +++++-
 .../TestOperationLoggingAPIWithMr.java          |   2 -
 .../TestOperationLoggingAPIWithTez.java         |   2 -
 .../operation/TestOperationLoggingLayout.java   |   2 -
 .../hadoop/hive/metastore/HiveMetaStore.java    |  18 ++
 .../hive/metastore/HouseKeeperService.java      |  39 ++++
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 153 +++++++++++-----
 .../hive/metastore/txn/TestTxnHandler.java      |   7 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  15 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java | 183 ++++++++++++++-----
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |  97 +++++++---
 .../apache/hadoop/hive/ql/io/orc/OrcSplit.java  |  25 ++-
 .../hadoop/hive/ql/lib/DefaultGraphWalker.java  |   2 +-
 .../calcite/reloperators/HiveFilter.java        |   2 +-
 .../calcite/translator/ExprNodeConverter.java   |  26 ---
 .../apache/hadoop/hive/ql/parse/ASTNode.java    |  18 +-
 .../hive/ql/txn/AcidHouseKeeperService.java     | 104 +++++++++++
 .../hive/ql/txn/compactor/CompactorMR.java      |  19 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |  10 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java    |   2 +-
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |  21 +++
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   1 +
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java |  27 +--
 .../hive/ql/io/orc/TestInputOutputFormat.java   |   6 +-
 .../hive/ql/lockmgr/TestDbTxnManager.java       |  35 +++-
 .../hadoop/hive/shims/Hadoop20SShims.java       |  11 ++
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |  66 +++++++
 .../apache/hadoop/hive/shims/HadoopShims.java   |  15 ++
 32 files changed, 782 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9d9dd72a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/9d9dd72a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/9d9dd72a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------


[22/52] [abbrv] hive git commit: HIVE-10289: Support filter on non-first partition key and non-string partition key (Daniel Dai reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-10289: Support filter on non-first partition key and non-string partition key (Daniel Dai reviewed by Alan Gates)


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

Branch: refs/heads/llap
Commit: 5e16d53e98e44567bbfa1b291f8a927a3e3e4b9b
Parents: 9d9dd72
Author: Daniel Dai <da...@hortonworks.com>
Authored: Mon Aug 24 11:20:55 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Mon Aug 24 11:20:55 2015 -0700

----------------------------------------------------------------------
 .../metastore/hbase/HbaseMetastoreProto.java    | 3732 +++++++++++++++++-
 .../metastore/hbase/HBaseFilterPlanUtil.java    |  341 +-
 .../hive/metastore/hbase/HBaseReadWrite.java    |  114 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   11 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |  129 +-
 .../metastore/hbase/PartitionKeyComparator.java |  292 ++
 .../metastore/hbase/hbase_metastore_proto.proto |   25 +
 .../hbase/TestHBaseFilterPlanUtil.java          |  278 +-
 .../BinarySortableSerDeWithEndPrefix.java       |   41 +
 9 files changed, 4702 insertions(+), 261 deletions(-)
----------------------------------------------------------------------



[35/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index fcc4f0b..2872f85 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1235,14 +1235,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size649;
-            ::apache::thrift::protocol::TType _etype652;
-            xfer += iprot->readListBegin(_etype652, _size649);
-            this->success.resize(_size649);
-            uint32_t _i653;
-            for (_i653 = 0; _i653 < _size649; ++_i653)
+            uint32_t _size719;
+            ::apache::thrift::protocol::TType _etype722;
+            xfer += iprot->readListBegin(_etype722, _size719);
+            this->success.resize(_size719);
+            uint32_t _i723;
+            for (_i723 = 0; _i723 < _size719; ++_i723)
             {
-              xfer += iprot->readString(this->success[_i653]);
+              xfer += iprot->readString(this->success[_i723]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1281,10 +1281,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter654;
-      for (_iter654 = this->success.begin(); _iter654 != this->success.end(); ++_iter654)
+      std::vector<std::string> ::const_iterator _iter724;
+      for (_iter724 = this->success.begin(); _iter724 != this->success.end(); ++_iter724)
       {
-        xfer += oprot->writeString((*_iter654));
+        xfer += oprot->writeString((*_iter724));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1328,14 +1328,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size655;
-            ::apache::thrift::protocol::TType _etype658;
-            xfer += iprot->readListBegin(_etype658, _size655);
-            (*(this->success)).resize(_size655);
-            uint32_t _i659;
-            for (_i659 = 0; _i659 < _size655; ++_i659)
+            uint32_t _size725;
+            ::apache::thrift::protocol::TType _etype728;
+            xfer += iprot->readListBegin(_etype728, _size725);
+            (*(this->success)).resize(_size725);
+            uint32_t _i729;
+            for (_i729 = 0; _i729 < _size725; ++_i729)
             {
-              xfer += iprot->readString((*(this->success))[_i659]);
+              xfer += iprot->readString((*(this->success))[_i729]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1452,14 +1452,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size660;
-            ::apache::thrift::protocol::TType _etype663;
-            xfer += iprot->readListBegin(_etype663, _size660);
-            this->success.resize(_size660);
-            uint32_t _i664;
-            for (_i664 = 0; _i664 < _size660; ++_i664)
+            uint32_t _size730;
+            ::apache::thrift::protocol::TType _etype733;
+            xfer += iprot->readListBegin(_etype733, _size730);
+            this->success.resize(_size730);
+            uint32_t _i734;
+            for (_i734 = 0; _i734 < _size730; ++_i734)
             {
-              xfer += iprot->readString(this->success[_i664]);
+              xfer += iprot->readString(this->success[_i734]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1498,10 +1498,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter665;
-      for (_iter665 = this->success.begin(); _iter665 != this->success.end(); ++_iter665)
+      std::vector<std::string> ::const_iterator _iter735;
+      for (_iter735 = this->success.begin(); _iter735 != this->success.end(); ++_iter735)
       {
-        xfer += oprot->writeString((*_iter665));
+        xfer += oprot->writeString((*_iter735));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1545,14 +1545,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size666;
-            ::apache::thrift::protocol::TType _etype669;
-            xfer += iprot->readListBegin(_etype669, _size666);
-            (*(this->success)).resize(_size666);
-            uint32_t _i670;
-            for (_i670 = 0; _i670 < _size666; ++_i670)
+            uint32_t _size736;
+            ::apache::thrift::protocol::TType _etype739;
+            xfer += iprot->readListBegin(_etype739, _size736);
+            (*(this->success)).resize(_size736);
+            uint32_t _i740;
+            for (_i740 = 0; _i740 < _size736; ++_i740)
             {
-              xfer += iprot->readString((*(this->success))[_i670]);
+              xfer += iprot->readString((*(this->success))[_i740]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2610,17 +2610,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size671;
-            ::apache::thrift::protocol::TType _ktype672;
-            ::apache::thrift::protocol::TType _vtype673;
-            xfer += iprot->readMapBegin(_ktype672, _vtype673, _size671);
-            uint32_t _i675;
-            for (_i675 = 0; _i675 < _size671; ++_i675)
+            uint32_t _size741;
+            ::apache::thrift::protocol::TType _ktype742;
+            ::apache::thrift::protocol::TType _vtype743;
+            xfer += iprot->readMapBegin(_ktype742, _vtype743, _size741);
+            uint32_t _i745;
+            for (_i745 = 0; _i745 < _size741; ++_i745)
             {
-              std::string _key676;
-              xfer += iprot->readString(_key676);
-              Type& _val677 = this->success[_key676];
-              xfer += _val677.read(iprot);
+              std::string _key746;
+              xfer += iprot->readString(_key746);
+              Type& _val747 = this->success[_key746];
+              xfer += _val747.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2659,11 +2659,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter678;
-      for (_iter678 = this->success.begin(); _iter678 != this->success.end(); ++_iter678)
+      std::map<std::string, Type> ::const_iterator _iter748;
+      for (_iter748 = this->success.begin(); _iter748 != this->success.end(); ++_iter748)
       {
-        xfer += oprot->writeString(_iter678->first);
-        xfer += _iter678->second.write(oprot);
+        xfer += oprot->writeString(_iter748->first);
+        xfer += _iter748->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2707,17 +2707,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size679;
-            ::apache::thrift::protocol::TType _ktype680;
-            ::apache::thrift::protocol::TType _vtype681;
-            xfer += iprot->readMapBegin(_ktype680, _vtype681, _size679);
-            uint32_t _i683;
-            for (_i683 = 0; _i683 < _size679; ++_i683)
+            uint32_t _size749;
+            ::apache::thrift::protocol::TType _ktype750;
+            ::apache::thrift::protocol::TType _vtype751;
+            xfer += iprot->readMapBegin(_ktype750, _vtype751, _size749);
+            uint32_t _i753;
+            for (_i753 = 0; _i753 < _size749; ++_i753)
             {
-              std::string _key684;
-              xfer += iprot->readString(_key684);
-              Type& _val685 = (*(this->success))[_key684];
-              xfer += _val685.read(iprot);
+              std::string _key754;
+              xfer += iprot->readString(_key754);
+              Type& _val755 = (*(this->success))[_key754];
+              xfer += _val755.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2871,14 +2871,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size686;
-            ::apache::thrift::protocol::TType _etype689;
-            xfer += iprot->readListBegin(_etype689, _size686);
-            this->success.resize(_size686);
-            uint32_t _i690;
-            for (_i690 = 0; _i690 < _size686; ++_i690)
+            uint32_t _size756;
+            ::apache::thrift::protocol::TType _etype759;
+            xfer += iprot->readListBegin(_etype759, _size756);
+            this->success.resize(_size756);
+            uint32_t _i760;
+            for (_i760 = 0; _i760 < _size756; ++_i760)
             {
-              xfer += this->success[_i690].read(iprot);
+              xfer += this->success[_i760].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2933,10 +2933,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter691;
-      for (_iter691 = this->success.begin(); _iter691 != this->success.end(); ++_iter691)
+      std::vector<FieldSchema> ::const_iterator _iter761;
+      for (_iter761 = this->success.begin(); _iter761 != this->success.end(); ++_iter761)
       {
-        xfer += (*_iter691).write(oprot);
+        xfer += (*_iter761).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -2988,14 +2988,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size692;
-            ::apache::thrift::protocol::TType _etype695;
-            xfer += iprot->readListBegin(_etype695, _size692);
-            (*(this->success)).resize(_size692);
-            uint32_t _i696;
-            for (_i696 = 0; _i696 < _size692; ++_i696)
+            uint32_t _size762;
+            ::apache::thrift::protocol::TType _etype765;
+            xfer += iprot->readListBegin(_etype765, _size762);
+            (*(this->success)).resize(_size762);
+            uint32_t _i766;
+            for (_i766 = 0; _i766 < _size762; ++_i766)
             {
-              xfer += (*(this->success))[_i696].read(iprot);
+              xfer += (*(this->success))[_i766].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3181,14 +3181,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size697;
-            ::apache::thrift::protocol::TType _etype700;
-            xfer += iprot->readListBegin(_etype700, _size697);
-            this->success.resize(_size697);
-            uint32_t _i701;
-            for (_i701 = 0; _i701 < _size697; ++_i701)
+            uint32_t _size767;
+            ::apache::thrift::protocol::TType _etype770;
+            xfer += iprot->readListBegin(_etype770, _size767);
+            this->success.resize(_size767);
+            uint32_t _i771;
+            for (_i771 = 0; _i771 < _size767; ++_i771)
             {
-              xfer += this->success[_i701].read(iprot);
+              xfer += this->success[_i771].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3243,10 +3243,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter702;
-      for (_iter702 = this->success.begin(); _iter702 != this->success.end(); ++_iter702)
+      std::vector<FieldSchema> ::const_iterator _iter772;
+      for (_iter772 = this->success.begin(); _iter772 != this->success.end(); ++_iter772)
       {
-        xfer += (*_iter702).write(oprot);
+        xfer += (*_iter772).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3298,14 +3298,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size703;
-            ::apache::thrift::protocol::TType _etype706;
-            xfer += iprot->readListBegin(_etype706, _size703);
-            (*(this->success)).resize(_size703);
-            uint32_t _i707;
-            for (_i707 = 0; _i707 < _size703; ++_i707)
+            uint32_t _size773;
+            ::apache::thrift::protocol::TType _etype776;
+            xfer += iprot->readListBegin(_etype776, _size773);
+            (*(this->success)).resize(_size773);
+            uint32_t _i777;
+            for (_i777 = 0; _i777 < _size773; ++_i777)
             {
-              xfer += (*(this->success))[_i707].read(iprot);
+              xfer += (*(this->success))[_i777].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3475,14 +3475,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size708;
-            ::apache::thrift::protocol::TType _etype711;
-            xfer += iprot->readListBegin(_etype711, _size708);
-            this->success.resize(_size708);
-            uint32_t _i712;
-            for (_i712 = 0; _i712 < _size708; ++_i712)
+            uint32_t _size778;
+            ::apache::thrift::protocol::TType _etype781;
+            xfer += iprot->readListBegin(_etype781, _size778);
+            this->success.resize(_size778);
+            uint32_t _i782;
+            for (_i782 = 0; _i782 < _size778; ++_i782)
             {
-              xfer += this->success[_i712].read(iprot);
+              xfer += this->success[_i782].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3537,10 +3537,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter713;
-      for (_iter713 = this->success.begin(); _iter713 != this->success.end(); ++_iter713)
+      std::vector<FieldSchema> ::const_iterator _iter783;
+      for (_iter783 = this->success.begin(); _iter783 != this->success.end(); ++_iter783)
       {
-        xfer += (*_iter713).write(oprot);
+        xfer += (*_iter783).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3592,14 +3592,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size714;
-            ::apache::thrift::protocol::TType _etype717;
-            xfer += iprot->readListBegin(_etype717, _size714);
-            (*(this->success)).resize(_size714);
-            uint32_t _i718;
-            for (_i718 = 0; _i718 < _size714; ++_i718)
+            uint32_t _size784;
+            ::apache::thrift::protocol::TType _etype787;
+            xfer += iprot->readListBegin(_etype787, _size784);
+            (*(this->success)).resize(_size784);
+            uint32_t _i788;
+            for (_i788 = 0; _i788 < _size784; ++_i788)
             {
-              xfer += (*(this->success))[_i718].read(iprot);
+              xfer += (*(this->success))[_i788].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3785,14 +3785,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size719;
-            ::apache::thrift::protocol::TType _etype722;
-            xfer += iprot->readListBegin(_etype722, _size719);
-            this->success.resize(_size719);
-            uint32_t _i723;
-            for (_i723 = 0; _i723 < _size719; ++_i723)
+            uint32_t _size789;
+            ::apache::thrift::protocol::TType _etype792;
+            xfer += iprot->readListBegin(_etype792, _size789);
+            this->success.resize(_size789);
+            uint32_t _i793;
+            for (_i793 = 0; _i793 < _size789; ++_i793)
             {
-              xfer += this->success[_i723].read(iprot);
+              xfer += this->success[_i793].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3847,10 +3847,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter724;
-      for (_iter724 = this->success.begin(); _iter724 != this->success.end(); ++_iter724)
+      std::vector<FieldSchema> ::const_iterator _iter794;
+      for (_iter794 = this->success.begin(); _iter794 != this->success.end(); ++_iter794)
       {
-        xfer += (*_iter724).write(oprot);
+        xfer += (*_iter794).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3902,14 +3902,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size725;
-            ::apache::thrift::protocol::TType _etype728;
-            xfer += iprot->readListBegin(_etype728, _size725);
-            (*(this->success)).resize(_size725);
-            uint32_t _i729;
-            for (_i729 = 0; _i729 < _size725; ++_i729)
+            uint32_t _size795;
+            ::apache::thrift::protocol::TType _etype798;
+            xfer += iprot->readListBegin(_etype798, _size795);
+            (*(this->success)).resize(_size795);
+            uint32_t _i799;
+            for (_i799 = 0; _i799 < _size795; ++_i799)
             {
-              xfer += (*(this->success))[_i729].read(iprot);
+              xfer += (*(this->success))[_i799].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5079,14 +5079,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size730;
-            ::apache::thrift::protocol::TType _etype733;
-            xfer += iprot->readListBegin(_etype733, _size730);
-            this->success.resize(_size730);
-            uint32_t _i734;
-            for (_i734 = 0; _i734 < _size730; ++_i734)
+            uint32_t _size800;
+            ::apache::thrift::protocol::TType _etype803;
+            xfer += iprot->readListBegin(_etype803, _size800);
+            this->success.resize(_size800);
+            uint32_t _i804;
+            for (_i804 = 0; _i804 < _size800; ++_i804)
             {
-              xfer += iprot->readString(this->success[_i734]);
+              xfer += iprot->readString(this->success[_i804]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5125,10 +5125,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter735;
-      for (_iter735 = this->success.begin(); _iter735 != this->success.end(); ++_iter735)
+      std::vector<std::string> ::const_iterator _iter805;
+      for (_iter805 = this->success.begin(); _iter805 != this->success.end(); ++_iter805)
       {
-        xfer += oprot->writeString((*_iter735));
+        xfer += oprot->writeString((*_iter805));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5172,14 +5172,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size736;
-            ::apache::thrift::protocol::TType _etype739;
-            xfer += iprot->readListBegin(_etype739, _size736);
-            (*(this->success)).resize(_size736);
-            uint32_t _i740;
-            for (_i740 = 0; _i740 < _size736; ++_i740)
+            uint32_t _size806;
+            ::apache::thrift::protocol::TType _etype809;
+            xfer += iprot->readListBegin(_etype809, _size806);
+            (*(this->success)).resize(_size806);
+            uint32_t _i810;
+            for (_i810 = 0; _i810 < _size806; ++_i810)
             {
-              xfer += iprot->readString((*(this->success))[_i740]);
+              xfer += iprot->readString((*(this->success))[_i810]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5317,14 +5317,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size741;
-            ::apache::thrift::protocol::TType _etype744;
-            xfer += iprot->readListBegin(_etype744, _size741);
-            this->success.resize(_size741);
-            uint32_t _i745;
-            for (_i745 = 0; _i745 < _size741; ++_i745)
+            uint32_t _size811;
+            ::apache::thrift::protocol::TType _etype814;
+            xfer += iprot->readListBegin(_etype814, _size811);
+            this->success.resize(_size811);
+            uint32_t _i815;
+            for (_i815 = 0; _i815 < _size811; ++_i815)
             {
-              xfer += iprot->readString(this->success[_i745]);
+              xfer += iprot->readString(this->success[_i815]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5363,10 +5363,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter746;
-      for (_iter746 = this->success.begin(); _iter746 != this->success.end(); ++_iter746)
+      std::vector<std::string> ::const_iterator _iter816;
+      for (_iter816 = this->success.begin(); _iter816 != this->success.end(); ++_iter816)
       {
-        xfer += oprot->writeString((*_iter746));
+        xfer += oprot->writeString((*_iter816));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5410,14 +5410,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size747;
-            ::apache::thrift::protocol::TType _etype750;
-            xfer += iprot->readListBegin(_etype750, _size747);
-            (*(this->success)).resize(_size747);
-            uint32_t _i751;
-            for (_i751 = 0; _i751 < _size747; ++_i751)
+            uint32_t _size817;
+            ::apache::thrift::protocol::TType _etype820;
+            xfer += iprot->readListBegin(_etype820, _size817);
+            (*(this->success)).resize(_size817);
+            uint32_t _i821;
+            for (_i821 = 0; _i821 < _size817; ++_i821)
             {
-              xfer += iprot->readString((*(this->success))[_i751]);
+              xfer += iprot->readString((*(this->success))[_i821]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5725,14 +5725,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size752;
-            ::apache::thrift::protocol::TType _etype755;
-            xfer += iprot->readListBegin(_etype755, _size752);
-            this->tbl_names.resize(_size752);
-            uint32_t _i756;
-            for (_i756 = 0; _i756 < _size752; ++_i756)
+            uint32_t _size822;
+            ::apache::thrift::protocol::TType _etype825;
+            xfer += iprot->readListBegin(_etype825, _size822);
+            this->tbl_names.resize(_size822);
+            uint32_t _i826;
+            for (_i826 = 0; _i826 < _size822; ++_i826)
             {
-              xfer += iprot->readString(this->tbl_names[_i756]);
+              xfer += iprot->readString(this->tbl_names[_i826]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5765,10 +5765,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter757;
-    for (_iter757 = this->tbl_names.begin(); _iter757 != this->tbl_names.end(); ++_iter757)
+    std::vector<std::string> ::const_iterator _iter827;
+    for (_iter827 = this->tbl_names.begin(); _iter827 != this->tbl_names.end(); ++_iter827)
     {
-      xfer += oprot->writeString((*_iter757));
+      xfer += oprot->writeString((*_iter827));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5797,10 +5797,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter758;
-    for (_iter758 = (*(this->tbl_names)).begin(); _iter758 != (*(this->tbl_names)).end(); ++_iter758)
+    std::vector<std::string> ::const_iterator _iter828;
+    for (_iter828 = (*(this->tbl_names)).begin(); _iter828 != (*(this->tbl_names)).end(); ++_iter828)
     {
-      xfer += oprot->writeString((*_iter758));
+      xfer += oprot->writeString((*_iter828));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5841,14 +5841,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size759;
-            ::apache::thrift::protocol::TType _etype762;
-            xfer += iprot->readListBegin(_etype762, _size759);
-            this->success.resize(_size759);
-            uint32_t _i763;
-            for (_i763 = 0; _i763 < _size759; ++_i763)
+            uint32_t _size829;
+            ::apache::thrift::protocol::TType _etype832;
+            xfer += iprot->readListBegin(_etype832, _size829);
+            this->success.resize(_size829);
+            uint32_t _i833;
+            for (_i833 = 0; _i833 < _size829; ++_i833)
             {
-              xfer += this->success[_i763].read(iprot);
+              xfer += this->success[_i833].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5903,10 +5903,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter764;
-      for (_iter764 = this->success.begin(); _iter764 != this->success.end(); ++_iter764)
+      std::vector<Table> ::const_iterator _iter834;
+      for (_iter834 = this->success.begin(); _iter834 != this->success.end(); ++_iter834)
       {
-        xfer += (*_iter764).write(oprot);
+        xfer += (*_iter834).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5958,14 +5958,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size765;
-            ::apache::thrift::protocol::TType _etype768;
-            xfer += iprot->readListBegin(_etype768, _size765);
-            (*(this->success)).resize(_size765);
-            uint32_t _i769;
-            for (_i769 = 0; _i769 < _size765; ++_i769)
+            uint32_t _size835;
+            ::apache::thrift::protocol::TType _etype838;
+            xfer += iprot->readListBegin(_etype838, _size835);
+            (*(this->success)).resize(_size835);
+            uint32_t _i839;
+            for (_i839 = 0; _i839 < _size835; ++_i839)
             {
-              xfer += (*(this->success))[_i769].read(iprot);
+              xfer += (*(this->success))[_i839].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6151,14 +6151,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size770;
-            ::apache::thrift::protocol::TType _etype773;
-            xfer += iprot->readListBegin(_etype773, _size770);
-            this->success.resize(_size770);
-            uint32_t _i774;
-            for (_i774 = 0; _i774 < _size770; ++_i774)
+            uint32_t _size840;
+            ::apache::thrift::protocol::TType _etype843;
+            xfer += iprot->readListBegin(_etype843, _size840);
+            this->success.resize(_size840);
+            uint32_t _i844;
+            for (_i844 = 0; _i844 < _size840; ++_i844)
             {
-              xfer += iprot->readString(this->success[_i774]);
+              xfer += iprot->readString(this->success[_i844]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6213,10 +6213,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter775;
-      for (_iter775 = this->success.begin(); _iter775 != this->success.end(); ++_iter775)
+      std::vector<std::string> ::const_iterator _iter845;
+      for (_iter845 = this->success.begin(); _iter845 != this->success.end(); ++_iter845)
       {
-        xfer += oprot->writeString((*_iter775));
+        xfer += oprot->writeString((*_iter845));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6268,14 +6268,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size776;
-            ::apache::thrift::protocol::TType _etype779;
-            xfer += iprot->readListBegin(_etype779, _size776);
-            (*(this->success)).resize(_size776);
-            uint32_t _i780;
-            for (_i780 = 0; _i780 < _size776; ++_i780)
+            uint32_t _size846;
+            ::apache::thrift::protocol::TType _etype849;
+            xfer += iprot->readListBegin(_etype849, _size846);
+            (*(this->success)).resize(_size846);
+            uint32_t _i850;
+            for (_i850 = 0; _i850 < _size846; ++_i850)
             {
-              xfer += iprot->readString((*(this->success))[_i780]);
+              xfer += iprot->readString((*(this->success))[_i850]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7603,14 +7603,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size781;
-            ::apache::thrift::protocol::TType _etype784;
-            xfer += iprot->readListBegin(_etype784, _size781);
-            this->new_parts.resize(_size781);
-            uint32_t _i785;
-            for (_i785 = 0; _i785 < _size781; ++_i785)
+            uint32_t _size851;
+            ::apache::thrift::protocol::TType _etype854;
+            xfer += iprot->readListBegin(_etype854, _size851);
+            this->new_parts.resize(_size851);
+            uint32_t _i855;
+            for (_i855 = 0; _i855 < _size851; ++_i855)
             {
-              xfer += this->new_parts[_i785].read(iprot);
+              xfer += this->new_parts[_i855].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7639,10 +7639,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter786;
-    for (_iter786 = this->new_parts.begin(); _iter786 != this->new_parts.end(); ++_iter786)
+    std::vector<Partition> ::const_iterator _iter856;
+    for (_iter856 = this->new_parts.begin(); _iter856 != this->new_parts.end(); ++_iter856)
     {
-      xfer += (*_iter786).write(oprot);
+      xfer += (*_iter856).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7667,10 +7667,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter787;
-    for (_iter787 = (*(this->new_parts)).begin(); _iter787 != (*(this->new_parts)).end(); ++_iter787)
+    std::vector<Partition> ::const_iterator _iter857;
+    for (_iter857 = (*(this->new_parts)).begin(); _iter857 != (*(this->new_parts)).end(); ++_iter857)
     {
-      xfer += (*_iter787).write(oprot);
+      xfer += (*_iter857).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7877,14 +7877,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size788;
-            ::apache::thrift::protocol::TType _etype791;
-            xfer += iprot->readListBegin(_etype791, _size788);
-            this->new_parts.resize(_size788);
-            uint32_t _i792;
-            for (_i792 = 0; _i792 < _size788; ++_i792)
+            uint32_t _size858;
+            ::apache::thrift::protocol::TType _etype861;
+            xfer += iprot->readListBegin(_etype861, _size858);
+            this->new_parts.resize(_size858);
+            uint32_t _i862;
+            for (_i862 = 0; _i862 < _size858; ++_i862)
             {
-              xfer += this->new_parts[_i792].read(iprot);
+              xfer += this->new_parts[_i862].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7913,10 +7913,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter793;
-    for (_iter793 = this->new_parts.begin(); _iter793 != this->new_parts.end(); ++_iter793)
+    std::vector<PartitionSpec> ::const_iterator _iter863;
+    for (_iter863 = this->new_parts.begin(); _iter863 != this->new_parts.end(); ++_iter863)
     {
-      xfer += (*_iter793).write(oprot);
+      xfer += (*_iter863).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7941,10 +7941,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter794;
-    for (_iter794 = (*(this->new_parts)).begin(); _iter794 != (*(this->new_parts)).end(); ++_iter794)
+    std::vector<PartitionSpec> ::const_iterator _iter864;
+    for (_iter864 = (*(this->new_parts)).begin(); _iter864 != (*(this->new_parts)).end(); ++_iter864)
     {
-      xfer += (*_iter794).write(oprot);
+      xfer += (*_iter864).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8167,14 +8167,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size795;
-            ::apache::thrift::protocol::TType _etype798;
-            xfer += iprot->readListBegin(_etype798, _size795);
-            this->part_vals.resize(_size795);
-            uint32_t _i799;
-            for (_i799 = 0; _i799 < _size795; ++_i799)
+            uint32_t _size865;
+            ::apache::thrift::protocol::TType _etype868;
+            xfer += iprot->readListBegin(_etype868, _size865);
+            this->part_vals.resize(_size865);
+            uint32_t _i869;
+            for (_i869 = 0; _i869 < _size865; ++_i869)
             {
-              xfer += iprot->readString(this->part_vals[_i799]);
+              xfer += iprot->readString(this->part_vals[_i869]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8211,10 +8211,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter800;
-    for (_iter800 = this->part_vals.begin(); _iter800 != this->part_vals.end(); ++_iter800)
+    std::vector<std::string> ::const_iterator _iter870;
+    for (_iter870 = this->part_vals.begin(); _iter870 != this->part_vals.end(); ++_iter870)
     {
-      xfer += oprot->writeString((*_iter800));
+      xfer += oprot->writeString((*_iter870));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8247,10 +8247,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter801;
-    for (_iter801 = (*(this->part_vals)).begin(); _iter801 != (*(this->part_vals)).end(); ++_iter801)
+    std::vector<std::string> ::const_iterator _iter871;
+    for (_iter871 = (*(this->part_vals)).begin(); _iter871 != (*(this->part_vals)).end(); ++_iter871)
     {
-      xfer += oprot->writeString((*_iter801));
+      xfer += oprot->writeString((*_iter871));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8719,14 +8719,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size802;
-            ::apache::thrift::protocol::TType _etype805;
-            xfer += iprot->readListBegin(_etype805, _size802);
-            this->part_vals.resize(_size802);
-            uint32_t _i806;
-            for (_i806 = 0; _i806 < _size802; ++_i806)
+            uint32_t _size872;
+            ::apache::thrift::protocol::TType _etype875;
+            xfer += iprot->readListBegin(_etype875, _size872);
+            this->part_vals.resize(_size872);
+            uint32_t _i876;
+            for (_i876 = 0; _i876 < _size872; ++_i876)
             {
-              xfer += iprot->readString(this->part_vals[_i806]);
+              xfer += iprot->readString(this->part_vals[_i876]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8771,10 +8771,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter807;
-    for (_iter807 = this->part_vals.begin(); _iter807 != this->part_vals.end(); ++_iter807)
+    std::vector<std::string> ::const_iterator _iter877;
+    for (_iter877 = this->part_vals.begin(); _iter877 != this->part_vals.end(); ++_iter877)
     {
-      xfer += oprot->writeString((*_iter807));
+      xfer += oprot->writeString((*_iter877));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8811,10 +8811,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter808;
-    for (_iter808 = (*(this->part_vals)).begin(); _iter808 != (*(this->part_vals)).end(); ++_iter808)
+    std::vector<std::string> ::const_iterator _iter878;
+    for (_iter878 = (*(this->part_vals)).begin(); _iter878 != (*(this->part_vals)).end(); ++_iter878)
     {
-      xfer += oprot->writeString((*_iter808));
+      xfer += oprot->writeString((*_iter878));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9613,14 +9613,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size809;
-            ::apache::thrift::protocol::TType _etype812;
-            xfer += iprot->readListBegin(_etype812, _size809);
-            this->part_vals.resize(_size809);
-            uint32_t _i813;
-            for (_i813 = 0; _i813 < _size809; ++_i813)
+            uint32_t _size879;
+            ::apache::thrift::protocol::TType _etype882;
+            xfer += iprot->readListBegin(_etype882, _size879);
+            this->part_vals.resize(_size879);
+            uint32_t _i883;
+            for (_i883 = 0; _i883 < _size879; ++_i883)
             {
-              xfer += iprot->readString(this->part_vals[_i813]);
+              xfer += iprot->readString(this->part_vals[_i883]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9665,10 +9665,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter814;
-    for (_iter814 = this->part_vals.begin(); _iter814 != this->part_vals.end(); ++_iter814)
+    std::vector<std::string> ::const_iterator _iter884;
+    for (_iter884 = this->part_vals.begin(); _iter884 != this->part_vals.end(); ++_iter884)
     {
-      xfer += oprot->writeString((*_iter814));
+      xfer += oprot->writeString((*_iter884));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9705,10 +9705,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter815;
-    for (_iter815 = (*(this->part_vals)).begin(); _iter815 != (*(this->part_vals)).end(); ++_iter815)
+    std::vector<std::string> ::const_iterator _iter885;
+    for (_iter885 = (*(this->part_vals)).begin(); _iter885 != (*(this->part_vals)).end(); ++_iter885)
     {
-      xfer += oprot->writeString((*_iter815));
+      xfer += oprot->writeString((*_iter885));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9915,14 +9915,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size816;
-            ::apache::thrift::protocol::TType _etype819;
-            xfer += iprot->readListBegin(_etype819, _size816);
-            this->part_vals.resize(_size816);
-            uint32_t _i820;
-            for (_i820 = 0; _i820 < _size816; ++_i820)
+            uint32_t _size886;
+            ::apache::thrift::protocol::TType _etype889;
+            xfer += iprot->readListBegin(_etype889, _size886);
+            this->part_vals.resize(_size886);
+            uint32_t _i890;
+            for (_i890 = 0; _i890 < _size886; ++_i890)
             {
-              xfer += iprot->readString(this->part_vals[_i820]);
+              xfer += iprot->readString(this->part_vals[_i890]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9975,10 +9975,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter821;
-    for (_iter821 = this->part_vals.begin(); _iter821 != this->part_vals.end(); ++_iter821)
+    std::vector<std::string> ::const_iterator _iter891;
+    for (_iter891 = this->part_vals.begin(); _iter891 != this->part_vals.end(); ++_iter891)
     {
-      xfer += oprot->writeString((*_iter821));
+      xfer += oprot->writeString((*_iter891));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10019,10 +10019,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter822;
-    for (_iter822 = (*(this->part_vals)).begin(); _iter822 != (*(this->part_vals)).end(); ++_iter822)
+    std::vector<std::string> ::const_iterator _iter892;
+    for (_iter892 = (*(this->part_vals)).begin(); _iter892 != (*(this->part_vals)).end(); ++_iter892)
     {
-      xfer += oprot->writeString((*_iter822));
+      xfer += oprot->writeString((*_iter892));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11023,14 +11023,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size823;
-            ::apache::thrift::protocol::TType _etype826;
-            xfer += iprot->readListBegin(_etype826, _size823);
-            this->part_vals.resize(_size823);
-            uint32_t _i827;
-            for (_i827 = 0; _i827 < _size823; ++_i827)
+            uint32_t _size893;
+            ::apache::thrift::protocol::TType _etype896;
+            xfer += iprot->readListBegin(_etype896, _size893);
+            this->part_vals.resize(_size893);
+            uint32_t _i897;
+            for (_i897 = 0; _i897 < _size893; ++_i897)
             {
-              xfer += iprot->readString(this->part_vals[_i827]);
+              xfer += iprot->readString(this->part_vals[_i897]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11067,10 +11067,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter828;
-    for (_iter828 = this->part_vals.begin(); _iter828 != this->part_vals.end(); ++_iter828)
+    std::vector<std::string> ::const_iterator _iter898;
+    for (_iter898 = this->part_vals.begin(); _iter898 != this->part_vals.end(); ++_iter898)
     {
-      xfer += oprot->writeString((*_iter828));
+      xfer += oprot->writeString((*_iter898));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11103,10 +11103,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter829;
-    for (_iter829 = (*(this->part_vals)).begin(); _iter829 != (*(this->part_vals)).end(); ++_iter829)
+    std::vector<std::string> ::const_iterator _iter899;
+    for (_iter899 = (*(this->part_vals)).begin(); _iter899 != (*(this->part_vals)).end(); ++_iter899)
     {
-      xfer += oprot->writeString((*_iter829));
+      xfer += oprot->writeString((*_iter899));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11293,17 +11293,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size830;
-            ::apache::thrift::protocol::TType _ktype831;
-            ::apache::thrift::protocol::TType _vtype832;
-            xfer += iprot->readMapBegin(_ktype831, _vtype832, _size830);
-            uint32_t _i834;
-            for (_i834 = 0; _i834 < _size830; ++_i834)
+            uint32_t _size900;
+            ::apache::thrift::protocol::TType _ktype901;
+            ::apache::thrift::protocol::TType _vtype902;
+            xfer += iprot->readMapBegin(_ktype901, _vtype902, _size900);
+            uint32_t _i904;
+            for (_i904 = 0; _i904 < _size900; ++_i904)
             {
-              std::string _key835;
-              xfer += iprot->readString(_key835);
-              std::string& _val836 = this->partitionSpecs[_key835];
-              xfer += iprot->readString(_val836);
+              std::string _key905;
+              xfer += iprot->readString(_key905);
+              std::string& _val906 = this->partitionSpecs[_key905];
+              xfer += iprot->readString(_val906);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11364,11 +11364,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter837;
-    for (_iter837 = this->partitionSpecs.begin(); _iter837 != this->partitionSpecs.end(); ++_iter837)
+    std::map<std::string, std::string> ::const_iterator _iter907;
+    for (_iter907 = this->partitionSpecs.begin(); _iter907 != this->partitionSpecs.end(); ++_iter907)
     {
-      xfer += oprot->writeString(_iter837->first);
-      xfer += oprot->writeString(_iter837->second);
+      xfer += oprot->writeString(_iter907->first);
+      xfer += oprot->writeString(_iter907->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11409,11 +11409,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter838;
-    for (_iter838 = (*(this->partitionSpecs)).begin(); _iter838 != (*(this->partitionSpecs)).end(); ++_iter838)
+    std::map<std::string, std::string> ::const_iterator _iter908;
+    for (_iter908 = (*(this->partitionSpecs)).begin(); _iter908 != (*(this->partitionSpecs)).end(); ++_iter908)
     {
-      xfer += oprot->writeString(_iter838->first);
-      xfer += oprot->writeString(_iter838->second);
+      xfer += oprot->writeString(_iter908->first);
+      xfer += oprot->writeString(_iter908->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11672,14 +11672,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size839;
-            ::apache::thrift::protocol::TType _etype842;
-            xfer += iprot->readListBegin(_etype842, _size839);
-            this->part_vals.resize(_size839);
-            uint32_t _i843;
-            for (_i843 = 0; _i843 < _size839; ++_i843)
+            uint32_t _size909;
+            ::apache::thrift::protocol::TType _etype912;
+            xfer += iprot->readListBegin(_etype912, _size909);
+            this->part_vals.resize(_size909);
+            uint32_t _i913;
+            for (_i913 = 0; _i913 < _size909; ++_i913)
             {
-              xfer += iprot->readString(this->part_vals[_i843]);
+              xfer += iprot->readString(this->part_vals[_i913]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11700,14 +11700,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size844;
-            ::apache::thrift::protocol::TType _etype847;
-            xfer += iprot->readListBegin(_etype847, _size844);
-            this->group_names.resize(_size844);
-            uint32_t _i848;
-            for (_i848 = 0; _i848 < _size844; ++_i848)
+            uint32_t _size914;
+            ::apache::thrift::protocol::TType _etype917;
+            xfer += iprot->readListBegin(_etype917, _size914);
+            this->group_names.resize(_size914);
+            uint32_t _i918;
+            for (_i918 = 0; _i918 < _size914; ++_i918)
             {
-              xfer += iprot->readString(this->group_names[_i848]);
+              xfer += iprot->readString(this->group_names[_i918]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11744,10 +11744,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter849;
-    for (_iter849 = this->part_vals.begin(); _iter849 != this->part_vals.end(); ++_iter849)
+    std::vector<std::string> ::const_iterator _iter919;
+    for (_iter919 = this->part_vals.begin(); _iter919 != this->part_vals.end(); ++_iter919)
     {
-      xfer += oprot->writeString((*_iter849));
+      xfer += oprot->writeString((*_iter919));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11760,10 +11760,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter850;
-    for (_iter850 = this->group_names.begin(); _iter850 != this->group_names.end(); ++_iter850)
+    std::vector<std::string> ::const_iterator _iter920;
+    for (_iter920 = this->group_names.begin(); _iter920 != this->group_names.end(); ++_iter920)
     {
-      xfer += oprot->writeString((*_iter850));
+      xfer += oprot->writeString((*_iter920));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11796,10 +11796,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter851;
-    for (_iter851 = (*(this->part_vals)).begin(); _iter851 != (*(this->part_vals)).end(); ++_iter851)
+    std::vector<std::string> ::const_iterator _iter921;
+    for (_iter921 = (*(this->part_vals)).begin(); _iter921 != (*(this->part_vals)).end(); ++_iter921)
     {
-      xfer += oprot->writeString((*_iter851));
+      xfer += oprot->writeString((*_iter921));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11812,10 +11812,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter852;
-    for (_iter852 = (*(this->group_names)).begin(); _iter852 != (*(this->group_names)).end(); ++_iter852)
+    std::vector<std::string> ::const_iterator _iter922;
+    for (_iter922 = (*(this->group_names)).begin(); _iter922 != (*(this->group_names)).end(); ++_iter922)
     {
-      xfer += oprot->writeString((*_iter852));
+      xfer += oprot->writeString((*_iter922));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12372,14 +12372,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size853;
-            ::apache::thrift::protocol::TType _etype856;
-            xfer += iprot->readListBegin(_etype856, _size853);
-            this->success.resize(_size853);
-            uint32_t _i857;
-            for (_i857 = 0; _i857 < _size853; ++_i857)
+            uint32_t _size923;
+            ::apache::thrift::protocol::TType _etype926;
+            xfer += iprot->readListBegin(_etype926, _size923);
+            this->success.resize(_size923);
+            uint32_t _i927;
+            for (_i927 = 0; _i927 < _size923; ++_i927)
             {
-              xfer += this->success[_i857].read(iprot);
+              xfer += this->success[_i927].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12426,10 +12426,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter858;
-      for (_iter858 = this->success.begin(); _iter858 != this->success.end(); ++_iter858)
+      std::vector<Partition> ::const_iterator _iter928;
+      for (_iter928 = this->success.begin(); _iter928 != this->success.end(); ++_iter928)
       {
-        xfer += (*_iter858).write(oprot);
+        xfer += (*_iter928).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12477,14 +12477,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size859;
-            ::apache::thrift::protocol::TType _etype862;
-            xfer += iprot->readListBegin(_etype862, _size859);
-            (*(this->success)).resize(_size859);
-            uint32_t _i863;
-            for (_i863 = 0; _i863 < _size859; ++_i863)
+            uint32_t _size929;
+            ::apache::thrift::protocol::TType _etype932;
+            xfer += iprot->readListBegin(_etype932, _size929);
+            (*(this->success)).resize(_size929);
+            uint32_t _i933;
+            for (_i933 = 0; _i933 < _size929; ++_i933)
             {
-              xfer += (*(this->success))[_i863].read(iprot);
+              xfer += (*(this->success))[_i933].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12582,14 +12582,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size864;
-            ::apache::thrift::protocol::TType _etype867;
-            xfer += iprot->readListBegin(_etype867, _size864);
-            this->group_names.resize(_size864);
-            uint32_t _i868;
-            for (_i868 = 0; _i868 < _size864; ++_i868)
+            uint32_t _size934;
+            ::apache::thrift::protocol::TType _etype937;
+            xfer += iprot->readListBegin(_etype937, _size934);
+            this->group_names.resize(_size934);
+            uint32_t _i938;
+            for (_i938 = 0; _i938 < _size934; ++_i938)
             {
-              xfer += iprot->readString(this->group_names[_i868]);
+              xfer += iprot->readString(this->group_names[_i938]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12634,10 +12634,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter869;
-    for (_iter869 = this->group_names.begin(); _iter869 != this->group_names.end(); ++_iter869)
+    std::vector<std::string> ::const_iterator _iter939;
+    for (_iter939 = this->group_names.begin(); _iter939 != this->group_names.end(); ++_iter939)
     {
-      xfer += oprot->writeString((*_iter869));
+      xfer += oprot->writeString((*_iter939));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12678,10 +12678,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter870;
-    for (_iter870 = (*(this->group_names)).begin(); _iter870 != (*(this->group_names)).end(); ++_iter870)
+    std::vector<std::string> ::const_iterator _iter940;
+    for (_iter940 = (*(this->group_names)).begin(); _iter940 != (*(this->group_names)).end(); ++_iter940)
     {
-      xfer += oprot->writeString((*_iter870));
+      xfer += oprot->writeString((*_iter940));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12722,14 +12722,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size871;
-            ::apache::thrift::protocol::TType _etype874;
-            xfer += iprot->readListBegin(_etype874, _size871);
-            this->success.resize(_size871);
-            uint32_t _i875;
-            for (_i875 = 0; _i875 < _size871; ++_i875)
+            uint32_t _size941;
+            ::apache::thrift::protocol::TType _etype944;
+            xfer += iprot->readListBegin(_etype944, _size941);
+            this->success.resize(_size941);
+            uint32_t _i945;
+            for (_i945 = 0; _i945 < _size941; ++_i945)
             {
-              xfer += this->success[_i875].read(iprot);
+              xfer += this->success[_i945].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12776,10 +12776,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter876;
-      for (_iter876 = this->success.begin(); _iter876 != this->success.end(); ++_iter876)
+      std::vector<Partition> ::const_iterator _iter946;
+      for (_iter946 = this->success.begin(); _iter946 != this->success.end(); ++_iter946)
       {
-        xfer += (*_iter876).write(oprot);
+        xfer += (*_iter946).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12827,14 +12827,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size877;
-            ::apache::thrift::protocol::TType _etype880;
-            xfer += iprot->readListBegin(_etype880, _size877);
-            (*(this->success)).resize(_size877);
-            uint32_t _i881;
-            for (_i881 = 0; _i881 < _size877; ++_i881)
+            uint32_t _size947;
+            ::apache::thrift::protocol::TType _etype950;
+            xfer += iprot->readListBegin(_etype950, _size947);
+            (*(this->success)).resize(_size947);
+            uint32_t _i951;
+            for (_i951 = 0; _i951 < _size947; ++_i951)
             {
-              xfer += (*(this->success))[_i881].read(iprot);
+              xfer += (*(this->success))[_i951].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13012,14 +13012,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size882;
-            ::apache::thrift::protocol::TType _etype885;
-            xfer += iprot->readListBegin(_etype885, _size882);
-            this->success.resize(_size882);
-            uint32_t _i886;
-            for (_i886 = 0; _i886 < _size882; ++_i886)
+            uint32_t _size952;
+            ::apache::thrift::protocol::TType _etype955;
+            xfer += iprot->readListBegin(_etype955, _size952);
+            this->success.resize(_size952);
+            uint32_t _i956;
+            for (_i956 = 0; _i956 < _size952; ++_i956)
             {
-              xfer += this->success[_i886].read(iprot);
+              xfer += this->success[_i956].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13066,10 +13066,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter887;
-      for (_iter887 = this->success.begin(); _iter887 != this->success.end(); ++_iter887)
+      std::vector<PartitionSpec> ::const_iterator _iter957;
+      for (_iter957 = this->success.begin(); _iter957 != this->success.end(); ++_iter957)
       {
-        xfer += (*_iter887).write(oprot);
+        xfer += (*_iter957).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13117,14 +13117,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size888;
-            ::apache::thrift::protocol::TType _etype891;
-            xfer += iprot->readListBegin(_etype891, _size888);
-            (*(this->success)).resize(_size888);
-            uint32_t _i892;
-            for (_i892 = 0; _i892 < _size888; ++_i892)
+            uint32_t _size958;
+            ::apache::thrift::protocol::TType _etype961;
+            xfer += iprot->readListBegin(_etype961, _size958);
+            (*(this->success)).resize(_size958);
+            uint32_t _i962;
+            for (_i962 = 0; _i962 < _size958; ++_i962)
             {
-              xfer += (*(this->success))[_i892].read(iprot);
+              xfer += (*(this->success))[_i962].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13302,14 +13302,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size893;
-            ::apache::thrift::protocol::TType _etype896;
-            xfer += iprot->readListBegin(_etype896, _size893);
-            this->success.resize(_size893);
-            uint32_t _i897;
-            for (_i897 = 0; _i897 < _size893; ++_i897)
+            uint32_t _size963;
+            ::apache::thrift::protocol::TType _etype966;
+            xfer += iprot->readListBegin(_etype966, _size963);
+            this->success.resize(_size963);
+            uint32_t _i967;
+            for (_i967 = 0; _i967 < _size963; ++_i967)
             {
-              xfer += iprot->readString(this->success[_i897]);
+              xfer += iprot->readString(this->success[_i967]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13348,10 +13348,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter898;
-      for (_iter898 = this->success.begin(); _iter898 != this->success.end(); ++_iter898)
+      std::vector<std::string> ::const_iterator _iter968;
+      for (_iter968 = this->success.begin(); _iter968 != this->success.end(); ++_iter968)
       {
-        xfer += oprot->writeString((*_iter898));
+        xfer += oprot->writeString((*_iter968));
       }
       xfer += oprot->writeListEnd();
     }
@@ -13395,14 +13395,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size899;
-            ::apache::thrift::protocol::TType _etype902;
-            xfer += iprot->readListBegin(_etype902, _size899);
-            (*(this->success)).resize(_size899);
-            uint32_t _i903;
-            for (_i903 = 0; _i903 < _size899; ++_i903)
+            uint32_t _size969;
+            ::apache::thrift::protocol::TType _etype972;
+            xfer += iprot->readListBegin(_etype972, _size969);
+            (*(this->success)).resize(_size969);
+            uint32_t _i973;
+            for (_i973 = 0; _i973 < _size969; ++_i973)
             {
-              xfer += iprot->readString((*(this->success))[_i903]);
+              xfer += iprot->readString((*(this->success))[_i973]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13476,14 +13476,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size904;
-            ::apache::thrift::protocol::TType _etype907;
-            xfer += iprot->readListBegin(_etype907, _size904);
-            this->part_vals.resize(_size904);
-            uint32_t _i908;
-            for (_i908 = 0; _i908 < _size904; ++_i908)
+            uint32_t _size974;
+            ::apache::thrift::protocol::TType _etype977;
+            xfer += iprot->readListBegin(_etype977, _size974);
+            this->part_vals.resize(_size974);
+            uint32_t _i978;
+            for (_i978 = 0; _i978 < _size974; ++_i978)
             {
-              xfer += iprot->readString(this->part_vals[_i908]);
+              xfer += iprot->readString(this->part_vals[_i978]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13528,10 +13528,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter909;
-    for (_iter909 = this->part_vals.begin(); _iter909 != this->part_vals.end(); ++_iter909)
+    std::vector<std::string> ::const_iterator _iter979;
+    for (_iter979 = this->part_vals.begin(); _iter979 != this->part_vals.end(); ++_iter979)
     {
-      xfer += oprot->writeString((*_iter909));
+      xfer += oprot->writeString((*_iter979));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13568,10 +13568,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter910;
-    for (_iter910 = (*(this->part_vals)).begin(); _iter910 != (*(this->part_vals)).end(); ++_iter910)
+    std::vector<std::string> ::const_iterator _iter980;
+    for (_iter980 = (*(this->part_vals)).begin(); _iter980 != (*(this->part_vals)).end(); ++_iter980)
     {
-      xfer += oprot->writeString((*_iter910));
+      xfer += oprot->writeString((*_iter980));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13616,14 +13616,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size911;
-            ::apache::thrift::protocol::TType _etype914;
-            xfer += iprot->readListBegin(_etype914, _size911);
-            this->success.resize(_size911);
-            uint32_t _i915;
-            for (_i915 = 0; _i915 < _size911; ++_i915)
+            uint32_t _size981;
+            ::apache::thrift::protocol::TType _etype984;
+            xfer += iprot->readListBegin(_etype984, _size981);
+            this->success.resize(_size981);
+            uint32_t _i985;
+            for (_i985 = 0; _i985 < _size981; ++_i985)
             {
-              xfer += this->success[_i915].read(iprot);
+              xfer += this->success[_i985].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13670,10 +13670,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter916;
-      for (_iter916 = this->success.begin(); _iter916 != this->success.end(); ++_iter916)
+      std::vector<Partition> ::const_iterator _iter986;
+      for (_iter986 = this->success.begin(); _iter986 != this->success.end(); ++_iter986)
       {
-        xfer += (*_iter916).write(oprot);
+        xfer += (*_iter986).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13721,408 +13721,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size917;
-            ::apache::thrift::protocol::TType _etype920;
-            xfer += iprot->readListBegin(_etype920, _size917);
-            (*(this->success)).resize(_size917);
-            uint32_t _i921;
-            for (_i921 = 0; _i921 < _size917; ++_i921)
-            {
-              xfer += (*(this->success))[_i921].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_partitions_ps_with_auth_args::~ThriftHiveMetastore_get_partitions_ps_with_auth_args() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    switch (fid)
-    {
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->db_name);
-          this->__isset.db_name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->tbl_name);
-          this->__isset.tbl_name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->part_vals.clear();
-            uint32_t _size922;
-            ::apache::thrift::protocol::TType _etype925;
-            xfer += iprot->readListBegin(_etype925, _size922);
-            this->part_vals.resize(_size922);
-            uint32_t _i926;
-            for (_i926 = 0; _i926 < _size922; ++_i926)
-            {
-              xfer += iprot->readString(this->part_vals[_i926]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.part_vals = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 4:
-        if (ftype == ::apache::thrift::protocol::T_I16) {
-          xfer += iprot->readI16(this->max_parts);
-          this->__isset.max_parts = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 5:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->user_name);
-          this->__isset.user_name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 6:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->group_names.clear();
-            uint32_t _size927;
-            ::apache::thrift::protocol::TType _etype930;
-            xfer += iprot->readListBegin(_etype930, _size927);
-            this->group_names.resize(_size927);
-            uint32_t _i931;
-            for (_i931 = 0; _i931 < _size927; ++_i931)
-            {
-              xfer += iprot->readString(this->group_names[_i931]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.group_names = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  oprot->incrementRecursionDepth();
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_with_auth_args");
-
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->db_name);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->tbl_name);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
-  {
-    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter932;
-    for (_iter932 = this->part_vals.begin(); _iter932 != this->part_vals.end(); ++_iter932)
-    {
-      xfer += oprot->writeString((*_iter932));
-    }
-    xfer += oprot->writeListEnd();
-  }
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4);
-  xfer += oprot->writeI16(this->max_parts);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 5);
-  xfer += oprot->writeString(this->user_name);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
-  {
-    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter933;
-    for (_iter933 = this->group_names.begin(); _iter933 != this->group_names.end(); ++_iter933)
-    {
-      xfer += oprot->writeString((*_iter933));
-    }
-    xfer += oprot->writeListEnd();
-  }
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  oprot->decrementRecursionDepth();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::~ThriftHiveMetastore_get_partitions_ps_with_auth_pargs() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  oprot->incrementRecursionDepth();
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_with_auth_pargs");
-
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->db_name)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->tbl_name)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
-  {
-    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter934;
-    for (_iter934 = (*(this->part_vals)).begin(); _iter934 != (*(this->part_vals)).end(); ++_iter934)
-    {
-      xfer += oprot->writeString((*_iter934));
-    }
-    xfer += oprot->writeListEnd();
-  }
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4);
-  xfer += oprot->writeI16((*(this->max_parts)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 5);
-  xfer += oprot->writeString((*(this->user_name)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
-  {
-    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter935;
-    for (_iter935 = (*(this->group_names)).begin(); _iter935 != (*(this->group_names)).end(); ++_iter935)
-    {
-      xfer += oprot->writeString((*_iter935));
-    }
-    xfer += oprot->writeListEnd();
-  }
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  oprot->decrementRecursionDepth();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_partitions_ps_with_auth_result::~ThriftHiveMetastore_get_partitions_ps_with_auth_result() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    switch (fid)
-    {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size936;
-            ::apache::thrift::protocol::TType _etype939;
-            xfer += iprot->readListBegin(_etype939, _size936);
-            this->success.resize(_size936);
-            uint32_t _i940;
-            for (_i940 = 0; _i940 < _size936; ++_i940)
-            {
-              xfer += this->success[_i940].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-  

<TRUNCATED>

[37/52] [abbrv] hive git commit: HIVE-11633 : import tool should print help by default (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-11633 : import tool should print help by default (Sergey Shelukhin, reviewed by Alan Gates)


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

Branch: refs/heads/llap
Commit: 4d66206d89f57849f08ff1d4fc7e3a48f61b6275
Parents: 129bed5
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Aug 25 11:03:53 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Aug 25 11:03:53 2015 -0700

----------------------------------------------------------------------
 .../hive/metastore/hbase/HBaseImport.java       | 38 +++++++++++++++-----
 1 file changed, 30 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4d66206d/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java
index e143de7..fac8e90 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseImport.java
@@ -75,14 +75,18 @@ public class HBaseImport {
 
   static final private Log LOG = LogFactory.getLog(HBaseImport.class.getName());
 
-  public static void main(String[] args) {
+  public static int main(String[] args) {
     try {
-      HBaseImport tool = new HBaseImport(args);
+      HBaseImport tool = new HBaseImport();
+      int rv = tool.init(args);
+      if (rv != 0) return rv;
       tool.run();
     } catch (Exception e) {
       System.err.println("Caught exception " + e.getClass().getName() + " with message <" +
           e.getMessage() + ">");
+      return 1;
     }
+    return 0;
   }
 
   private ThreadLocal<RawStore> rdbmsStore = new ThreadLocal<RawStore>() {
@@ -121,8 +125,14 @@ public class HBaseImport {
   private int parallel;
   private int batchSize;
 
+  private HBaseImport() {}
+
   @VisibleForTesting
-  HBaseImport(String... args) throws ParseException {
+  public HBaseImport(String... args) throws ParseException {
+    init(args);
+  }
+
+  private int init(String... args) throws ParseException {
     Options options = new Options();
 
     doAll = doKerberos = false;
@@ -185,38 +195,45 @@ public class HBaseImport {
 
     // Process help, if it was asked for, this must be done first
     if (cli.hasOption('h')) {
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("hbaseschematool", options);
-      // returning here results in nothing else happening, because none of the other flags have
-      // been set.
-      return;
+      printHelp(options);
+      return 1;
     }
 
+    boolean hasCmd = false;
     // Now process the other command line args
     if (cli.hasOption('a')) {
+      hasCmd = true;
       doAll = true;
     }
     if (cli.hasOption('b')) {
       batchSize = Integer.valueOf(cli.getOptionValue('b'));
     }
     if (cli.hasOption('d')) {
+      hasCmd = true;
       dbsToImport = Arrays.asList(cli.getOptionValues('d'));
     }
     if (cli.hasOption('f')) {
+      hasCmd = true;
       functionsToImport = Arrays.asList(cli.getOptionValues('f'));
     }
     if (cli.hasOption('p')) {
       parallel = Integer.valueOf(cli.getOptionValue('p'));
     }
     if (cli.hasOption('r')) {
+      hasCmd = true;
       rolesToImport = Arrays.asList(cli.getOptionValues('r'));
     }
     if (cli.hasOption('k')) {
       doKerberos = true;
     }
     if (cli.hasOption('t')) {
+      hasCmd = true;
       tablesToImport = Arrays.asList(cli.getOptionValues('t'));
     }
+    if (!hasCmd) {
+      printHelp(options);
+      return 1;
+    }
 
     dbs = new ArrayList<>();
     // We don't want to bound the size of the table queue because we keep it all in memory
@@ -225,6 +242,11 @@ public class HBaseImport {
 
     // Bound the size of this queue so we don't get too much in memory.
     partQueue = new ArrayBlockingQueue<>(parallel * 2);
+    return 0;
+  }
+
+  private void printHelp(Options options) {
+    (new HelpFormatter()).printHelp("hbaseschematool", options);
   }
 
   @VisibleForTesting


[31/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java
index c538b72..8355cee 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GrantRevokeRoleRequest implements org.apache.thrift.TBase<GrantRevokeRoleRequest, GrantRevokeRoleRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GrantRevokeRoleRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GrantRevokeRoleRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java
index dd7ee80..f360916 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokeRoleResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GrantRevokeRoleResponse implements org.apache.thrift.TBase<GrantRevokeRoleResponse, GrantRevokeRoleResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GrantRevokeRoleResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GrantRevokeRoleResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java
index f0e2a60..44c7958 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class HeartbeatRequest implements org.apache.thrift.TBase<HeartbeatRequest, HeartbeatRequest._Fields>, java.io.Serializable, Cloneable, Comparable<HeartbeatRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HeartbeatRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java
index de4fe40..bae4cda 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class HeartbeatTxnRangeRequest implements org.apache.thrift.TBase<HeartbeatTxnRangeRequest, HeartbeatTxnRangeRequest._Fields>, java.io.Serializable, Cloneable, Comparable<HeartbeatTxnRangeRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HeartbeatTxnRangeRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index c8d2314..54b0e93 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<HeartbeatTxnRangeResponse, HeartbeatTxnRangeResponse._Fields>, java.io.Serializable, Cloneable, Comparable<HeartbeatTxnRangeResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HeartbeatTxnRangeResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java
index 58fb744..009bd55 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectPrivilege.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class HiveObjectPrivilege implements org.apache.thrift.TBase<HiveObjectPrivilege, HiveObjectPrivilege._Fields>, java.io.Serializable, Cloneable, Comparable<HiveObjectPrivilege> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HiveObjectPrivilege");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
index 13a97ab..9d581d2 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class HiveObjectRef implements org.apache.thrift.TBase<HiveObjectRef, HiveObjectRef._Fields>, java.io.Serializable, Cloneable, Comparable<HiveObjectRef> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HiveObjectRef");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
index ea997ce..98d0f22 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Index implements org.apache.thrift.TBase<Index, Index._Fields>, java.io.Serializable, Cloneable, Comparable<Index> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Index");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java
index a4bed6c..9c03813 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/IndexAlreadyExistsException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class IndexAlreadyExistsException extends TException implements org.apache.thrift.TBase<IndexAlreadyExistsException, IndexAlreadyExistsException._Fields>, java.io.Serializable, Cloneable, Comparable<IndexAlreadyExistsException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IndexAlreadyExistsException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index 5c07c0b..3dc80f1 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEventRequestData, InsertEventRequestData._Fields>, java.io.Serializable, Cloneable, Comparable<InsertEventRequestData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InsertEventRequestData");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java
index 5992099..e169271 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidInputException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class InvalidInputException extends TException implements org.apache.thrift.TBase<InvalidInputException, InvalidInputException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidInputException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidInputException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java
index 60e9b39..47c16e8 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidObjectException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class InvalidObjectException extends TException implements org.apache.thrift.TBase<InvalidObjectException, InvalidObjectException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidObjectException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidObjectException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java
index e565e1e..969fd8a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidOperationException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class InvalidOperationException extends TException implements org.apache.thrift.TBase<InvalidOperationException, InvalidOperationException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidOperationException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidOperationException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java
index 00ff5ee..8a6db46 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InvalidPartitionException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class InvalidPartitionException extends TException implements org.apache.thrift.TBase<InvalidPartitionException, InvalidPartitionException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidPartitionException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidPartitionException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
index 106adaf..dfccfb5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class LockComponent implements org.apache.thrift.TBase<LockComponent, LockComponent._Fields>, java.io.Serializable, Cloneable, Comparable<LockComponent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockComponent");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index d319d0c..f3596db 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockRequest._Fields>, java.io.Serializable, Cloneable, Comparable<LockRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java
index 0f4e115..168e8bc 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class LockResponse implements org.apache.thrift.TBase<LockResponse, LockResponse._Fields>, java.io.Serializable, Cloneable, Comparable<LockResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java
index 3df92ab..96ed366 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LongColumnStatsData.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class LongColumnStatsData implements org.apache.thrift.TBase<LongColumnStatsData, LongColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<LongColumnStatsData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LongColumnStatsData");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
index ab00b5d..04a942a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class MetaException extends TException implements org.apache.thrift.TBase<MetaException, MetaException._Fields>, java.io.Serializable, Cloneable, Comparable<MetaException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MetaException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetadataPpdResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetadataPpdResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetadataPpdResult.java
new file mode 100644
index 0000000..cfae60d
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MetadataPpdResult.java
@@ -0,0 +1,508 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class MetadataPpdResult implements org.apache.thrift.TBase<MetadataPpdResult, MetadataPpdResult._Fields>, java.io.Serializable, Cloneable, Comparable<MetadataPpdResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MetadataPpdResult");
+
+  private static final org.apache.thrift.protocol.TField METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("metadata", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField INCLUDE_BITSET_FIELD_DESC = new org.apache.thrift.protocol.TField("includeBitset", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new MetadataPpdResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new MetadataPpdResultTupleSchemeFactory());
+  }
+
+  private ByteBuffer metadata; // required
+  private ByteBuffer includeBitset; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    METADATA((short)1, "metadata"),
+    INCLUDE_BITSET((short)2, "includeBitset");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // METADATA
+          return METADATA;
+        case 2: // INCLUDE_BITSET
+          return INCLUDE_BITSET;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.METADATA, new org.apache.thrift.meta_data.FieldMetaData("metadata", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.INCLUDE_BITSET, new org.apache.thrift.meta_data.FieldMetaData("includeBitset", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MetadataPpdResult.class, metaDataMap);
+  }
+
+  public MetadataPpdResult() {
+  }
+
+  public MetadataPpdResult(
+    ByteBuffer metadata,
+    ByteBuffer includeBitset)
+  {
+    this();
+    this.metadata = org.apache.thrift.TBaseHelper.copyBinary(metadata);
+    this.includeBitset = org.apache.thrift.TBaseHelper.copyBinary(includeBitset);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public MetadataPpdResult(MetadataPpdResult other) {
+    if (other.isSetMetadata()) {
+      this.metadata = org.apache.thrift.TBaseHelper.copyBinary(other.metadata);
+    }
+    if (other.isSetIncludeBitset()) {
+      this.includeBitset = org.apache.thrift.TBaseHelper.copyBinary(other.includeBitset);
+    }
+  }
+
+  public MetadataPpdResult deepCopy() {
+    return new MetadataPpdResult(this);
+  }
+
+  @Override
+  public void clear() {
+    this.metadata = null;
+    this.includeBitset = null;
+  }
+
+  public byte[] getMetadata() {
+    setMetadata(org.apache.thrift.TBaseHelper.rightSize(metadata));
+    return metadata == null ? null : metadata.array();
+  }
+
+  public ByteBuffer bufferForMetadata() {
+    return org.apache.thrift.TBaseHelper.copyBinary(metadata);
+  }
+
+  public void setMetadata(byte[] metadata) {
+    this.metadata = metadata == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(metadata, metadata.length));
+  }
+
+  public void setMetadata(ByteBuffer metadata) {
+    this.metadata = org.apache.thrift.TBaseHelper.copyBinary(metadata);
+  }
+
+  public void unsetMetadata() {
+    this.metadata = null;
+  }
+
+  /** Returns true if field metadata is set (has been assigned a value) and false otherwise */
+  public boolean isSetMetadata() {
+    return this.metadata != null;
+  }
+
+  public void setMetadataIsSet(boolean value) {
+    if (!value) {
+      this.metadata = null;
+    }
+  }
+
+  public byte[] getIncludeBitset() {
+    setIncludeBitset(org.apache.thrift.TBaseHelper.rightSize(includeBitset));
+    return includeBitset == null ? null : includeBitset.array();
+  }
+
+  public ByteBuffer bufferForIncludeBitset() {
+    return org.apache.thrift.TBaseHelper.copyBinary(includeBitset);
+  }
+
+  public void setIncludeBitset(byte[] includeBitset) {
+    this.includeBitset = includeBitset == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(includeBitset, includeBitset.length));
+  }
+
+  public void setIncludeBitset(ByteBuffer includeBitset) {
+    this.includeBitset = org.apache.thrift.TBaseHelper.copyBinary(includeBitset);
+  }
+
+  public void unsetIncludeBitset() {
+    this.includeBitset = null;
+  }
+
+  /** Returns true if field includeBitset is set (has been assigned a value) and false otherwise */
+  public boolean isSetIncludeBitset() {
+    return this.includeBitset != null;
+  }
+
+  public void setIncludeBitsetIsSet(boolean value) {
+    if (!value) {
+      this.includeBitset = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case METADATA:
+      if (value == null) {
+        unsetMetadata();
+      } else {
+        setMetadata((ByteBuffer)value);
+      }
+      break;
+
+    case INCLUDE_BITSET:
+      if (value == null) {
+        unsetIncludeBitset();
+      } else {
+        setIncludeBitset((ByteBuffer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case METADATA:
+      return getMetadata();
+
+    case INCLUDE_BITSET:
+      return getIncludeBitset();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case METADATA:
+      return isSetMetadata();
+    case INCLUDE_BITSET:
+      return isSetIncludeBitset();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof MetadataPpdResult)
+      return this.equals((MetadataPpdResult)that);
+    return false;
+  }
+
+  public boolean equals(MetadataPpdResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_metadata = true && this.isSetMetadata();
+    boolean that_present_metadata = true && that.isSetMetadata();
+    if (this_present_metadata || that_present_metadata) {
+      if (!(this_present_metadata && that_present_metadata))
+        return false;
+      if (!this.metadata.equals(that.metadata))
+        return false;
+    }
+
+    boolean this_present_includeBitset = true && this.isSetIncludeBitset();
+    boolean that_present_includeBitset = true && that.isSetIncludeBitset();
+    if (this_present_includeBitset || that_present_includeBitset) {
+      if (!(this_present_includeBitset && that_present_includeBitset))
+        return false;
+      if (!this.includeBitset.equals(that.includeBitset))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_metadata = true && (isSetMetadata());
+    list.add(present_metadata);
+    if (present_metadata)
+      list.add(metadata);
+
+    boolean present_includeBitset = true && (isSetIncludeBitset());
+    list.add(present_includeBitset);
+    if (present_includeBitset)
+      list.add(includeBitset);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(MetadataPpdResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetMetadata()).compareTo(other.isSetMetadata());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMetadata()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metadata, other.metadata);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIncludeBitset()).compareTo(other.isSetIncludeBitset());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIncludeBitset()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.includeBitset, other.includeBitset);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("MetadataPpdResult(");
+    boolean first = true;
+
+    sb.append("metadata:");
+    if (this.metadata == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.metadata, sb);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("includeBitset:");
+    if (this.includeBitset == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.includeBitset, sb);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetMetadata()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'metadata' is unset! Struct:" + toString());
+    }
+
+    if (!isSetIncludeBitset()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'includeBitset' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class MetadataPpdResultStandardSchemeFactory implements SchemeFactory {
+    public MetadataPpdResultStandardScheme getScheme() {
+      return new MetadataPpdResultStandardScheme();
+    }
+  }
+
+  private static class MetadataPpdResultStandardScheme extends StandardScheme<MetadataPpdResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, MetadataPpdResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // METADATA
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.metadata = iprot.readBinary();
+              struct.setMetadataIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // INCLUDE_BITSET
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.includeBitset = iprot.readBinary();
+              struct.setIncludeBitsetIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, MetadataPpdResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.metadata != null) {
+        oprot.writeFieldBegin(METADATA_FIELD_DESC);
+        oprot.writeBinary(struct.metadata);
+        oprot.writeFieldEnd();
+      }
+      if (struct.includeBitset != null) {
+        oprot.writeFieldBegin(INCLUDE_BITSET_FIELD_DESC);
+        oprot.writeBinary(struct.includeBitset);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class MetadataPpdResultTupleSchemeFactory implements SchemeFactory {
+    public MetadataPpdResultTupleScheme getScheme() {
+      return new MetadataPpdResultTupleScheme();
+    }
+  }
+
+  private static class MetadataPpdResultTupleScheme extends TupleScheme<MetadataPpdResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, MetadataPpdResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeBinary(struct.metadata);
+      oprot.writeBinary(struct.includeBitset);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, MetadataPpdResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.metadata = iprot.readBinary();
+      struct.setMetadataIsSet(true);
+      struct.includeBitset = iprot.readBinary();
+      struct.setIncludeBitsetIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java
index 501e857..b6f4fd4 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class NoSuchLockException extends TException implements org.apache.thrift.TBase<NoSuchLockException, NoSuchLockException._Fields>, java.io.Serializable, Cloneable, Comparable<NoSuchLockException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NoSuchLockException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java
index e8cb821..f4ebee8 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class NoSuchObjectException extends TException implements org.apache.thrift.TBase<NoSuchObjectException, NoSuchObjectException._Fields>, java.io.Serializable, Cloneable, Comparable<NoSuchObjectException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NoSuchObjectException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java
index 9997b93..687e750 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class NoSuchTxnException extends TException implements org.apache.thrift.TBase<NoSuchTxnException, NoSuchTxnException._Fields>, java.io.Serializable, Cloneable, Comparable<NoSuchTxnException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NoSuchTxnException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
index 6f594c5..b7b1a87 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class NotificationEvent implements org.apache.thrift.TBase<NotificationEvent, NotificationEvent._Fields>, java.io.Serializable, Cloneable, Comparable<NotificationEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotificationEvent");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
index 0c6dc01..2c02b6b 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class NotificationEventRequest implements org.apache.thrift.TBase<NotificationEventRequest, NotificationEventRequest._Fields>, java.io.Serializable, Cloneable, Comparable<NotificationEventRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotificationEventRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index 3295c3c..ff79fc9 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class NotificationEventResponse implements org.apache.thrift.TBase<NotificationEventResponse, NotificationEventResponse._Fields>, java.io.Serializable, Cloneable, Comparable<NotificationEventResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotificationEventResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
index a09575d..ff8d200 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class OpenTxnRequest implements org.apache.thrift.TBase<OpenTxnRequest, OpenTxnRequest._Fields>, java.io.Serializable, Cloneable, Comparable<OpenTxnRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenTxnRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
index d874654..bf1f310 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsResponse, OpenTxnsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<OpenTxnsResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenTxnsResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
index d83b83d..7f57e7d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, java.io.Serializable, Cloneable, Comparable<Order> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Order");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
index e38798a..1e473d6 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Partition implements org.apache.thrift.TBase<Partition, Partition._Fields>, java.io.Serializable, Cloneable, Comparable<Partition> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Partition");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
index 6ccebb3..d765cd6 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PartitionListComposingSpec implements org.apache.thrift.TBase<PartitionListComposingSpec, PartitionListComposingSpec._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionListComposingSpec> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionListComposingSpec");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
index 8d2f1b4..99eaa4a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PartitionSpec implements org.apache.thrift.TBase<PartitionSpec, PartitionSpec._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionSpec> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionSpec");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
index 08d8548..131967b 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PartitionSpecWithSharedSD implements org.apache.thrift.TBase<PartitionSpecWithSharedSD, PartitionSpecWithSharedSD._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionSpecWithSharedSD> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionSpecWithSharedSD");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
index 57ff72e..ca6dff2 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PartitionWithoutSD implements org.apache.thrift.TBase<PartitionWithoutSD, PartitionWithoutSD._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionWithoutSD> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionWithoutSD");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
index b10f3c8..08b1439 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PartitionsByExprRequest implements org.apache.thrift.TBase<PartitionsByExprRequest, PartitionsByExprRequest._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionsByExprRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionsByExprRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
index 3a0376d..de09261 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PartitionsByExprResult implements org.apache.thrift.TBase<PartitionsByExprResult, PartitionsByExprResult._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionsByExprResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionsByExprResult");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
index bfa77f6..8359883 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PartitionsStatsRequest implements org.apache.thrift.TBase<PartitionsStatsRequest, PartitionsStatsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionsStatsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionsStatsRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
index 757f209..a020261 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PartitionsStatsResult implements org.apache.thrift.TBase<PartitionsStatsResult, PartitionsStatsResult._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionsStatsResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionsStatsResult");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java
index 889a41c..7fa2bee 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PrincipalPrivilegeSet implements org.apache.thrift.TBase<PrincipalPrivilegeSet, PrincipalPrivilegeSet._Fields>, java.io.Serializable, Cloneable, Comparable<PrincipalPrivilegeSet> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PrincipalPrivilegeSet");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java
index 741ace7..37149f1 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PrivilegeBag implements org.apache.thrift.TBase<PrivilegeBag, PrivilegeBag._Fields>, java.io.Serializable, Cloneable, Comparable<PrivilegeBag> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PrivilegeBag");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java
index ba52582..22471c7 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class PrivilegeGrantInfo implements org.apache.thrift.TBase<PrivilegeGrantInfo, PrivilegeGrantInfo._Fields>, java.io.Serializable, Cloneable, Comparable<PrivilegeGrantInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PrivilegeGrantInfo");
 


[05/52] [abbrv] hive git commit: HIVE-11343 Merge branch 'master' into hbase-metastore

Posted by se...@apache.org.
HIVE-11343 Merge branch 'master' into hbase-metastore

Conflicts:
	metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
	pom.xml


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

Branch: refs/heads/llap
Commit: 61db7b80cbefa5761afd904dfef2eb25e759d6f4
Parents: a310524 2240dbd
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jul 22 13:45:28 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jul 22 13:45:28 2015 -0700

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 NOTICE                                          |    2 +-
 RELEASE_NOTES.txt                               |  132 +
 accumulo-handler/pom.xml                        |    2 +-
 .../accumulo/AccumuloConnectionParameters.java  |  111 +-
 .../hive/accumulo/AccumuloStorageHandler.java   |   50 +
 .../hive/accumulo/HiveAccumuloHelper.java       |  280 +
 .../mr/HiveAccumuloTableInputFormat.java        |   74 +-
 .../mr/HiveAccumuloTableOutputFormat.java       |   63 +-
 .../TestAccumuloConnectionParameters.java       |   19 +
 .../hive/accumulo/TestHiveAccumuloHelper.java   |   75 +
 .../mr/TestHiveAccumuloTableInputFormat.java    |    8 +-
 .../mr/TestHiveAccumuloTableOutputFormat.java   |    2 +-
 .../positive/accumulo_predicate_pushdown.q.out  |   76 +-
 .../results/positive/accumulo_queries.q.out     |   70 +-
 ant/pom.xml                                     |    2 +-
 beeline/pom.xml                                 |    2 +-
 .../java/org/apache/hive/beeline/BeeLine.java   |    7 +-
 .../org/apache/hive/beeline/BeeLineOpts.java    |   13 +-
 .../java/org/apache/hive/beeline/Commands.java  |   23 +-
 .../apache/hive/beeline/DatabaseConnection.java |    9 +
 beeline/src/main/resources/BeeLine.properties   |    2 +
 bin/ext/hiveserver2.cmd                         |   51 +-
 bin/ext/hplsql.sh                               |   37 +
 bin/hplsql                                      |   25 +
 bin/hplsql.cmd                                  |   58 +
 cli/pom.xml                                     |    2 +-
 common/pom.xml                                  |    2 +-
 .../apache/hadoop/hive/common/FileUtils.java    |    9 +-
 .../hadoop/hive/common/JvmPauseMonitor.java     |   19 +-
 .../hive/common/jsonexplain/tez/Attr.java       |    6 +-
 .../hive/common/jsonexplain/tez/Connection.java |    6 +-
 .../hadoop/hive/common/jsonexplain/tez/Op.java  |   64 +-
 .../hive/common/jsonexplain/tez/Printer.java    |   41 +
 .../hive/common/jsonexplain/tez/Stage.java      |   95 +-
 .../common/jsonexplain/tez/TezJsonParser.java   |   61 +-
 .../hive/common/jsonexplain/tez/Vertex.java     |   75 +-
 .../hive/common/metrics/LegacyMetrics.java      |   81 +-
 .../hive/common/metrics/common/Metrics.java     |   35 +-
 .../common/metrics/common/MetricsConstant.java  |   35 +
 .../common/metrics/common/MetricsFactory.java   |   30 +-
 .../common/metrics/common/MetricsVariable.java  |   26 +
 .../metrics/metrics2/CodahaleMetrics.java       |   99 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   40 +-
 .../hive/conf/LoopingByteArrayInputStream.java  |    2 +-
 .../hive/common/metrics/TestLegacyMetrics.java  |    6 +-
 .../metrics/metrics2/TestCodahaleMetrics.java   |   58 +-
 contrib/pom.xml                                 |    2 +-
 .../util/typedbytes/TypedBytesInput.java        |    2 +-
 .../util/typedbytes/TypedBytesOutput.java       |    2 +-
 .../util/typedbytes/TypedBytesRecordInput.java  |    2 +-
 .../util/typedbytes/TypedBytesRecordOutput.java |    2 +-
 .../typedbytes/TypedBytesWritableInput.java     |    2 +-
 .../typedbytes/TypedBytesWritableOutput.java    |    2 +-
 data/conf/spark/standalone/hive-site.xml        |    6 +
 data/conf/spark/yarn-client/hive-site.xml       |    6 +
 data/files/ct_events_clean.txt                  |   76 +
 data/files/emp2.txt                             |   16 +
 data/files/encoding-utf8.txt                    |   12 +
 data/files/encoding_iso-8859-1.txt              |    4 +
 data/files/service_request_clean.txt            |   76 +
 data/files/sortdp.txt                           |   32 +
 data/scripts/q_test_cleanup.sql                 |    2 +-
 dev-support/jenkins-submit-build.sh             |   10 +-
 hbase-handler/pom.xml                           |    2 +-
 .../results/positive/external_table_ppd.q.out   |    1 -
 .../positive/hbase_binary_storage_queries.q.out |    2 -
 .../src/test/results/positive/hbase_stats.q.out |    7 -
 .../test/results/positive/hbase_stats2.q.out    |    7 -
 .../test/results/positive/hbase_stats3.q.out    |   12 -
 .../positive/hbase_stats_empty_partition.q.out  |    2 -
 hcatalog/core/pom.xml                           |    2 +-
 .../SemanticAnalysis/HCatSemanticAnalyzer.java  |    7 +-
 hcatalog/hcatalog-pig-adapter/pom.xml           |    2 +-
 .../hcatalog/pig/TestHCatLoaderEncryption.java  |   18 +-
 hcatalog/pom.xml                                |    2 +-
 hcatalog/server-extensions/pom.xml              |    9 +-
 .../src/test/e2e/templeton/deployers/env.sh     |    5 +-
 hcatalog/streaming/pom.xml                      |    8 +-
 .../streaming/AbstractRecordWriter.java         |    4 +-
 .../streaming/mutate/HiveConfFactory.java       |   63 +
 .../mutate/UgiMetaStoreClientFactory.java       |  102 +
 .../streaming/mutate/client/AcidTable.java      |  112 +
 .../mutate/client/AcidTableSerializer.java      |  100 +
 .../mutate/client/ClientException.java          |   15 +
 .../mutate/client/ConnectionException.java      |   15 +
 .../streaming/mutate/client/MutatorClient.java  |  149 +
 .../mutate/client/MutatorClientBuilder.java     |  115 +
 .../streaming/mutate/client/TableType.java      |   37 +
 .../streaming/mutate/client/Transaction.java    |  114 +
 .../mutate/client/TransactionException.java     |   15 +
 .../mutate/client/lock/HeartbeatFactory.java    |   30 +
 .../mutate/client/lock/HeartbeatTimerTask.java  |   66 +
 .../streaming/mutate/client/lock/Lock.java      |  305 +
 .../mutate/client/lock/LockException.java       |   15 +
 .../mutate/client/lock/LockFailureListener.java |   26 +
 .../mutate/doc-files/system-overview.dot        |   27 +
 .../hive/hcatalog/streaming/mutate/package.html |  520 ++
 .../mutate/worker/BucketIdException.java        |   11 +
 .../mutate/worker/BucketIdResolver.java         |   11 +
 .../mutate/worker/BucketIdResolverImpl.java     |   76 +
 .../mutate/worker/GroupRevisitedException.java  |   11 +
 .../mutate/worker/GroupingValidator.java        |   74 +
 .../mutate/worker/MetaStorePartitionHelper.java |  102 +
 .../streaming/mutate/worker/Mutator.java        |   21 +
 .../mutate/worker/MutatorCoordinator.java       |  278 +
 .../worker/MutatorCoordinatorBuilder.java       |  101 +
 .../streaming/mutate/worker/MutatorFactory.java |   16 +
 .../streaming/mutate/worker/MutatorImpl.java    |   86 +
 .../streaming/mutate/worker/OperationType.java  |    7 +
 .../worker/PartitionCreationException.java      |   15 +
 .../mutate/worker/PartitionHelper.java          |   17 +
 .../mutate/worker/RecordInspector.java          |   11 +
 .../mutate/worker/RecordInspectorImpl.java      |   45 +
 .../mutate/worker/RecordSequenceException.java  |   11 +
 .../mutate/worker/SequenceValidator.java        |   49 +
 .../mutate/worker/WarehousePartitionHelper.java |   69 +
 .../mutate/worker/WorkerException.java          |   15 +
 .../streaming/mutate/ExampleUseCase.java        |   82 +
 .../streaming/mutate/MutableRecord.java         |   50 +
 .../mutate/ReflectiveMutatorFactory.java        |   51 +
 .../streaming/mutate/StreamingAssert.java       |  191 +
 .../streaming/mutate/StreamingTestUtils.java    |  261 +
 .../streaming/mutate/TestMutations.java         |  544 ++
 .../mutate/client/TestAcidTableSerializer.java  |   66 +
 .../mutate/client/TestMutatorClient.java        |  176 +
 .../mutate/client/TestTransaction.java          |   95 +
 .../client/lock/TestHeartbeatTimerTask.java     |  100 +
 .../streaming/mutate/client/lock/TestLock.java  |  310 +
 .../mutate/worker/TestBucketIdResolverImpl.java |   38 +
 .../mutate/worker/TestGroupingValidator.java    |   70 +
 .../worker/TestMetaStorePartitionHelper.java    |  112 +
 .../mutate/worker/TestMutatorCoordinator.java   |  244 +
 .../mutate/worker/TestMutatorImpl.java          |   99 +
 .../mutate/worker/TestRecordInspectorImpl.java  |   31 +
 .../mutate/worker/TestSequenceValidator.java    |   91 +
 .../worker/TestWarehousePartitionHelper.java    |   57 +
 hcatalog/webhcat/java-client/pom.xml            |    2 +-
 .../hive/hcatalog/api/HCatClientHMSImpl.java    |   14 +-
 hcatalog/webhcat/svr/pom.xml                    |    2 +-
 .../hive/hcatalog/templeton/AppConfig.java      |   30 +-
 .../hive/hcatalog/templeton/HiveDelegator.java  |   15 +-
 .../hive/hcatalog/templeton/JarDelegator.java   |    8 +-
 .../hcatalog/templeton/LauncherDelegator.java   |   14 +
 .../hive/hcatalog/templeton/PigDelegator.java   |   13 +-
 .../hcatalog/templeton/SecureProxySupport.java  |    6 +-
 .../apache/hive/hcatalog/templeton/Server.java  |   34 +-
 .../hive/hcatalog/templeton/SqoopDelegator.java |   20 +-
 .../hcatalog/templeton/StreamingDelegator.java  |    3 +-
 .../templeton/tool/JobSubmissionConstants.java  |    2 +
 .../hcatalog/templeton/tool/LaunchMapper.java   |  214 +-
 .../templeton/tool/TempletonControllerJob.java  |    7 +-
 hplsql/pom.xml                                  |  128 +
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     | 1452 ++++
 .../java/org/apache/hive/hplsql/Arguments.java  |  206 +
 .../main/java/org/apache/hive/hplsql/Conf.java  |  175 +
 .../main/java/org/apache/hive/hplsql/Conn.java  |  245 +
 .../java/org/apache/hive/hplsql/Converter.java  |   56 +
 .../main/java/org/apache/hive/hplsql/Copy.java  |  426 ++
 .../main/java/org/apache/hive/hplsql/Exec.java  | 2044 ++++++
 .../java/org/apache/hive/hplsql/Expression.java |  574 ++
 .../main/java/org/apache/hive/hplsql/File.java  |  132 +
 .../java/org/apache/hive/hplsql/Handler.java    |   41 +
 .../java/org/apache/hive/hplsql/Hplsql.java     |   25 +
 .../java/org/apache/hive/hplsql/Interval.java   |  109 +
 .../main/java/org/apache/hive/hplsql/Query.java |  171 +
 .../main/java/org/apache/hive/hplsql/Scope.java |   69 +
 .../java/org/apache/hive/hplsql/Select.java     |  411 ++
 .../java/org/apache/hive/hplsql/Signal.java     |   48 +
 .../main/java/org/apache/hive/hplsql/Stmt.java  | 1082 +++
 .../org/apache/hive/hplsql/StreamGobbler.java   |   51 +
 .../main/java/org/apache/hive/hplsql/Timer.java |   59 +
 .../main/java/org/apache/hive/hplsql/Udf.java   |  117 +
 .../main/java/org/apache/hive/hplsql/Utils.java |  296 +
 .../main/java/org/apache/hive/hplsql/Var.java   |  437 ++
 .../apache/hive/hplsql/functions/Function.java  |  717 ++
 .../hive/hplsql/functions/FunctionDatetime.java |  151 +
 .../hive/hplsql/functions/FunctionMisc.java     |  188 +
 .../hive/hplsql/functions/FunctionOra.java      |  231 +
 .../hive/hplsql/functions/FunctionString.java   |  276 +
 hplsql/src/main/resources/hplsql-site.xml       |   95 +
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |  306 +
 .../db/create_procedure_return_cursor.sql       |   53 +
 .../db/create_procedure_return_cursor2.sql      |   59 +
 hplsql/src/test/queries/local/add.sql           |    2 +
 hplsql/src/test/queries/local/assign.sql        |    7 +
 hplsql/src/test/queries/local/bool_expr.sql     |   47 +
 hplsql/src/test/queries/local/break.sql         |   10 +
 hplsql/src/test/queries/local/case.sql          |   35 +
 hplsql/src/test/queries/local/cast.sql          |    4 +
 hplsql/src/test/queries/local/char.sql          |    1 +
 hplsql/src/test/queries/local/coalesce.sql      |    4 +
 hplsql/src/test/queries/local/concat.sql        |    2 +
 .../src/test/queries/local/create_function.sql  |   11 +
 .../src/test/queries/local/create_function2.sql |   11 +
 .../src/test/queries/local/create_procedure.sql |    9 +
 hplsql/src/test/queries/local/date.sql          |    5 +
 hplsql/src/test/queries/local/dbms_output.sql   |    6 +
 hplsql/src/test/queries/local/declare.sql       |   16 +
 .../test/queries/local/declare_condition.sql    |    8 +
 .../test/queries/local/declare_condition2.sql   |   10 +
 hplsql/src/test/queries/local/decode.sql        |   10 +
 hplsql/src/test/queries/local/equal.sql         |   55 +
 hplsql/src/test/queries/local/exception.sql     |   14 +
 hplsql/src/test/queries/local/exit.sql          |   31 +
 hplsql/src/test/queries/local/expr.sql          |   21 +
 hplsql/src/test/queries/local/for_range.sql     |   20 +
 hplsql/src/test/queries/local/if.sql            |   68 +
 hplsql/src/test/queries/local/instr.sql         |   49 +
 hplsql/src/test/queries/local/interval.sql      |   15 +
 hplsql/src/test/queries/local/lang.sql          |   57 +
 hplsql/src/test/queries/local/leave.sql         |   33 +
 hplsql/src/test/queries/local/len.sql           |    1 +
 hplsql/src/test/queries/local/length.sql        |    1 +
 hplsql/src/test/queries/local/lower.sql         |    1 +
 hplsql/src/test/queries/local/nvl.sql           |    4 +
 hplsql/src/test/queries/local/nvl2.sql          |    2 +
 hplsql/src/test/queries/local/print.sql         |    5 +
 hplsql/src/test/queries/local/return.sql        |    3 +
 hplsql/src/test/queries/local/seterror.sql      |   10 +
 hplsql/src/test/queries/local/sub.sql           |    1 +
 hplsql/src/test/queries/local/substr.sql        |    2 +
 hplsql/src/test/queries/local/substring.sql     |    8 +
 hplsql/src/test/queries/local/timestamp.sql     |    4 +
 hplsql/src/test/queries/local/timestamp_iso.sql |    2 +
 hplsql/src/test/queries/local/to_char.sql       |    1 +
 hplsql/src/test/queries/local/to_timestamp.sql  |    5 +
 hplsql/src/test/queries/local/trim.sql          |    1 +
 hplsql/src/test/queries/local/twopipes.sql      |    1 +
 hplsql/src/test/queries/local/upper.sql         |    1 +
 hplsql/src/test/queries/local/values_into.sql   |    6 +
 hplsql/src/test/queries/local/while.sql         |   20 +
 .../db/create_procedure_return_cursor.out.txt   |  135 +
 .../db/create_procedure_return_cursor2.out.txt  |  139 +
 hplsql/src/test/results/local/add.out.txt       |    2 +
 hplsql/src/test/results/local/assign.out.txt    |    8 +
 hplsql/src/test/results/local/bool_expr.out.txt |   32 +
 hplsql/src/test/results/local/break.out.txt     |   29 +
 hplsql/src/test/results/local/case.out.txt      |   12 +
 hplsql/src/test/results/local/cast.out.txt      |    8 +
 hplsql/src/test/results/local/char.out.txt      |    1 +
 hplsql/src/test/results/local/coalesce.out.txt  |    4 +
 hplsql/src/test/results/local/concat.out.txt    |    2 +
 .../test/results/local/create_function.out.txt  |    9 +
 .../test/results/local/create_function2.out.txt |   10 +
 .../test/results/local/create_procedure.out.txt |    8 +
 hplsql/src/test/results/local/date.out.txt      |    4 +
 .../src/test/results/local/dbms_output.out.txt  |    3 +
 hplsql/src/test/results/local/declare.out.txt   |   13 +
 .../results/local/declare_condition.out.txt     |    7 +
 .../results/local/declare_condition2.out.txt    |   12 +
 hplsql/src/test/results/local/decode.out.txt    |   13 +
 hplsql/src/test/results/local/equal.out.txt     |   48 +
 hplsql/src/test/results/local/exception.out.txt |   13 +
 .../src/test/results/local/exception2.out.txt   |    5 +
 hplsql/src/test/results/local/exit.out.txt      |   42 +
 hplsql/src/test/results/local/expr.out.txt      |   29 +
 hplsql/src/test/results/local/for_range.out.txt |   65 +
 hplsql/src/test/results/local/if.out.txt        |   40 +
 hplsql/src/test/results/local/instr.out.txt     |   33 +
 hplsql/src/test/results/local/interval.out.txt  |   11 +
 hplsql/src/test/results/local/lang.out.txt      |   34 +
 hplsql/src/test/results/local/leave.out.txt     |   42 +
 hplsql/src/test/results/local/len.out.txt       |    1 +
 hplsql/src/test/results/local/length.out.txt    |    1 +
 hplsql/src/test/results/local/lower.out.txt     |    1 +
 hplsql/src/test/results/local/nvl.out.txt       |    4 +
 hplsql/src/test/results/local/nvl2.out.txt      |    2 +
 .../test/results/local/plhqlexception.out.txt   |    6 +
 .../test/results/local/plhqlexception1.out.txt  |   10 +
 .../test/results/local/plhqlexception2.out.txt  |  106 +
 hplsql/src/test/results/local/print.out.txt     |    6 +
 hplsql/src/test/results/local/return.out.txt    |    3 +
 .../results/local/select_conversion.out.txt     |    9 +
 hplsql/src/test/results/local/seterror.out.txt  |    6 +
 hplsql/src/test/results/local/sub.out.txt       |    1 +
 hplsql/src/test/results/local/substr.out.txt    |    2 +
 hplsql/src/test/results/local/substring.out.txt |    8 +
 hplsql/src/test/results/local/timestamp.out.txt |    4 +
 .../test/results/local/timestamp_iso.out.txt    |    2 +
 hplsql/src/test/results/local/to_char.out.txt   |    1 +
 .../src/test/results/local/to_timestamp.out.txt |    4 +
 hplsql/src/test/results/local/trim.out.txt      |    1 +
 hplsql/src/test/results/local/twopipes.out.txt  |    1 +
 hplsql/src/test/results/local/upper.out.txt     |    1 +
 .../src/test/results/local/values_into.out.txt  |   11 +
 hplsql/src/test/results/local/while.out.txt     |   72 +
 hwi/pom.xml                                     |    2 +-
 itests/custom-serde/pom.xml                     |    2 +-
 itests/hcatalog-unit/pom.xml                    |    2 +-
 itests/hive-jmh/pom.xml                         |    2 +-
 itests/hive-minikdc/pom.xml                     |   22 +-
 itests/hive-unit-hadoop2/pom.xml                |    2 +-
 itests/hive-unit/pom.xml                        |    2 +-
 .../hive/metastore/TestHiveMetaStore.java       |   57 +-
 .../hive/metastore/TestMetaStoreMetrics.java    |   66 +-
 .../hive/ql/txn/compactor/TestCompactor.java    |   11 +-
 .../hive/beeline/TestBeeLineWithArgs.java       |   87 +
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |   16 +
 itests/pom.xml                                  |    3 +-
 itests/qtest-accumulo/pom.xml                   |  508 ++
 .../test/java/org/apache/hive/TestDummy.java    |   31 +
 itests/qtest-spark/pom.xml                      |    2 +-
 itests/qtest/pom.xml                            |   43 +-
 .../test/resources/testconfiguration.properties |   15 +-
 itests/test-serde/pom.xml                       |    2 +-
 itests/util/pom.xml                             |    2 +-
 jdbc/pom.xml                                    |    2 +-
 .../org/apache/hive/jdbc/HiveConnection.java    |   19 +-
 .../apache/hive/jdbc/HivePreparedStatement.java |    2 +-
 metastore/pom.xml                               |    2 +-
 .../upgrade/derby/hive-schema-2.0.0.derby.sql   |  336 +
 .../derby/upgrade-1.2.0-to-2.0.0.derby.sql      |    3 +
 .../scripts/upgrade/derby/upgrade.order.derby   |    2 +-
 .../upgrade/mssql/hive-schema-2.0.0.mssql.sql   |  947 +++
 .../mssql/upgrade-1.2.0-to-2.0.0.mssql.sql      |    5 +
 .../scripts/upgrade/mssql/upgrade.order.mssql   |    2 +-
 .../upgrade/mysql/021-HIVE-7018.mysql.sql       |   53 +
 .../upgrade/mysql/hive-schema-1.3.0.mysql.sql   |   10 +-
 .../upgrade/mysql/hive-schema-2.0.0.mysql.sql   |  834 +++
 .../mysql/upgrade-1.2.0-to-1.3.0.mysql.sql      |    2 +-
 .../mysql/upgrade-1.2.0-to-2.0.0.mysql.sql      |    4 +
 .../scripts/upgrade/mysql/upgrade.order.mysql   |    2 +-
 .../upgrade/oracle/hive-schema-2.0.0.oracle.sql |  788 ++
 .../oracle/upgrade-1.2.0-to-2.0.0.oracle.sql    |    4 +
 .../scripts/upgrade/oracle/upgrade.order.oracle |    2 +-
 .../postgres/hive-schema-2.0.0.postgres.sql     | 1457 ++++
 .../upgrade-1.2.0-to-2.0.0.postgres.sql         |   12 +
 .../upgrade/postgres/upgrade.order.postgres     |    2 +-
 .../hive/metastore/AggregateStatsCache.java     |    5 +-
 .../apache/hadoop/hive/metastore/Deadline.java  |    2 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  241 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   20 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   61 +-
 .../hive/metastore/MetaStoreDirectSql.java      |   72 +-
 .../hive/metastore/MetaStoreSchemaInfo.java     |   14 +-
 .../hadoop/hive/metastore/MetaStoreUtils.java   |   26 -
 .../hadoop/hive/metastore/ObjectStore.java      | 1885 +++--
 .../hive/metastore/PartitionDropOptions.java    |    6 -
 .../hadoop/hive/metastore/ProtectMode.java      |   97 -
 .../hive/metastore/StatObjectConverter.java     |   16 +-
 .../hive/metastore/tools/HiveMetaTool.java      |   23 +-
 .../hive/metastore/txn/CompactionInfo.java      |    9 +
 .../metastore/txn/CompactionTxnHandler.java     |   19 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   36 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |  230 +
 odbc/pom.xml                                    |    2 +-
 packaging/pom.xml                               |    2 +-
 pom.xml                                         |   32 +-
 ql/pom.xml                                      |   14 +-
 .../UDAFTemplates/VectorUDAFMinMaxString.txt    |    3 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   43 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   22 +-
 .../org/apache/hadoop/hive/ql/QueryPlan.java    |    9 +-
 .../apache/hadoop/hive/ql/exec/ColumnInfo.java  |    2 +-
 .../hive/ql/exec/CommonMergeJoinOperator.java   |   76 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  207 +-
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |   55 +-
 .../hadoop/hive/ql/exec/FetchOperator.java      |    5 +-
 .../hadoop/hive/ql/exec/FilterOperator.java     |    3 +-
 .../apache/hadoop/hive/ql/exec/JoinUtil.java    |   87 +-
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |   61 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |    6 +
 .../hive/ql/exec/OrcFileMergeOperator.java      |   70 +-
 .../hive/ql/exec/PartitionKeySampler.java       |    9 +-
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java |    7 +-
 .../apache/hadoop/hive/ql/exec/TaskFactory.java |    2 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |   30 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |   12 +-
 .../hive/ql/exec/mr/ExecMapperContext.java      |   10 +-
 .../persistence/HybridHashTableContainer.java   |   59 +-
 .../persistence/MapJoinBytesTableContainer.java |   75 +-
 .../exec/persistence/MapJoinTableContainer.java |    5 +
 .../ql/exec/spark/SparkReduceRecordHandler.java |   31 +-
 .../hive/ql/exec/tez/HashTableLoader.java       |   81 +-
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |   24 +
 .../hive/ql/exec/tez/KeyValuesAdapter.java      |   47 +
 .../hive/ql/exec/tez/KeyValuesFromKeyValue.java |   90 +
 .../ql/exec/tez/KeyValuesFromKeyValues.java     |   48 +
 .../ql/exec/tez/MergeFileRecordProcessor.java   |   42 +-
 .../hive/ql/exec/tez/ReduceRecordProcessor.java |   11 +-
 .../hive/ql/exec/tez/ReduceRecordSource.java    |   48 +-
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  |    2 +-
 .../hive/ql/exec/tez/TezSessionPoolManager.java |   21 +-
 .../hive/ql/exec/tez/TezSessionState.java       |   10 -
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |   10 +-
 .../hive/ql/exec/vector/BytesColumnVector.java  |   14 +
 .../hive/ql/exec/vector/ColumnVector.java       |    8 +
 .../ql/exec/vector/DecimalColumnVector.java     |   12 +
 .../hive/ql/exec/vector/DoubleColumnVector.java |   12 +
 .../hive/ql/exec/vector/LongColumnVector.java   |   12 +
 .../ql/exec/vector/VectorGroupByOperator.java   |   16 +-
 .../ql/exec/vector/VectorMapJoinOperator.java   |    1 -
 .../ql/exec/vector/VectorSelectOperator.java    |    3 -
 .../ql/exec/vector/VectorizationContext.java    |   14 +-
 .../ql/exec/vector/VectorizedBatchUtil.java     |   41 +-
 .../hive/ql/exec/vector/VectorizedRowBatch.java |   78 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |    2 +-
 .../mapjoin/VectorMapJoinCommonOperator.java    |    1 +
 .../mapjoin/VectorMapJoinRowBytesContainer.java |    9 +-
 .../fast/VectorMapJoinFastHashTable.java        |    5 +
 .../fast/VectorMapJoinFastTableContainer.java   |    5 +
 .../hashtable/VectorMapJoinHashTable.java       |    4 +
 .../VectorMapJoinOptimizedHashTable.java        |    4 +
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   27 +-
 .../hadoop/hive/ql/hooks/HookContext.java       |   12 +
 .../hadoop/hive/ql/hooks/LineageInfo.java       |   96 +
 .../hadoop/hive/ql/hooks/LineageLogger.java     |  439 ++
 .../hadoop/hive/ql/hooks/WriteEntity.java       |    6 +-
 .../hadoop/hive/ql/io/AcidInputFormat.java      |   60 +-
 .../hadoop/hive/ql/io/AcidOutputFormat.java     |   49 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |  152 +-
 .../hadoop/hive/ql/io/FileFormatException.java  |   30 +
 .../ql/io/HiveContextAwareRecordReader.java     |    2 +-
 .../hadoop/hive/ql/io/HiveFileFormatUtils.java  |   19 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   38 +-
 .../org/apache/hadoop/hive/ql/io/IOContext.java |   43 -
 .../apache/hadoop/hive/ql/io/IOContextMap.java  |   81 +
 .../hive/ql/io/orc/ColumnStatisticsImpl.java    |  217 +-
 .../apache/hadoop/hive/ql/io/orc/FileDump.java  |    7 +-
 .../hadoop/hive/ql/io/orc/MemoryManager.java    |   43 +-
 .../apache/hadoop/hive/ql/io/orc/OrcConf.java   |  134 +
 .../apache/hadoop/hive/ql/io/orc/OrcFile.java   |   55 +-
 .../hive/ql/io/orc/OrcFileKeyWrapper.java       |   27 +-
 .../io/orc/OrcFileStripeMergeRecordReader.java  |   23 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   60 +-
 .../hive/ql/io/orc/OrcNewInputFormat.java       |   16 +-
 .../hadoop/hive/ql/io/orc/OrcNewSplit.java      |   13 +-
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |   84 +-
 .../hadoop/hive/ql/io/orc/OrcRecordUpdater.java |   63 +-
 .../apache/hadoop/hive/ql/io/orc/OrcSplit.java  |   16 +-
 .../apache/hadoop/hive/ql/io/orc/OrcUtils.java  |    4 +
 .../apache/hadoop/hive/ql/io/orc/Reader.java    |   34 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   19 +-
 .../hive/ql/io/orc/RecordReaderFactory.java     |    7 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   42 +-
 .../ql/io/orc/RunLengthIntegerReaderV2.java     |    8 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |   96 +-
 .../io/parquet/FilterPredicateLeafBuilder.java  |    8 +-
 .../hive/ql/io/parquet/LeafFilterFactory.java   |   20 +-
 .../ql/io/parquet/MapredParquetInputFormat.java |    3 +-
 .../io/parquet/MapredParquetOutputFormat.java   |    2 +-
 .../parquet/VectorizedParquetInputFormat.java   |    2 +-
 .../convert/DataWritableRecordConverter.java    |    8 +-
 .../ql/io/parquet/convert/ETypeConverter.java   |   10 +-
 .../convert/HiveCollectionConverter.java        |    6 +-
 .../io/parquet/convert/HiveGroupConverter.java  |   14 +-
 .../io/parquet/convert/HiveSchemaConverter.java |   18 +-
 .../io/parquet/convert/HiveStructConverter.java |    6 +-
 .../hive/ql/io/parquet/convert/Repeated.java    |   12 +-
 .../parquet/read/DataWritableReadSupport.java   |   20 +-
 .../read/ParquetRecordReaderWrapper.java        |  133 +-
 .../ql/io/parquet/serde/ParquetHiveSerDe.java   |    4 +-
 .../hive/ql/io/parquet/timestamp/NanoTime.java  |    6 +-
 .../parquet/write/DataWritableWriteSupport.java |    8 +-
 .../ql/io/parquet/write/DataWritableWriter.java |  648 +-
 .../write/ParquetRecordWriterWrapper.java       |    6 +-
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |  343 +-
 .../hadoop/hive/ql/lib/RuleExactMatch.java      |   21 +-
 .../apache/hadoop/hive/ql/lib/RuleRegExp.java   |  191 +-
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |    4 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   22 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |   19 +-
 .../hadoop/hive/ql/lockmgr/HiveLockObject.java  |   37 +-
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |   50 +-
 .../hive/ql/lockmgr/HiveTxnManagerImpl.java     |  140 +
 .../zookeeper/ZooKeeperHiveLockManager.java     |    2 +
 .../ql/log/NoDeleteRollingFileAppender.java     |  176 +
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |    1 -
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   77 +-
 .../hadoop/hive/ql/metadata/HiveUtils.java      |    8 -
 .../hadoop/hive/ql/metadata/Partition.java      |   51 -
 .../apache/hadoop/hive/ql/metadata/Table.java   |   65 +-
 .../formatting/MetaDataFormatUtils.java         |   24 +-
 .../BucketingSortingReduceSinkOptimizer.java    |    4 +
 .../hive/ql/optimizer/ColumnPrunerProcCtx.java  |   27 +
 .../ql/optimizer/ColumnPrunerProcFactory.java   |   49 +-
 .../hive/ql/optimizer/ConstantPropagate.java    |   13 +-
 .../ql/optimizer/ConstantPropagateProcCtx.java  |   31 +-
 .../optimizer/ConstantPropagateProcFactory.java |  169 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |  234 +-
 .../hive/ql/optimizer/MapJoinProcessor.java     |   44 +-
 .../ql/optimizer/NonBlockingOpDeDupProc.java    |    9 -
 .../hadoop/hive/ql/optimizer/Optimizer.java     |    4 +
 .../hadoop/hive/ql/optimizer/PrunerUtils.java   |   14 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |   84 +-
 .../optimizer/RemoveDynamicPruningBySize.java   |    2 +-
 .../ql/optimizer/calcite/HiveCalciteUtil.java   |  148 +-
 .../ql/optimizer/calcite/HiveRelOptUtil.java    |  316 +
 .../calcite/cost/HiveOnTezCostModel.java        |   25 +-
 .../calcite/reloperators/HiveJoin.java          |   25 +-
 .../calcite/reloperators/HiveMultiJoin.java     |  221 +
 .../calcite/reloperators/HiveSemiJoin.java      |   57 +-
 .../rules/HiveInsertExchange4JoinRule.java      |   40 +-
 .../calcite/rules/HiveJoinAddNotNullRule.java   |   16 +-
 .../calcite/rules/HiveJoinCommuteRule.java      |   96 +
 .../rules/HiveJoinProjectTransposeRule.java     |  283 +
 .../HiveJoinPushTransitivePredicatesRule.java   |  139 +
 .../calcite/rules/HiveJoinToMultiJoinRule.java  |  387 +-
 .../calcite/rules/HiveProjectMergeRule.java     |    1 -
 .../calcite/rules/HiveRelFieldTrimmer.java      |  107 +
 .../calcite/stats/HiveRelMdSelectivity.java     |   11 +-
 .../calcite/translator/ASTConverter.java        |   20 +-
 .../calcite/translator/ExprNodeConverter.java   |   41 +-
 .../calcite/translator/HiveOpConverter.java     |  363 +-
 .../translator/HiveOpConverterPostProc.java     |   56 +-
 .../calcite/translator/JoinTypeCheckCtx.java    |    2 +-
 .../translator/PlanModifierForASTConv.java      |   12 +-
 .../correlation/ReduceSinkDeDuplication.java    |    3 +
 .../ql/optimizer/lineage/ExprProcFactory.java   |   98 +
 .../hive/ql/optimizer/lineage/Generator.java    |   16 +-
 .../hive/ql/optimizer/lineage/LineageCtx.java   |   79 +-
 .../ql/optimizer/lineage/OpProcFactory.java     |  228 +-
 .../BucketingSortingInferenceOptimizer.java     |    8 +-
 .../ql/optimizer/physical/MemoryDecider.java    |  288 +
 .../ql/optimizer/physical/SerializeFilter.java  |  178 +
 .../hive/ql/optimizer/physical/Vectorizer.java  |  147 +-
 .../stats/annotation/StatsRulesProcFactory.java |   42 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   95 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  112 +-
 .../apache/hadoop/hive/ql/parse/EximUtil.java   |   31 +-
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |   11 +-
 .../hadoop/hive/ql/parse/GenTezProcContext.java |   20 +
 .../hadoop/hive/ql/parse/GenTezUtils.java       |   90 +-
 .../apache/hadoop/hive/ql/parse/GenTezWork.java |   91 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    5 -
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   38 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |   79 +-
 .../hadoop/hive/ql/parse/PTFInvocationSpec.java |    8 +
 .../hadoop/hive/ql/parse/PTFTranslator.java     |    1 +
 .../apache/hadoop/hive/ql/parse/ParseUtils.java |   51 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  217 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    8 +-
 .../hadoop/hive/ql/parse/TezCompiler.java       |   33 +-
 .../hadoop/hive/ql/parse/TypeCheckCtx.java      |   15 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |    4 +
 .../hadoop/hive/ql/parse/WindowingSpec.java     |   84 +-
 .../hive/ql/plan/AbstractOperatorDesc.java      |   14 +
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |   20 +-
 .../apache/hadoop/hive/ql/plan/BaseWork.java    |    9 +-
 .../hive/ql/plan/CommonMergeJoinDesc.java       |    4 +
 .../hadoop/hive/ql/plan/ConditionalWork.java    |    4 +-
 .../hadoop/hive/ql/plan/DropTableDesc.java      |   26 +-
 .../apache/hadoop/hive/ql/plan/ExplainWork.java |   19 +-
 .../hadoop/hive/ql/plan/ExprNodeDescUtils.java  |  115 +
 .../hadoop/hive/ql/plan/FileSinkDesc.java       |   27 +-
 .../apache/hadoop/hive/ql/plan/FilterDesc.java  |   14 +
 .../hadoop/hive/ql/plan/HiveOperation.java      |    2 -
 .../apache/hadoop/hive/ql/plan/JoinDesc.java    |   11 +-
 .../apache/hadoop/hive/ql/plan/MapJoinDesc.java |   11 +
 .../hadoop/hive/ql/plan/MergeJoinWork.java      |    8 +-
 .../hadoop/hive/ql/plan/OperatorDesc.java       |    2 +
 .../apache/hadoop/hive/ql/plan/ReduceWork.java  |    2 +-
 .../hadoop/hive/ql/plan/TableScanDesc.java      |   21 +-
 .../hadoop/hive/ql/plan/VectorGroupByDesc.java  |   23 +-
 .../hive/ql/ppd/ExprWalkerProcFactory.java      |   17 +-
 .../hadoop/hive/ql/processors/SetProcessor.java |    3 +
 .../authorization/plugin/HiveAuthorizer.java    |   11 +
 .../plugin/HiveAuthorizerImpl.java              |   22 +
 .../authorization/plugin/HiveV1Authorizer.java  |   20 +
 .../hadoop/hive/ql/session/LineageState.java    |    9 +-
 .../hadoop/hive/ql/session/OperationLog.java    |    2 +-
 .../hadoop/hive/ql/session/SessionState.java    |   40 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |   14 +-
 .../hive/ql/txn/compactor/CompactorMR.java      |    4 +-
 .../ql/udf/generic/GenericUDAFCollectList.java  |    5 -
 .../ql/udf/generic/GenericUDAFCollectSet.java   |    5 -
 .../ql/udf/generic/GenericUDAFComputeStats.java |    4 +-
 .../hive/ql/udf/generic/GenericUDAFCount.java   |    4 -
 .../ql/udf/generic/GenericUDAFCovariance.java   |    4 -
 .../ql/udf/generic/GenericUDAFCumeDist.java     |    4 -
 .../ql/udf/generic/GenericUDAFDenseRank.java    |    4 -
 .../generic/GenericUDAFHistogramNumeric.java    |    2 +-
 .../generic/GenericUDAFPercentileApprox.java    |    2 +-
 .../hive/ql/udf/generic/GenericUDFMapKeys.java  |    6 +-
 .../udf/generic/NumDistinctValueEstimator.java  |   18 +
 .../hive/ql/udf/generic/NumericHistogram.java   |   14 +
 .../hive/ql/udf/ptf/WindowingTableFunction.java |  135 +-
 .../hadoop/hive/ql/util/JavaDataModel.java      |   30 -
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   68 +-
 .../hive/ql/exec/TestFileSinkOperator.java      |    3 +-
 .../hadoop/hive/ql/exec/TestOperators.java      |    3 +-
 .../hadoop/hive/ql/exec/TestUtilities.java      |   11 +
 .../exec/vector/TestVectorGroupByOperator.java  |    2 +-
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java |   73 +-
 .../ql/io/TestHiveBinarySearchRecordReader.java |    2 +-
 .../hadoop/hive/ql/io/TestIOContextMap.java     |  133 +
 .../hadoop/hive/ql/io/orc/TestFileDump.java     |   57 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |  114 +-
 .../hive/ql/io/orc/TestOrcRawRecordMerger.java  |   57 +-
 .../hive/ql/io/orc/TestOrcRecordUpdater.java    |    6 +-
 .../io/parquet/AbstractTestParquetDirect.java   |    8 +-
 .../ql/io/parquet/TestArrayCompatibility.java   |   18 +-
 .../ql/io/parquet/TestDataWritableWriter.java   |    8 +-
 .../ql/io/parquet/TestHiveSchemaConverter.java  |   16 +-
 .../hive/ql/io/parquet/TestMapStructures.java   |   10 +-
 .../parquet/TestMapredParquetInputFormat.java   |    2 +-
 .../parquet/TestMapredParquetOutputFormat.java  |    2 +-
 .../io/parquet/TestParquetRowGroupFilter.java   |    6 +-
 .../hive/ql/io/sarg/TestSearchArgumentImpl.java |   34 +-
 .../hadoop/hive/ql/lib/TestRuleRegExp.java      |  118 +
 .../hive/ql/lockmgr/TestDummyTxnManager.java    |  119 +-
 .../parse/TestUpdateDeleteSemanticAnalyzer.java |   36 +-
 .../hive/ql/txn/compactor/CompactorTest.java    |   20 +-
 .../hive/ql/txn/compactor/TestCleaner.java      |    8 +-
 .../hive/ql/txn/compactor/TestCleaner2.java     |   14 +
 .../hive/ql/txn/compactor/TestInitiator.java    |    4 +
 .../hive/ql/txn/compactor/TestWorker.java       |   49 +-
 .../hive/ql/txn/compactor/TestWorker2.java      |   16 +
 .../alter_partition_invalidspec.q               |    8 -
 .../clientnegative/alter_partition_nodrop.q     |    9 -
 .../alter_partition_nodrop_table.q              |    9 -
 .../clientnegative/alter_partition_offline.q    |   11 -
 .../clientnegative/drop_table_failure3.q        |   12 -
 .../queries/clientnegative/exchange_partition.q |   19 +
 .../queries/clientnegative/load_orc_negative1.q |    4 +
 .../queries/clientnegative/load_orc_negative2.q |    6 +
 .../clientnegative/load_orc_negative_part.q     |   14 +
 .../clientnegative/lockneg_try_lock_db_in_use.q |    2 +-
 .../queries/clientnegative/protectmode_part.q   |   15 -
 .../queries/clientnegative/protectmode_part1.q  |   21 -
 .../queries/clientnegative/protectmode_part2.q  |    9 -
 .../clientnegative/protectmode_part_no_drop.q   |   10 -
 .../clientnegative/protectmode_part_no_drop2.q  |   11 -
 .../queries/clientnegative/protectmode_tbl1.q   |    8 -
 .../queries/clientnegative/protectmode_tbl2.q   |   12 -
 .../queries/clientnegative/protectmode_tbl3.q   |   10 -
 .../queries/clientnegative/protectmode_tbl4.q   |   15 -
 .../queries/clientnegative/protectmode_tbl5.q   |   15 -
 .../queries/clientnegative/protectmode_tbl6.q   |    8 -
 .../queries/clientnegative/protectmode_tbl7.q   |   13 -
 .../queries/clientnegative/protectmode_tbl8.q   |   13 -
 .../clientnegative/protectmode_tbl_no_drop.q    |    9 -
 .../clientnegative/ptf_negative_NoWindowDefn.q  |    9 +
 .../test/queries/clientnegative/sa_fail_hook3.q |    4 -
 .../alter_partition_protect_mode.q              |   26 -
 .../queries/clientpositive/ambiguitycheck.q     |    2 -
 ql/src/test/queries/clientpositive/auto_join0.q |    1 +
 ql/src/test/queries/clientpositive/auto_join1.q |    1 +
 .../test/queries/clientpositive/auto_join21.q   |    1 +
 .../test/queries/clientpositive/auto_join29.q   |    1 +
 .../test/queries/clientpositive/auto_join30.q   |    1 +
 .../clientpositive/auto_sortmerge_join_10.q     |    1 +
 .../clientpositive/auto_sortmerge_join_13.q     |    1 +
 .../clientpositive/auto_sortmerge_join_14.q     |    1 +
 .../clientpositive/auto_sortmerge_join_15.q     |    1 +
 .../clientpositive/auto_sortmerge_join_9.q      |    1 +
 .../queries/clientpositive/avro_timestamp.q     |    2 +
 .../queries/clientpositive/avro_timestamp_win.q |   28 +
 .../queries/clientpositive/ba_table_union.q     |    2 +-
 ql/src/test/queries/clientpositive/bucket2.q    |    1 +
 ql/src/test/queries/clientpositive/bucket3.q    |    1 +
 ql/src/test/queries/clientpositive/bucket4.q    |    1 +
 .../clientpositive/bucket_map_join_tez1.q       |    1 +
 .../clientpositive/bucket_map_join_tez2.q       |    1 +
 .../queries/clientpositive/cbo_rp_auto_join0.q  |    1 +
 .../queries/clientpositive/cbo_rp_auto_join1.q  |    1 +
 .../test/queries/clientpositive/cbo_rp_join0.q  |   27 +
 .../test/queries/clientpositive/cbo_rp_join1.q  |   22 +
 .../test/queries/clientpositive/cbo_subq_in.q   |    1 +
 .../test/queries/clientpositive/cbo_windowing.q |    4 +-
 .../test/queries/clientpositive/constprog_dpp.q |   17 +
 .../clientpositive/correlationoptimizer1.q      |    1 +
 ql/src/test/queries/clientpositive/count.q      |    1 +
 ql/src/test/queries/clientpositive/cp_sel.q     |   11 +
 ql/src/test/queries/clientpositive/cross_join.q |    1 +
 .../clientpositive/cross_product_check_1.q      |    1 +
 .../clientpositive/cross_product_check_2.q      |    1 +
 ql/src/test/queries/clientpositive/ctas.q       |    1 +
 .../disable_merge_for_bucketing.q               |    1 +
 .../drop_partitions_ignore_protection.q         |   10 -
 .../clientpositive/dynamic_partition_pruning.q  |    1 +
 .../dynamic_partition_pruning_2.q               |    1 +
 .../clientpositive/dynpart_sort_opt_bucketing.q |   73 +
 .../dynpart_sort_opt_vectorization.q            |    1 +
 .../clientpositive/dynpart_sort_optimization.q  |    1 +
 .../clientpositive/dynpart_sort_optimization2.q |    1 +
 .../queries/clientpositive/encoding_nonutf8.q   |    7 +
 .../clientpositive/encryption_drop_partition.q  |   18 +
 .../encryption_insert_partition_static.q        |   17 -
 .../test/queries/clientpositive/explainuser_3.q |  115 +
 .../queries/clientpositive/fouter_join_ppr.q    |   73 +
 ql/src/test/queries/clientpositive/groupby1.q   |    1 +
 ql/src/test/queries/clientpositive/groupby2.q   |    1 +
 ql/src/test/queries/clientpositive/groupby3.q   |    1 +
 ...groupby_complex_types_multi_single_reducer.q |    8 +-
 .../clientpositive/groupby_grouping_sets6.q     |    1 -
 ql/src/test/queries/clientpositive/having.q     |    1 +
 .../clientpositive/hybridgrace_hashjoin_1.q     |    1 +
 .../clientpositive/hybridgrace_hashjoin_2.q     |    6 +-
 .../test/queries/clientpositive/insert_into1.q  |    1 +
 .../test/queries/clientpositive/insert_into2.q  |    1 +
 .../insert_non_utf8_encoding_table.q            |   20 +
 ql/src/test/queries/clientpositive/join0.q      |    1 +
 ql/src/test/queries/clientpositive/join1.q      |    1 +
 ql/src/test/queries/clientpositive/join42.q     |   36 +
 ql/src/test/queries/clientpositive/join43.q     |   83 +
 .../join_merge_multi_expressions.q              |    1 +
 .../test/queries/clientpositive/join_nullsafe.q |    1 +
 .../queries/clientpositive/limit_pushdown.q     |    1 +
 ql/src/test/queries/clientpositive/lineage2.q   |  116 +
 ql/src/test/queries/clientpositive/lineage3.q   |  162 +
 .../queries/clientpositive/load_dyn_part1.q     |    1 +
 .../queries/clientpositive/load_dyn_part2.q     |    1 +
 .../queries/clientpositive/load_dyn_part3.q     |    1 +
 ql/src/test/queries/clientpositive/load_orc.q   |   10 +
 .../test/queries/clientpositive/load_orc_part.q |   15 +
 .../test/queries/clientpositive/lvj_mapjoin.q   |    1 +
 .../queries/clientpositive/mapjoin_decimal.q    |    1 +
 .../queries/clientpositive/mapjoin_mapjoin.q    |    1 +
 ql/src/test/queries/clientpositive/mapreduce1.q |    1 +
 ql/src/test/queries/clientpositive/mapreduce2.q |    1 +
 ql/src/test/queries/clientpositive/merge1.q     |    1 +
 ql/src/test/queries/clientpositive/merge2.q     |    1 +
 ql/src/test/queries/clientpositive/mergejoin.q  |    3 +-
 .../clientpositive/metadata_only_queries.q      |    1 +
 ql/src/test/queries/clientpositive/mrr.q        |    3 +
 ql/src/test/queries/clientpositive/orc_merge1.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge2.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge3.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge4.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge5.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge6.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge7.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge9.q |   44 +
 .../clientpositive/orc_merge_incompat1.q        |    1 +
 .../clientpositive/orc_merge_incompat2.q        |    1 +
 ql/src/test/queries/clientpositive/parallel.q   |    1 +
 .../clientpositive/partition_timestamp.q        |    2 +
 .../clientpositive/partition_timestamp2.q       |    2 +
 .../clientpositive/partition_timestamp2_win.q   |   58 +
 .../clientpositive/partition_timestamp_win.q    |   59 +
 .../test/queries/clientpositive/protectmode.q   |   63 -
 .../test/queries/clientpositive/protectmode2.q  |   23 -
 ql/src/test/queries/clientpositive/ptf.q        |    1 +
 .../test/queries/clientpositive/ptf_matchpath.q |    1 +
 .../test/queries/clientpositive/ptf_streaming.q |    1 +
 .../queries/clientpositive/ql_rewrite_gbtoidx.q |    4 +-
 .../test/queries/clientpositive/script_pipe.q   |    1 +
 .../queries/clientpositive/selectDistinctStar.q |    3 +-
 .../clientpositive/select_dummy_source.q        |    1 +
 .../queries/clientpositive/select_same_col.q    |    5 +-
 ql/src/test/queries/clientpositive/skewjoin.q   |    1 +
 .../clientpositive/smb_join_partition_key.q     |   35 +
 .../queries/clientpositive/stats_noscan_1.q     |    1 +
 .../queries/clientpositive/stats_only_null.q    |    1 +
 .../queries/clientpositive/subquery_exists.q    |    1 +
 .../test/queries/clientpositive/subquery_in.q   |    1 +
 ql/src/test/queries/clientpositive/temp_table.q |    1 +
 .../clientpositive/tez_bmj_schema_evolution.q   |    1 +
 ql/src/test/queries/clientpositive/tez_dml.q    |    1 +
 .../clientpositive/tez_dynpart_hashjoin_1.q     |  101 +
 .../clientpositive/tez_dynpart_hashjoin_2.q     |   83 +
 ql/src/test/queries/clientpositive/tez_join.q   |    1 +
 .../test/queries/clientpositive/tez_join_hash.q |    1 +
 .../clientpositive/tez_join_result_complex.q    |  139 +
 .../queries/clientpositive/tez_join_tests.q     |    1 +
 .../queries/clientpositive/tez_joins_explain.q  |    1 +
 .../test/queries/clientpositive/tez_self_join.q |   33 +
 ql/src/test/queries/clientpositive/tez_smb_1.q  |   32 +
 .../test/queries/clientpositive/tez_smb_main.q  |    1 +
 ql/src/test/queries/clientpositive/tez_union.q  |    1 +
 ql/src/test/queries/clientpositive/tez_union2.q |    1 +
 .../tez_union_dynamic_partition.q               |   22 +
 .../queries/clientpositive/tez_union_group_by.q |    1 +
 .../clientpositive/tez_union_multiinsert.q      |    1 +
 .../tez_vector_dynpart_hashjoin_1.q             |  102 +
 .../tez_vector_dynpart_hashjoin_2.q             |   84 +
 ql/src/test/queries/clientpositive/transform1.q |    1 +
 .../clientpositive/udaf_percentile_approx_23.q  |    2 -
 ql/src/test/queries/clientpositive/union2.q     |    1 +
 ql/src/test/queries/clientpositive/union3.q     |    1 +
 ql/src/test/queries/clientpositive/union35.q    |    2 -
 ql/src/test/queries/clientpositive/union4.q     |    1 +
 ql/src/test/queries/clientpositive/union5.q     |    1 +
 ql/src/test/queries/clientpositive/union6.q     |    1 +
 ql/src/test/queries/clientpositive/union7.q     |    1 +
 ql/src/test/queries/clientpositive/union8.q     |    1 +
 ql/src/test/queries/clientpositive/union9.q     |    1 +
 .../queries/clientpositive/unionDistinct_1.q    |    1 +
 .../queries/clientpositive/update_all_types.q   |    2 +
 .../queries/clientpositive/update_orig_table.q  |    2 +
 .../queries/clientpositive/vector_aggregate_9.q |    1 +
 .../vector_aggregate_without_gby.q              |   14 +
 .../queries/clientpositive/vector_between_in.q  |    1 +
 .../clientpositive/vector_binary_join_groupby.q |    1 +
 .../test/queries/clientpositive/vector_bucket.q |    1 +
 .../clientpositive/vector_cast_constant.q       |    1 +
 .../test/queries/clientpositive/vector_char_2.q |    1 +
 .../test/queries/clientpositive/vector_char_4.q |    1 +
 .../clientpositive/vector_char_mapjoin1.q       |    1 +
 .../queries/clientpositive/vector_char_simple.q |    1 +
 .../queries/clientpositive/vector_coalesce.q    |   52 +-
 .../queries/clientpositive/vector_coalesce_2.q  |    1 +
 .../clientpositive/vector_count_distinct.q      |    1 +
 .../queries/clientpositive/vector_data_types.q  |    1 +
 .../test/queries/clientpositive/vector_date_1.q |    1 +
 .../queries/clientpositive/vector_decimal_1.q   |    1 +
 .../clientpositive/vector_decimal_10_0.q        |    1 +
 .../queries/clientpositive/vector_decimal_2.q   |    1 +
 .../clientpositive/vector_decimal_aggregate.q   |    1 +
 .../clientpositive/vector_decimal_cast.q        |    1 +
 .../clientpositive/vector_decimal_expressions.q |   12 +-
 .../clientpositive/vector_decimal_mapjoin.q     |    1 +
 .../clientpositive/vector_decimal_math_funcs.q  |    1 +
 .../clientpositive/vector_decimal_precision.q   |    1 +
 .../clientpositive/vector_decimal_round.q       |    1 +
 .../clientpositive/vector_decimal_round_2.q     |    1 +
 .../queries/clientpositive/vector_decimal_udf.q |    1 +
 .../clientpositive/vector_decimal_udf2.q        |    1 +
 .../queries/clientpositive/vector_distinct_2.q  |    1 +
 ql/src/test/queries/clientpositive/vector_elt.q |    1 +
 .../queries/clientpositive/vector_groupby_3.q   |    1 +
 .../clientpositive/vector_groupby_reduce.q      |    4 +
 .../clientpositive/vector_grouping_sets.q       |    1 +
 .../queries/clientpositive/vector_if_expr.q     |    1 +
 .../queries/clientpositive/vector_inner_join.q  |    1 +
 .../queries/clientpositive/vector_interval_1.q  |    1 +
 .../queries/clientpositive/vector_interval_2.q  |    1 +
 .../clientpositive/vector_interval_mapjoin.q    |    1 +
 .../test/queries/clientpositive/vector_join30.q |    1 +
 .../clientpositive/vector_left_outer_join.q     |    1 +
 .../clientpositive/vector_left_outer_join2.q    |    1 +
 .../clientpositive/vector_leftsemi_mapjoin.q    |    1 +
 .../clientpositive/vector_mapjoin_reduce.q      |    1 +
 .../vector_mr_diff_schema_alias.q               |    1 +
 .../clientpositive/vector_multi_insert.q        |    1 +
 .../vector_non_string_partition.q               |    1 +
 .../clientpositive/vector_null_projection.q     |    1 +
 .../clientpositive/vector_nullsafe_join.q       |    1 +
 .../queries/clientpositive/vector_orderby_5.q   |    1 +
 .../queries/clientpositive/vector_outer_join0.q |    1 +
 .../queries/clientpositive/vector_outer_join1.q |   20 +-
 .../queries/clientpositive/vector_outer_join2.q |   19 +-
 .../queries/clientpositive/vector_outer_join3.q |   18 +-
 .../queries/clientpositive/vector_outer_join4.q |   17 +-
 .../queries/clientpositive/vector_outer_join5.q |    1 +
 .../vector_partition_diff_num_cols.q            |    1 +
 .../vector_partitioned_date_time.q              |    5 +-
 .../vector_partitioned_date_time_win.q          |  129 +
 .../vector_reduce_groupby_decimal.q             |    1 +
 .../clientpositive/vector_string_concat.q       |    1 +
 .../queries/clientpositive/vector_varchar_4.q   |    1 +
 .../clientpositive/vector_varchar_mapjoin1.q    |    1 +
 .../clientpositive/vector_varchar_simple.q      |    1 +
 .../queries/clientpositive/vectorization_0.q    |    3 +
 .../queries/clientpositive/vectorization_1.q    |    3 +
 .../queries/clientpositive/vectorization_10.q   |    3 +
 .../queries/clientpositive/vectorization_11.q   |    3 +
 .../queries/clientpositive/vectorization_12.q   |    3 +
 .../queries/clientpositive/vectorization_13.q   |  142 +-
 .../queries/clientpositive/vectorization_14.q   |    3 +
 .../queries/clientpositive/vectorization_15.q   |    3 +
 .../queries/clientpositive/vectorization_16.q   |    1 +
 .../queries/clientpositive/vectorization_17.q   |   51 +
 .../queries/clientpositive/vectorization_2.q    |    3 +
 .../queries/clientpositive/vectorization_3.q    |    3 +
 .../queries/clientpositive/vectorization_4.q    |    3 +
 .../queries/clientpositive/vectorization_5.q    |    3 +
 .../queries/clientpositive/vectorization_6.q    |    3 +
 .../queries/clientpositive/vectorization_7.q    |   80 +-
 .../queries/clientpositive/vectorization_8.q    |   81 +-
 .../queries/clientpositive/vectorization_9.q    |   39 +-
 .../clientpositive/vectorization_decimal_date.q |    1 +
 .../queries/clientpositive/vectorization_div0.q |    1 +
 .../clientpositive/vectorization_limit.q        |    1 +
 .../clientpositive/vectorization_part_project.q |    1 +
 .../clientpositive/vectorization_part_varchar.q |    7 +
 .../clientpositive/vectorization_pushdown.q     |    1 +
 .../vectorization_short_regress.q               |  420 +-
 .../clientpositive/vectorized_bucketmapjoin1.q  |    1 +
 .../queries/clientpositive/vectorized_case.q    |    1 +
 .../queries/clientpositive/vectorized_casts.q   |    3 +
 .../queries/clientpositive/vectorized_context.q |    1 +
 .../clientpositive/vectorized_date_funcs.q      |    3 +
 .../clientpositive/vectorized_distinct_gby.q    |    1 +
 .../vectorized_dynamic_partition_pruning.q      |    1 +
 .../queries/clientpositive/vectorized_mapjoin.q |    1 +
 .../clientpositive/vectorized_math_funcs.q      |    1 +
 .../clientpositive/vectorized_nested_mapjoin.q  |    1 +
 .../queries/clientpositive/vectorized_parquet.q |    1 +
 .../clientpositive/vectorized_shufflejoin.q     |    1 +
 .../clientpositive/vectorized_string_funcs.q    |    1 +
 .../clientpositive/vectorized_timestamp_funcs.q |    1 +
 ql/src/test/queries/clientpositive/windowing.q  |    2 +-
 .../clientpositive/windowing_windowspec3.q      |   62 +
 .../resources/orc-file-dump-bloomfilter.out     |   92 +-
 .../resources/orc-file-dump-bloomfilter2.out    |   92 +-
 .../orc-file-dump-dictionary-threshold.out      |   76 +-
 ql/src/test/resources/orc-file-dump.json        |  108 +-
 ql/src/test/resources/orc-file-dump.out         |   84 +-
 ql/src/test/resources/orc-file-has-null.out     |   62 +-
 .../alter_numbuckets_partitioned_table.q.out    |    8 -
 .../results/beelinepositive/create_like.q.out   |    3 -
 .../results/beelinepositive/create_like2.q.out  |    1 -
 .../beelinepositive/create_like_view.q.out      |    4 -
 .../beelinepositive/create_skewed_table1.q.out  |    3 -
 .../results/beelinepositive/create_view.q.out   |   14 -
 .../create_view_partitioned.q.out               |    3 -
 ql/src/test/results/beelinepositive/ctas.q.out  |    5 -
 .../describe_formatted_view_partitioned.q.out   |    1 -
 .../beelinepositive/describe_table.q.out        |    3 -
 .../test/results/beelinepositive/merge3.q.out   |    1 -
 .../part_inherit_tbl_props.q.out                |    1 -
 .../part_inherit_tbl_props_empty.q.out          |    1 -
 .../part_inherit_tbl_props_with_star.q.out      |    1 -
 .../results/beelinepositive/protectmode2.q.out  |    2 -
 .../test/results/beelinepositive/stats1.q.out   |    2 -
 .../test/results/beelinepositive/stats10.q.out  |    3 -
 .../test/results/beelinepositive/stats11.q.out  |    4 -
 .../test/results/beelinepositive/stats12.q.out  |    5 -
 .../test/results/beelinepositive/stats13.q.out  |    6 -
 .../test/results/beelinepositive/stats14.q.out  |    5 -
 .../test/results/beelinepositive/stats15.q.out  |    5 -
 .../test/results/beelinepositive/stats16.q.out  |    2 -
 .../test/results/beelinepositive/stats18.q.out  |    2 -
 .../test/results/beelinepositive/stats2.q.out   |    2 -
 .../test/results/beelinepositive/stats3.q.out   |    2 -
 .../test/results/beelinepositive/stats4.q.out   |    6 -
 .../test/results/beelinepositive/stats5.q.out   |    1 -
 .../test/results/beelinepositive/stats6.q.out   |    5 -
 .../test/results/beelinepositive/stats7.q.out   |    3 -
 .../test/results/beelinepositive/stats8.q.out   |   10 -
 .../test/results/beelinepositive/stats9.q.out   |    1 -
 .../beelinepositive/stats_empty_partition.q.out |    1 -
 .../clientnegative/alter_file_format.q.out      |    1 -
 .../alter_view_as_select_with_partition.q.out   |    1 -
 .../clientnegative/exchange_partition.q.out     |   54 +
 .../clientnegative/load_orc_negative1.q.out     |    9 +
 .../clientnegative/load_orc_negative2.q.out     |   25 +
 .../clientnegative/load_orc_negative_part.q.out |   52 +
 .../lockneg_try_lock_db_in_use.q.out            |    9 +-
 .../ptf_negative_NoWindowDefn.q.out             |    1 +
 .../stats_partialscan_autogether.q.out          |    2 -
 .../clientnegative/udf_assert_true.q.out        |   12 +-
 .../clientnegative/udf_assert_true2.q.out       |    6 +-
 .../clientpositive/alter_file_format.q.out      |   19 -
 .../clientpositive/alter_merge_orc.q.out        |   24 +-
 .../clientpositive/alter_merge_stats_orc.q.out  |   30 +-
 .../alter_numbuckets_partitioned_table.q.out    |   16 -
 .../alter_numbuckets_partitioned_table2.q.out   |   27 -
 ...lter_numbuckets_partitioned_table2_h23.q.out |   27 -
 ...alter_numbuckets_partitioned_table_h23.q.out |   16 -
 .../alter_partition_change_col.q.out            |    8 +-
 .../alter_partition_clusterby_sortby.q.out      |    7 -
 .../clientpositive/alter_skewed_table.q.out     |    6 -
 .../clientpositive/alter_table_cascade.q.out    |    8 +-
 .../clientpositive/alter_table_not_sorted.q.out |    2 -
 .../clientpositive/alter_table_serde2.q.out     |    6 -
 .../clientpositive/alter_view_as_select.q.out   |    3 -
 .../clientpositive/annotate_stats_groupby.q.out |   28 +-
 .../annotate_stats_groupby2.q.out               |    8 +-
 .../annotate_stats_join_pkfk.q.out              |   20 +-
 .../clientpositive/annotate_stats_part.q.out    |    6 +-
 .../clientpositive/annotate_stats_select.q.out  |   52 +-
 .../clientpositive/annotate_stats_table.q.out   |    4 +-
 .../clientpositive/authorization_index.q.out    |    1 -
 .../results/clientpositive/auto_join12.q.out    |   54 +-
 .../results/clientpositive/auto_join13.q.out    |   26 +-
 .../results/clientpositive/auto_join14.q.out    |   12 +-
 .../results/clientpositive/auto_join5.q.out     |    8 +-
 .../results/clientpositive/auto_join8.q.out     |    2 +-
 .../auto_join_without_localtask.q.out           |  218 +-
 .../clientpositive/autogen_colalias.q.out       |    4 +-
 .../clientpositive/avro_timestamp.q.java1.7.out |    8 +-
 .../clientpositive/avro_timestamp.q.java1.8.out |    8 +-
 .../avro_timestamp_win.q.java1.7.out            |  134 +
 .../avro_timestamp_win.q.java1.8.out            |  134 +
 .../results/clientpositive/ba_table_union.q.out |   16 +-
 .../test/results/clientpositive/bucket5.q.out   |    1 -
 .../bucketsortoptimize_insert_7.q.out           |    2 +-
 .../clientpositive/cbo_rp_auto_join0.q.out      |   32 +-
 .../clientpositive/cbo_rp_auto_join1.q.out      |  619 +-
 .../results/clientpositive/cbo_rp_join0.q.out   | 6886 ++++++++++++++++++
 .../results/clientpositive/cbo_rp_join1.q.out   |  426 ++
 .../results/clientpositive/cbo_subq_in.q.out    |    2 +
 .../results/clientpositive/cbo_windowing.q.out  |   88 +-
 .../test/results/clientpositive/cluster.q.out   |   20 +-
 .../test/results/clientpositive/combine2.q.out  |   16 +-
 .../constantPropagateForSubQuery.q.out          |   38 +-
 .../clientpositive/correlationoptimizer15.q.out |  120 +-
 .../clientpositive/correlationoptimizer6.q.out  | 1011 ++-
 ql/src/test/results/clientpositive/cp_sel.q.out |  195 +
 .../create_alter_list_bucketing_table1.q.out    |    7 -
 .../results/clientpositive/create_like.q.out    |    9 -
 .../results/clientpositive/create_like2.q.out   |    1 -
 .../clientpositive/create_like_tbl_props.q.out  |    5 -
 .../clientpositive/create_like_view.q.out       |    4 -
 .../clientpositive/create_or_replace_view.q.out |    5 -
 .../clientpositive/create_skewed_table1.q.out   |    3 -
 .../results/clientpositive/create_view.q.out    |   14 -
 .../create_view_partitioned.q.out               |    3 -
 .../clientpositive/create_view_translate.q.out  |    2 -
 .../results/clientpositive/cross_join.q.out     |    8 +-
 ql/src/test/results/clientpositive/ctas.q.out   |    5 -
 .../results/clientpositive/ctas_colname.q.out   |    7 -
 .../results/clientpositive/ctas_hadoop20.q.out  |    5 -
 .../ctas_uses_database_location.q.out           |    1 -
 .../clientpositive/database_location.q.out      |    2 -
 .../results/clientpositive/decimal_serde.q.out  |    2 -
 .../clientpositive/default_file_format.q.out    |    5 -
 .../describe_comment_indent.q.out               |    1 -
 .../describe_comment_nonascii.q.out             |    1 -
 .../describe_formatted_view_partitioned.q.out   |    2 -
 .../clientpositive/describe_syntax.q.out        |    6 -
 .../results/clientpositive/describe_table.q.out |    7 -
 .../dynpart_sort_opt_bucketing.q.out            |  277 +
 .../dynpart_sort_opt_vectorization.q.out        |   48 +-
 .../dynpart_sort_optimization.q.out             |   32 -
 .../dynpart_sort_optimization2.q.out            |   32 +-
 .../dynpart_sort_optimization_acid.q.out        |   60 +-
 .../clientpositive/encoding_nonutf8.q.out       |   36 +
 .../encrypted/encryption_drop_partition.q.out   |   81 +
 .../encrypted/encryption_drop_table.q.out       |    2 +-
 .../encryption_insert_partition_dynamic.q.out   |   18 +-
 .../encryption_insert_partition_static.q.out    |  739 +-
 .../encrypted/encryption_insert_values.q.out    |    1 -
 .../clientpositive/exim_hidden_files.q.out      |    1 -
 .../extrapolate_part_stats_full.q.out           |   24 +-
 .../extrapolate_part_stats_partial.q.out        |   76 +-
 .../extrapolate_part_stats_partial_ndv.q.out    |   38 +-
 .../clientpositive/filter_join_breaktask.q.out  |   12 +-
 .../clientpositive/fold_eq_with_case_when.q.out |    6 +-
 .../test/results/clientpositive/fold_when.q.out |    2 +-
 .../clientpositive/fouter_join_ppr.q.out        | 1694 +++++
 .../test/results/clientpositive/groupby10.q.out |  140 +-
 .../test/results/clientpositive/groupby11.q.out |   70 +-
 .../test/results/clientpositive/groupby2.q.out  |   33 +-
 .../clientpositive/groupby2_map_skew.q.out      |   33 +-
 .../test/results/clientpositive/groupby8.q.out  |  140 +-
 .../clientpositive/groupby8_map_skew.q.out      |   70 +-
 ...pby_complex_types_multi_single_reducer.q.out |   68 +-
 .../results/clientpositive/groupby_cube1.q.out  |   29 +-
 .../clientpositive/groupby_grouping_sets6.q.out |   46 +-
 .../clientpositive/groupby_rollup1.q.out        |   29 +-
 .../clientpositive/groupby_sort_1_23.q.out      |   10 +-
 .../clientpositive/groupby_sort_skew_1_23.q.out |   10 +-
 ql/src/test/results/clientpositive/having.q.out |   28 +-
 .../clientpositive/index_auto_mult_tables.q.out |   12 +
 .../index_auto_mult_tables_compact.q.out        |    9 +
 .../clientpositive/index_auto_partitioned.q.out |    9 +
 .../clientpositive/index_auto_unused.q.out      |    4 +-
 .../clientpositive/index_auto_update.q.out      |    2 +
 .../results/clientpositive/index_bitmap.q.out   |   24 +
 .../index_bitmap_auto_partitioned.q.out         |   12 +
 .../clientpositive/index_bitmap_rc.q.out        |   24 +
 .../results/clientpositive/index_compact.q.out  |   18 +
 .../clientpositive/index_compact_2.q.out        |   18 +
 .../clientpositive/index_skewtable.q.out        |    1 -
 .../clientpositive/infer_bucket_sort.q.out      |   50 -
 .../infer_bucket_sort_bucketed_table.q.out      |    2 -
 .../infer_bucket_sort_convert_join.q.out        |    4 -
 .../infer_bucket_sort_dyn_part.q.out            |   16 -
 .../infer_bucket_sort_grouping_operators.q.out  |   12 -
 .../infer_bucket_sort_list_bucket.q.out         |    4 -
 .../infer_bucket_sort_map_operators.q.out       |    8 -
 .../infer_bucket_sort_merge.q.out               |    4 -
 .../infer_bucket_sort_multi_insert.q.out        |   16 -
 .../infer_bucket_sort_num_buckets.q.out         |    4 -
 .../infer_bucket_sort_reducers_power_two.q.out  |   12 -
 ql/src/test/results/clientpositive/input7.q.out |    2 +-
 .../results/clientpositive/input_part10.q.out   |    5 +-
 .../results/clientpositive/insert_into5.q.out   |    5 +-
 .../insert_non_utf8_encoding_table.q.out        |   89 +
 ql/src/test/results/clientpositive/join12.q.out |   28 +-
 ql/src/test/results/clientpositive/join13.q.out |   32 +-
 ql/src/test/results/clientpositive/join14.q.out |   14 +-
 ql/src/test/results/clientpositive/join28.q.out |    2 +-
 ql/src/test/results/clientpositive/join32.q.out |   38 +-
 .../clientpositive/join32_lessSize.q.out        |  128 +-
 ql/src/test/results/clientpositive/join33.q.out |   38 +-
 ql/src/test/results/clientpositive/join34.q.out |    2 +-
 ql/src/test/results/clientpositive/join35.q.out |    2 +-
 ql/src/test/results/clientpositive/join42.q.out |  312 +
 ql/src/test/results/clientpositive/join43.q.out |  648 ++
 ql/src/test/results/clientpositive/join5.q.out  |   20 +-
 ql/src/test/results/clientpositive/join8.q.out  |    2 +-
 .../clientpositive/join_alt_syntax.q.out        |  104 +-
 .../clientpositive/join_cond_pushdown_1.q.out   |   42 +-
 .../clientpositive/join_cond_pushdown_2.q.out   |   62 +-
 .../clientpositive/join_cond_pushdown_3.q.out   |   42 +-
 .../clientpositive/join_cond_pushdown_4.q.out   |   62 +-
 .../results/clientpositive/join_merging.q.out   |  117 +-
 .../results/clientpositive/join_nulls.q.out     |    2 +-
 .../results/clientpositive/lateral_view.q.out   |   68 +-
 .../clientpositive/lateral_view_explode2.q.out  |    6 +-
 .../clientpositive/lateral_view_noalias.q.out   |   22 +-
 .../results/clientpositive/lb_fs_stats.q.out    |    2 -
 .../results/clientpositive/limit_pushdown.q.out |   98 +-
 .../test/results/clientpositive/lineage1.q.out  |    4 +-
 .../test/results/clientpositive/lineage2.q.out  |  677 ++
 .../test/results/clientpositive/lineage3.q.out  |  293 +
 .../clientpositive/list_bucket_dml_1.q.out      |    4 -
 .../list_bucket_dml_10.q.java1.7.out            |    2 -
 .../list_bucket_dml_10.q.java1.8.out            |    2 -
 .../list_bucket_dml_11.q.java1.7.out            |    2 -
 .../list_bucket_dml_11.q.java1.8.out            |    2 -
 .../list_bucket_dml_12.q.java1.7.out            |    4 +-
 .../list_bucket_dml_12.q.java1.8.out            |    2 -
 .../list_bucket_dml_13.q.java1.7.out            |    4 +-
 .../list_bucket_dml_13.q.java1.8.out            |    2 -
 .../clientpositive/list_bucket_dml_14.q.out     |    1 -
 .../list_bucket_dml_2.q.java1.7.out             |    2 -
 .../list_bucket_dml_2.q.java1.8.out             |    2 -
 .../clientpositive/list_bucket_dml_3.q.out      |    2 -
 .../list_bucket_dml_4.q.java1.7.out             |    4 -
 .../list_bucket_dml_4.q.java1.8.out             |    4 -
 .../list_bucket_dml_5.q.java1.7.out             |    4 -
 .../list_bucket_dml_5.q.java1.8.out             |    4 -
 .../list_bucket_dml_6.q.java1.7.out             |    8 -
 .../list_bucket_dml_6.q.java1.8.out             |    8 -
 .../clientpositive/list_bucket_dml_7.q.out      |    8 -
 .../list_bucket_dml_8.q.java1.7.out             |    6 -
 .../list_bucket_dml_8.q.java1.8.out             |    6 -
 .../list_bucket_dml_9.q.java1.7.out             |    4 -
 .../list_bucket_dml_9.q.java1.8.out             |    4 -
 .../list_bucket_query_multiskew_1.q.out         |    2 -
 .../list_bucket_query_multiskew_2.q.out         |    2 -
 .../list_bucket_query_multiskew_3.q.out         |    6 -
 .../list_bucket_query_oneskew_1.q.out           |    2 -
 .../list_bucket_query_oneskew_2.q.out           |    2 -
 .../list_bucket_query_oneskew_3.q.out           |    2 -
 .../clientpositive/load_dyn_part13.q.out        |    8 +-
 .../clientpositive/load_dyn_part14.q.out        |   63 +-
 .../test/results/clientpositive/load_orc.q.out  |   43 +
 .../results/clientpositive/load_orc_part.q.out  |   70 +
 .../clientpositive/louter_join_ppr.q.out        |   74 +-
 .../clientpositive/mapjoin_mapjoin.q.out        |  120 +-
 .../clientpositive/mapjoin_subquery.q.out       |    4 +-
 ql/src/test/results/clientpositive/merge3.q.out |    1 -
 .../results/clientpositive/multiMapJoin1.q.out  |   10 +-
 .../results/clientpositive/multi_insert.q.out   |   32 +-
 ...i_insert_move_tasks_share_dependencies.q.out |   32 +-
 .../results/clientpositive/optional_outer.q.out |   36 +-
 .../results/clientpositive/orc_analyze.q.out    |   48 -
 .../results/clientpositive/orc_create.q.out     |    6 -
 .../orc_dictionary_threshold.q.out              |    2 +-
 .../results/clientpositive/orc_merge9.q.out     |  186 +
 .../clientpositive/outer_join_ppr.q.java1.7.out |  168 +-
 .../clientpositive/parallel_orderby.q.out       |    2 -
 .../parquet_array_null_element.q.out            |    1 -
 .../results/clientpositive/parquet_create.q.out |    1 -
 .../clientpositive/parquet_partitioned.q.out    |    1 -
 .../results/clientpositive/parquet_serde.q.out  |    5 -
 .../clientpositive/part_inherit_tbl_props.q.out |    2 -
 .../part_inherit_tbl_props_empty.q.out          |    2 -
 .../part_inherit_tbl_props_with_star.q.out      |    2 -
 .../partition_coltype_literals.q.out            |   16 -
 .../clientpositive/partition_timestamp.q.out    |    8 +-
 .../clientpositive/partition_timestamp2.q.out   |    8 +-
 .../partition_timestamp2_win.q.out              |  399 +
 .../partition_timestamp_win.q.out               |  316 +
 .../results/clientpositive/ppd_gby_join.q.out   |  104 +-
 .../test/results/clientpositive/ppd_join.q.out  |  106 +-
 .../test/results/clientpositive/ppd_join2.q.out |   88 +-
 .../test/results/clientpositive/ppd_join3.q.out |  114 +-
 .../clientpositive/ppd_outer_join4.q.out        |   88 +-
 .../results/clientpositive/ppd_random.q.out     |   80 +-
 .../results/clientpositive/ppd_udf_case.q.out   |   40 +-
 .../results/clientpositive/ppd_union_view.q.out |   78 +-
 .../results/clientpositive/protectmode2.q.out   |    2 -
 ql/src/test/results/clientpositive/ptf.q.out    |   28 +-
 .../clientpositive/rcfile_default_format.q.out  |    8 -
 .../clientpositive/rcfile_null_value.q.out      |   20 +-
 .../clientpositive/router_join_ppr.q.out        |  170 +-
 .../clientpositive/selectDistinctStar.q.out     |    2 -
 .../clientpositive/select_same_col.q.out        |    8 +-
 .../test/results/clientpositive/skewjoin.q.out  |   46 +-
 .../clientpositive/smb_join_partition_key.q.out |  128 +
 .../clientpositive/spark/alter_merge_orc.q.out  |   24 +-
 .../spark/alter_merge_stats_orc.q.out           |   30 +-
 .../spark/annotate_stats_join.q.out             |  204 +-
 .../clientpositive/spark/auto_join1.q.out       |   60 +-
 .../clientpositive/spark/auto_join10.q.out      |   28 +-
 .../clientpositive/spark/auto_join11.q.out      |   32 +-
 .../clientpositive/spark/auto_join12.q.out      |   54 +-
 .../clientpositive/spark/auto_join13.q.out      |   40 +-
 .../clientpositive/spark/auto_join14.q.out      |   70 +-
 .../clientpositive/spark/auto_join16.q.out      |   68 +-
 .../clientpositive/spark/auto_join17.q.out      |   60 +-
 .../clientpositive/spark/auto_join18.q.out      |   30 +-
 .../spark/auto_join18_multi_distinct.q.out      |   30 +-
 .../clientpositive/spark/auto_join19.q.out      |   56 +-
 .../clientpositive/spark/auto_join26.q.out      |   68 +-
 .../clientpositive/spark/auto_join27.q.out      |   24 +-
 .../clientpositive/spark/auto_join4.q.out       |   14 +-
 .../clientpositive/spark/auto_join5.q.out       |   14 +-
 .../clientpositive/spark/auto_join6.q.out       |   10 +-
 .../clientpositive/spark/auto_join8.q.out       |   14 +-
 .../clientpositive/spark/auto_join9.q.out       |   56 +-
 .../spark/auto_join_filters.q.out               |    4 +-
 .../clientpositive/spark/auto_join_nulls.q.out  |    6 +-
 .../spark/auto_join_without_localtask.q.out     |  122 +-
 .../results/clientpositive/spark/bucket5.q.out  |    1 -
 .../spark/bucketsortoptimize_insert_7.q.out     |   18 +-
 .../clientpositive/spark/cbo_subq_in.q.out      |    2 +
 .../spark/constprog_partitioner.q.out           |   80 +-
 .../results/clientpositive/spark/count.q.out    |   14 +-
 .../clientpositive/spark/cross_join.q.out       |   74 +-
 .../spark/cross_product_check_1.q.out           |   40 +-
 .../spark/cross_product_check_2.q.out           |   38 +-
 .../results/clientpositive/spark/ctas.q.out     |    5 -
 .../spark/filter_join_breaktask.q.out           |   12 +-
 .../results/clientpositive/spark/groupby1.q.out |    8 +-
 .../clientpositive/spark/groupby10.q.out        |   88 +-
 .../clientpositive/spark/groupby11.q.out        |   44 +-
 .../results/clientpositive/spark/groupby2.q.out |   17 +-
 .../results/clientpositive/spark/groupby4.q.out |    6 +-
 .../results/clientpositive/spark/groupby8.q.out |   88 +-
 .../spark/groupby8_map_skew.q.out               |   44 +-
 ...pby_complex_types_multi_single_reducer.q.out |   90 +-
 .../clientpositive/spark/groupby_cube1.q.out    |   41 +-
 .../clientpositive/spark/groupby_position.q.out |  130 +-
 .../clientpositive/spark/groupby_rollup1.q.out  |   37 +-
 .../spark/groupby_sort_1_23.q.out               |   92 +-
 .../spark/groupby_sort_skew_1_23.q.out          |   92 +-
 .../results/clientpositive/spark/having.q.out   |   94 +-
 .../spark/index_auto_self_join.q.out            |  116 +-
 .../infer_bucket_sort_bucketed_table.q.out      |    2 -
 .../spark/infer_bucket_sort_convert_join.q.out  |    4 -
 .../spark/infer_bucket_sort_map_operators.q.out |   13 +-
 .../spark/infer_bucket_sort_merge.q.out         |    8 +-
 .../spark/infer_bucket_sort_num_buckets.q.out   |    4 -
 .../infer_bucket_sort_reducers_power_two.q.out  |   12 -
 .../clientpositive/spark/innerjoin.q.out        |   54 +-
 .../results/clientpositive/spark/join1.q.out    |   38 +-
 .../results/clientpositive/spark/join10.q.out   |   28 +-
 .../results/clientpositive/spark/join11.q.out   |   18 +-
 .../results/clientpositive/spark/join12.q.out   |   26 +-
 .../results/clientpositive/spark/join13.q.out   |   82 +-
 .../results/clientpositive/spark/join14.q.out   |   60 +-
 .../results/clientpositive/spark/join16.q.out   |   36 +-
 .../results/clientpositive/spark/join17.q.out   |   52 +-
 .../results/clientpositive/spark/join18.q.out   |   12 +-
 .../spark/join18_multi_distinct.q.out           |   12 +-
 .../results/clientpositive/spark/join28.q.out   |    2 +-
 .../results/clientpositive/spark/join29.q.out   |   66 +-
 .../results/clientpositive/spark/join31.q.out   |  104 +-
 .../results/clientpositive/spark/join32.q.out   |  104 +-
 .../clientpositive/spark/join32_lessSize.q.out  |  242 +-
 .../results/clientpositive/spark/join33.q.out   |  104 +-
 .../results/clientpositive/spark/join34.q.out   |   36 +-
 .../results/clientpositive/spark/join35.q.out   |   92 +-
 .../results/clientpositive/spark/join4.q.out    |   16 +-
 .../results/clientpositive/spark/join40.q.out   |  116 +-
 .../results/clientpositive/spark/join5.q.out    |   16 +-
 .../results/clientpositive/spark/join6.q.out    |   10 +-
 .../results/clientpositive/spark/join8.q.out    |   16 +-
 .../results/clientpositive/spark/join9.q.out    |   42 +-
 .../clientpositive/spark/join_alt_syntax.q.out  |  314 +-
 .../spark/join_cond_pushdown_1.q.out            |   70 +-
 .../spark/join_cond_pushdown_2.q.out            |  134 +-
 .../spark/join_cond_pushdown_3.q.out            |   70 +-
 .../spark/join_cond_pushdown_4.q.out            |  134 +-
 .../clientpositive/spark/join_merging.q.out     |  115 +-
 .../results/clientpositive/spark/join_vc.q.out  |   40 +-
 .../spark/lateral_view_explode2.q.out           |    6 +-
 .../spark/limit_partition_metadataonly.q.out    |    4 +-
 .../clientpositive/spark/limit_pushdown.q.out   |   58 +-
 .../spark/list_bucket_dml_10.q.java1.7.out      |    2 -
 .../spark/list_bucket_dml_10.q.java1.8.out      |    2 -
 .../spark/list_bucket_dml_2.q.java1.7.out       |    2 -
 .../spark/list_bucket_dml_2.q.java1.8.out       |    2 -
 .../spark/list_bucket_dml_2.q.out               |  Bin 28747 -> 28667 bytes
 .../clientpositive/spark/load_dyn_part13.q.out  |   12 +-
 .../clientpositive/spark/load_dyn_part14.q.out  |   57 +-
 .../clientpositive/spark/louter_join_ppr.q.out  |  454 +-
 .../clientpositive/spark/mapjoin_mapjoin.q.out  |  202 +-
 .../clientpositive/spark/mapjoin_subquery.q.out |    4 +-
 .../results/clientpositive/spark/merge1.q.out   |    4 +-
 .../results/clientpositive/spark/merge2.q.out   |    4 +-
 .../spark/metadata_only_queries.q.out           |   12 +-
 .../clientpositive/spark/multi_insert.q.out     |   32 +-
 ...i_insert_move_tasks_share_dependencies.q.out |   32 +-
 .../clientpositive/spark/orc_analyze.q.out      |   22 -
 .../results/clientpositive/spark/order2.q.out   |    2 +-
 .../spark/outer_join_ppr.q.java1.7.out          |  292 +-
 .../spark/outer_join_ppr.q.java1.8.out          |  366 +-
 .../clientpositive/spark/parallel_join1.q.out   |   38 +-
 .../clientpositive/spark/parallel_orderby.q.out |    2 -
 .../test/results/clientpositive/spark/pcr.q.out |    4 +-
 .../clientpositive/spark/ppd_gby_join.q.out     |   90 +-
 .../results/clientpositive/spark/ppd_join.q.out |   96 +-
 .../clientpositive/spark/ppd_join2.q.out        |   86 +-
 .../clientpositive/spark/ppd_join3.q.out        |  116 +-
 .../clientpositive/spark/ppd_join_filter.q.out  |  442 +-
 .../clientpositive/spark/ppd_outer_join1.q.out  |  120 +-
 .../clientpositive/spark/ppd_outer_join2.q.out  |  158 +-
 .../clientpositive/spark/ppd_outer_join3.q.out  |  154 +-
 .../clientpositive/spark/ppd_outer_join4.q.out  |  124 +-
 .../test/results/clientpositive/spark/ptf.q.out |   28 +-
 .../spark/ql_rewrite_gbtoidx_cbo_1.q.out        |  208 +-
 .../clientpositive/spark/router_join_ppr.q.out  |  442 +-
 .../results/clientpositive/spark/semijoin.q.out |   26 +-
 .../results/clientpositive/spark/skewjoin.q.out |  206 +-
 .../clientpositive/spark/skewjoin_noskew.q.out  |   32 +-
 .../results/clientpositive/spark/stats1.q.out   |    4 +-
 .../results/clientpositive/spark/stats10.q.out  |    5 -
 .../results/clientpositive/spark/stats12.q.out  |    9 -
 .../results/clientpositive/spark/stats13.q.out  |   10 -
 .../results/clientpositive/spark/stats14.q.out  |    7 -
 .../results/clientpositive/spark/stats15.q.out  |    7 -
 .../results/clientpositive/spark/stats16.q.out  |    2 -
 .../results/clientpositive/spark/stats18.q.out  |    4 -
 .../results/clientpositive/spark/stats2.q.out   |    2 -
 .../results/clientpositive/spark/stats20.q.out  |    2 -
 .../results/clientpositive/spark/stats3.q.out   |    2 -
 .../results/clientpositive/spark/stats5.q.out   |    1 -
 .../results/clientpositive/spark/stats6.q.out   |    9 -
 .../results/clientpositive/spark/stats7.q.out   |    5 -
 .../results/clientpositive/spark/stats8.q.out   |   18 -
 .../results/clientpositive/spark/stats9.q.out   |    1 -
 .../clientpositive/spark/stats_counter.q.out    |    2 -
 .../spark/stats_counter_partitioned.q.out       |   16 -
 .../clientpositive/spark/stats_noscan_1.q.out   |   17 -
 .../clientpositive/spark/stats_noscan_2.q.out   |    6 -
 .../clientpositive/spark/stats_only_null.q.out  |   12 +-
 .../spark/stats_partscan_1_23.q.out             |    6 -
 .../results/clientpositive/spark/statsfs.q.out  |   14 -
 .../clientpositive/spark/subquery_exists.q.out  |   20 +-
 .../clientpositive/spark/subquery_in.q.out      |  230 +-
 .../spark/subquery_multiinsert.q.java1.8.out    |   14 +-
 .../spark/table_access_keys_stats.q.out         |   18 +-
 .../clientpositive/spark/temp_table.q.out       |    4 +-
 .../results/clientpositive/spark/union.q.out    |    4 +-
 .../results/clientpositive/spark/union10.q.out  |    6 +-
 .../results/clientpositive/spark/union11.q.out  |   82 +-
 .../results/clientpositive/spark/union12.q.out  |    2 +-
 .../results/clientpositive/spark/union13.q.out  |    2 +-
 .../results/clientpositive/spark/union14.q.out  |   52 +-
 .../results/clientpositive/spark/union15.q.out  |   80 +-
 .../results/clientpositive/spark/union2.q.out   |    2 +-
 .../results/clientpositive/spark/union20.q.out  |   10 +-
 .../results/clientpositive/spark/union22.q.out  |    4 +-
 .../results/clientpositive/spark/union24.q.out  |   36 +-
 .../results/clientpositive/spark/union25.q.out  |   38 +-
 .../results/clientpositive/spark/union28.q.out  |   82 +-
 .../results/clientpositive/spark/union29.q.out  |    4 +-
 .../results/clientpositive/spark/union30.q.out  |   82 +-
 .../results/clientpositive/spark/union33.q.out  |   16 +-
 .../results/clientpositive/spark/union4.q.out   |    4 +-
 .../results/clientpositive/spark/union5.q.out   |   54 +-
 .../results/clientpositive/spark/union6.q.out   |    2 +-
 .../results/clientpositive/spark/union7.q.out   |   52 +-
 .../results/clientpositive/spark/union8.q.out   |    4 +-
 .../results/clientpositive/spark/union9.q.out   |    4 +-
 .../clientpositive/spark/union_date_trim.q.out  |    4 +-
 .../clientpositive/spark/union_remove_1.q.out   |   13 +-
 .../clientpositive/spark/union_remove_10.q.out  |   11 +-
 .../clientpositive/spark/union_remove_11.q.out  |    3 +-
 .../clientpositive/spark/union_remove_12.q.out  |    3 +-
 .../clientpositive/spark/union_remove_13.q.out  |    1 -
 .../clientpositive/spark/union_remove_14.q.out  |    3 +-
 .../clientpositive/spark/union_remove_15.q.out  |   17 +-
 .../clientpositive/spark/union_remove_16.q.out  |   17 +-
 .../clientpositive/spark/union_remove_17.q.out  |    5 +-
 .../clientpositive/spark/union_remove_18.q.out  |   33 +-
 .../clientpositive/spark/union_remove_19.q.out  |  205 +-
 .../clientpositive/spark/union_remove_2.q.out   |   11 +-
 .../clientpositive/spark/union_remove_20.q.out  |   13 +-
 .../clientpositive/spark/union_remove_21.q.out  |   73 +-
 .../clientpositive/spark/union_remove_22.q.out  |   21 +-
 .../clientpositive/spark/union_remove_23.q.out  |    3 +-
 .../clientpositive/spark/union_remove_24.q.out  |   13 +-
 .../clientpositive/spark/union_remove_25.q.out  |   30 +-
 .../clientpositive/spark/union_remove_3.q.out   |    3 +-
 .../clientpositive/spark/union_remove_4.q.out   |   13 +-
 .../clientpositive/spark/union_remove_5.q.out   |   11 +-
 .../clientpositive/spark/union_remove_6.q.out   |    8 +-
 .../spark/union_remove_6_subq.q.out             |   16 +-
 .../clientpositive/spark/union_remove_7.q.out   |   13 +-
 .../clientpositive/spark/union_remove_8.q.out   |   11 +-
 .../clientpositive/spark/union_remove_9.q.out   |    7 +-
 .../clientpositive/spark/union_top_level.q.out  |  382 +-
 .../spark/vector_cast_constant.q.java1.7.out    |    6 +-
 .../spark/vector_cast_constant.q.java1.8.out    |    6 +-
 .../spark/vector_decimal_aggregate.q.out        |   12 +-
 .../spark/vector_distinct_2.q.out               |    4 +-
 .../clientpositive/spark/vector_elt.q.out       |    2 +-
 .../clientpositive/spark/vector_groupby_3.q.out |    6 +-
 .../spark/vector_mapjoin_reduce.q.out           |  261 +-
 .../clientpositive/spark/vector_orderby_5.q.out |    6 +-
 .../spark/vector_string_concat.q.out            |    6 +-
 .../clientpositive/spark/vectorization_0.q.out  |   28 +-
 .../clientpositive/spark/vectorization_1.q.out  |    8 +-
 .../clientpositive/spark/vectorization_10.q.out |  428 +-
 .../clientpositive/spark/vectorization_11.q.out |   74 +-
 .../clientpositive/spark/vectorization_12.q.out |  994 +--
 .../clientpositive/spark/vectorization_13.q.out |  430 +-
 .../clientpositive/spark/vectorization_14.q.out | 1182 +--
 .../clientpositive/spark/vectorization_15.q.out |   68 +-
 .../clientpositive/spark/vectorization_16.q.out |    8 +-
 .../clientpositive/spark/vectorization_17.q.out |  507 ++
 .../clientpositive/spark/vectorization_2.q.out  |    8 +-
 .../clientpositive/spark/vectorization_3.q.out  |    8 +-
 .../clientpositive/spark/vectorization_4.q.out  |    8 +-
 .../clientpositive/spark/vectorization_5.q.out  |    8 +-
 .../clientpositive/spark/vectorization_6.q.out  | 3042 ++++----
 .../clientpositive/spark/vectorization_7.q.out  |  328 +-
 .../clientpositive/spark/vectorization_8.q.out  |  314 +-
 .../clientpositive/spark/vectorization_9.q.out  |  965 ++-
 .../spark/vectorization_div0.q.out              |    6 +-
 .../spark/vectorization_part_project.q.out      |    2 +-
 .../spark/vectorization_pushdown.q.out          |    8 +-
 .../spark/vectorization_short_regress.q.out     | 1330 ++--
 .../clientpositive/spark/vectorized_case.q.out  |    2 +-
 .../spark/vectorized_mapjoin.q.out              |   60 +-
 .../clientpositive/spark/vectorized_ptf.q.out   |  136 +-
 .../spark/vectorized_shufflejoin.q.out          |   52 +-
 .../spark/vectorized_timestamp_funcs.q.out      |   12 +-
 .../clientpositive/spark/windowing.q.out        |   44 +-
 ql/src/test/results/clientpositive/stats1.q.out |    2 -
 .../test/results/clientpositive/stats10.q.out   |    5 -
 .../test/results/clientpositive/stats11.q.out   |    8 -
 .../test/results/clientpositive/stats12.q.out   |    9 -
 .../test/results/clientpositive/stats13.q.out   |   10 -
 .../test/results/clientpositive/stats14.q.out   |    7 -
 .../test/results/clientpositive/stats15.q.out   |    7 -
 .../test/results/clientpositive/stats16.q.out   |    2 -
 .../test/results/clientpositive/stats18.q.out   |    4 -
 .../test/results/clientpositive/stats19.q.out   |   12 -
 ql/src/test/results/clientpositive/stats2.q.out |    2 -
 .../test/results/clientpositive/stats20.q.out   |    2 -
 ql/src/test/results/clientpositive/stats3.q.out |    2 -
 ql/src/test/results/clientpositive/stats4.q.out |   10 -
 ql/src/test/results/clientpositive/stats5.q.out |    1 -
 ql/src/test/results/clientpositive/stats6.q.out |    9 -
 ql/src/test/results/clientpositive/stats7.q.out |    5 -
 ql/src/test/results/clientpositive/stats8.q.out |   18 -
 ql/src/test/results/clientpositive/stats9.q.out |    1 -
 .../results/clientpositive/stats_counter.q.out  |    2 -
 .../stats_counter_partitioned.q.out             |   16 -
 .../clientpositive/stats_empty_partition.q.out  |    2 -
 .../clientpositive/stats_invalidation.q.out     |    2 -
 .../stats_list_bucket.q.java1.7.out             |    3 -
 .../stats_list_bucket.q.java1.8.out             |    3 -
 .../results/clientpositive/stats_noscan_1.q.out |   17 -
 .../results/clientpositive/stats_noscan_2.q.out |    6 -
 .../clientpositive/stats_only_null.q.out        |    4 -
 .../clientpositive/stats_partscan_1.q.out       |    6 -
 .../clientpositive/stats_partscan_1_23.q.out    |    6 -
 .../test/results/clientpositive/statsfs.q.out   |   14 -
 .../clientpositive/subquery_in_having.q.out     |  128 +-
 .../subquery_notin_having.q.java1.7.out         |   34 +-
 .../subquery_unqualcolumnrefs.q.out             |   32 +-
 .../temp_table_windowing_expressions.q.out      |    8 +-
 .../clientpositive/tez/alter_merge_orc.q.out    |   24 +-
 .../tez/alter_merge_stats_orc.q.out             |   30 +-
 .../clientpositive/tez/cbo_subq_in.q.out        |    2 +
 .../clientpositive/tez/cbo_windowing.q.out      |   88 +-
 .../clientpositive/tez/constprog_dpp.q.out      |  113 +
 .../results/clientpositive/tez/cross_join.q.out |    8 +-
 .../test/results/clientpositive/tez/ctas.q.out  |    5 -
 .../tez/dynpart_sort_opt_vectorization.q.out    |   48 +-
 .../tez/dynpart_sort_optimization.q.out         |   32 -
 .../tez/dynpart_sort_optimization2.q.out        |   32 +-
 .../clientpositive/tez/explainuser_1.q.out      | 1999 ++---
 .../clientpositive/tez/explainuser_2.q.out      | 3438 ++++-----
 .../clientpositive/tez/explainuser_3.q.out      |  522 ++
 .../results/clientpositive/tez/groupby2.q.out   |   17 +-
 .../results/clientpositive/tez/having.q.out     |   28 +-
 .../results/clientpositive/tez/insert1.q.out    |  392 +-
 .../clientpositive/tez/limit_pushdown.q.out     |   78 +-
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |  266 +-
 .../test/results/clientpositive/tez/mrr.q.out   |   82 +-
 .../clientpositive/tez/orc_analyze.q.out        |   48 -
 .../results/clientpositive/tez/orc_merge9.q.out |  186 +
 .../test/results/clientpositive/tez/ptf.q.out   |   28 +-
 .../clientpositive/tez/selectDistinctStar.q.out |    2 -
 .../results/clientpositive/tez/skewjoin.q.out   |   26 +-
 .../clientpositive/tez/stats_counter.q.out      |    2 -
 .../tez/stats_counter_partitioned.q.out         |   16 -
 .../clientpositive/tez/stats_noscan_1.q.out     |   17 -
 .../clientpositive/tez/stats_only_null.q.out    |    4 -
 .../tez/tez_dynpart_hashjoin_1.q.out            |  791 ++
 .../tez/tez_dynpart_hashjoin_2.q.out            |  564 ++
 .../results/clientpositive/tez/tez_fsstat.q.out |    2 -
 .../tez/tez_join_result_complex.q.out           | 2159 ++++++
 .../clientpositive/tez/tez_self_join.q.out      |  205 +
 .../results/clientpositive/tez/tez_smb_1.q.out  |  278 +
 .../results/clientpositive/tez/tez_union.q.out  |   88 +-
 .../tez/tez_union_dynamic_partition.q.out       |  156 +
 .../tez/tez_union_multiinsert.q.out             |    4 +-
 .../tez/tez_vector_dynpart_hashjoin_1.q.out     |  804 ++
 .../tez/tez_vector_dynpart_hashjoin_2.q.out     |  570 ++
 .../results/clientpositive/tez/union4.q.out     |    4 +
 .../results/clientpositive/tez/union6.q.out     |    4 +
 .../clientpositive/tez/unionDistinct_1.q.out    |   20 +-
 .../clientpositive/tez/update_all_types.q.out   |   48 +-
 .../clientpositive/tez/update_orig_table.q.out  |   17 +-
 .../clientpositive/tez/vector_coalesce.q.out    |  378 +-
 .../tez/vector_count_distinct.q.out             |    1 +
 .../clientpositive/tez/vector_decimal_2.q.out   |   55 +-
 .../tez/vector_decimal_expressions.q.out        |   99 +-
 .../tez/vector_decimal_round_2.q.out            |    8 +-
 .../tez/vector_groupby_reduce.q.out             |   10 +-
 .../clientpositive/tez/vector_interval_1.q.out  |   24 +-
 .../tez/vector_leftsemi_mapjoin.q.out           | 1032 +--
 .../tez/vector_multi_insert.q.out               |   20 +-
 .../clientpositive/tez/vector_outer_join1.q.out |  252 +-
 .../clientpositive/tez/vector_outer_join2.q.out |  122 +-
 .../clientpositive/tez/vector_outer_join3.q.out |  164 +-
 .../clientpositive/tez/vector_outer_join4.q.out | 1060 +--
 .../tez/vector_partitioned_date_time.q.out      |   10 +-
 .../clientpositive/tez/vectorization_0.q.out    |    8 +-
 .../clientpositive/tez/vectorization_1.q.out    |    8 +-
 .../clientpositive/tez/vectorization_10.q.out   |  428 +-
 .../clientpositive/tez/vectorization_11.q.out   |   74 +-
 .../clientpositive/tez/vectorization_12.q.out   |  994 +--
 .../clientpositive/tez/vectorization_13.q.out   |  414 +-
 .../clientpositive/tez/vectorization_14.q.out   | 1170 +--
 .../clientpositive/tez/vectorization_15.q.out   |   58 +-
 .../clientpositive/tez/vectorization_17.q.out   |  507 ++
 .../clientpositive/tez/vectorization_2.q.out    |    8 +-
 .../clientpositive/tez/vectorization_3.q.out    |    8 +-
 .../clientpositive/tez/vectorization_4.q.out    |    8 +-
 .../clientpositive/tez/vectorization_5.q.out    |    8 +-
 .../clientpositive/tez/vectorization_6.q.out    | 3042 ++++----
 .../clientpositive/tez/vectorization_7.q.out    |  320 +-
 .../clientpositive/tez/vectorization_8.q.out    |  306 +-
 .../clientpositive/tez/vectorization_9.q.out    |  957 ++-
 .../tez/vectorization_short_regress.q.out       | 1224 ++--
 .../clientpositive/tez/vectorized_casts.q.out   |   42 +-
 .../tez/vectorized_date_funcs.q.out             |  272 +-
 .../tez/vectorized_distinct_gby.q.out           |    1 +
 .../vectorized_dynamic_partition_pruning.q.out  |    8 +
 .../clientpositive/tez/vectorized_ptf.q.out     |  136 +-
 .../tez_join_result_complex.q.out               | 1337 ++++
 .../clientpositive/truncate_column.q.out        |   11 -
 .../udaf_percentile_approx_23.q.out             |   10 +-
 .../results/clientpositive/udtf_stack.q.out     |   12 +-
 .../clientpositive/unicode_notation.q.out       |    3 -
 .../test/results/clientpositive/union22.q.out   |    4 +-
 .../test/results/clientpositive/union28.q.out   |    4 +-
 .../test/results/clientpositive/union29.q.out   |    4 +-
 .../test/results/clientpositive/union30.q.out   |    4 +-
 .../test/results/clientpositive/union33.q.out   |    4 +-
 .../clientpositive/unionDistinct_1.q.out        |   20 +-
 .../clientpositive/union_date_trim.q.out        |    4 +-
 .../results/clientpositive/union_remove_1.q.out |    5 +-
 .../clientpositive/union_remove_10.q.out        |    3 +-
 .../clientpositive/union_remove_11.q.out        |    3 +-
 .../clientpositive/union_remove_12.q.out        |    3 +-
 .../clientpositive/union_remove_13.q.out        |    1 -
 .../clientpositive/union_remove_14.q.out        |    3 +-
 .../clientpositive/union_remove_15.q.out        |    9 +-
 .../clientpositive/union_remove_16.q.out        |    9 +-
 .../clientpositive/union_remove_17.q.out        |    5 +-
 .../clientpositive/union_remove_18.q.out        |   25 +-
 .../clientpositive/union_remove_19.q.out        |   13 +-
 .../results/clientpositive/union_remove_2.q.out |    3 +-
 .../clientpositive/union_remove_20.q.out        |    5 +-
 .../clientpositive/union_remove_21.q.out        |    3 +-
 .../clientpositive/union_remove_22.q.out        |   13 +-
 .../clientpositive/union_remove_23.q.out        |    3 +-
 .../clientpositive/union_remove_24.q.out        |    5 +-
 .../clientpositive/union_remove_25.q.out        |   22 +-
 .../results/clientpositive/union_remove_3.q.out |    3 +-
 .../results/clientpositive/union_remove_4.q.out |    5 +-
 .../results/clientpositive/union_remove_5.q.out |    3 +-
 .../results/clientpositive/union_remove_6.q.out |    8 +-
 .../clientpositive/union_remove_6_subq.q.out    |    8 +-
 .../results/clientpositive/union_remove_7.q.out |    5 +-
 .../results/clientpositive/union_remove_8.q.out |    3 +-
 .../results/clientpositive/union_remove_9.q.out |    3 +-
 .../clientpositive/union_top_level.q.out        |  316 +-
 .../clientpositive/update_all_types.q.out       |   48 +-
 .../clientpositive/update_orig_table.q.out      |   17 +-
 .../vector_aggregate_without_gby.q.out          |   96 +
 .../clientpositive/vector_coalesce.q.out        |  286 +-
 .../clientpositive/vector_decimal_2.q.out       |   55 +-
 .../vector_decimal_expressions.q.out            |   73 +-
 .../clientpositive/vector_decimal_round_2.q.out |    8 +-
 .../clientpositive/vector_groupby_reduce.q.out  |   10 +-
 .../clientpositive/vector_interval_1.q.out      |   24 +-
 .../clientpositive/vector_outer_join1.q.out     |  244 +-
 .../clientpositive/vector_outer_join2.q.out     |  118 +-
 .../clientpositive/vector_outer_join3.q.out     |  152 +-
 .../clientpositive/vector_outer_join4.q.out     | 1052 +--
 .../vector_partitioned_date_time.q.out          |   10 +-
 .../vector_partitioned_date_time_win.q.out      | 2036 ++++++
 .../clientpositive/vectorization_0.q.out        |    8 +-
 .../clientpositive/vectorization_1.q.out        |    8 +-
 .../clientpositive/vectorization_10.q.out       |  428 +-
 .../clientpositive/vectorization_11.q.out       |   74 +-
 .../clientpositive/vectorization_12.q.out       |  994 +--
 .../clientpositive/vectorization_13.q.out       |  414 +-
 .../clientpositive/vectorization_14.q.out       | 1170 +--
 .../clientpositive/vectorization_15.q.out       |   58 +-
 .../clientpositive/vectorization_17.q.out       |  500 ++
 .../clientpositive/vectorization_2.q.out        |    8 +-
 .../clientpositive/vectorization_3.q.out        |    8 +-
 .../clientpositive/vectorization_4.q.out        |    8 +-
 .../clientpositive/vectorization_5.q.out        |    8 +-
 .../clientpositive/vectorization_6.q.out        | 3042 ++++----
 .../clientpositive/vectorization_7.q.out        |  312 +-
 .../clientpositive/vectorization_8.q.out        |  298 +-
 .../clientpositive/vectorization_9.q.out        |  957 ++-
 .../vectorization_part_varchar.q.out            |   72 +
 .../vectorization_short_regress.q.out           | 1224 ++--
 .../clientpositive/vectorized_casts.q.out       |   42 +-
 .../clientpositive/vectorized_date_funcs.q.out  |  272 +-
 .../results/clientpositive/vectorized_ptf.q.out |  132 +-
 .../test/results/clientpositive/windowing.q.out |   44 +-
 .../clientpositive/windowing_expressions.q.out  |    8 +-
 .../clientpositive/windowing_windowspec3.q.out  |  232 +
 serde/pom.xml                                   |    4 +-
 .../hadoop/hive/ql/io/sarg/ExpressionTree.java  |  157 +
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  |   14 +-
 .../hive/serde2/AbstractEncodingAwareSerDe.java |    7 +-
 .../hive/serde2/ColumnProjectionUtils.java      |   29 +-
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |    4 +-
 .../hive/serde2/avro/AvroDeserializer.java      |   11 +-
 .../hadoop/hive/serde2/io/DateWritable.java     |   14 -
 .../hive/serde2/io/TimestampWritable.java       |    2 +-
 .../hive/serde2/lazybinary/LazyBinaryDate.java  |    4 +-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java |   11 +-
 .../objectinspector/ObjectInspectorUtils.java   |   18 +-
 .../hive/serde2/avro/TestAvroDeserializer.java  |   63 +-
 .../avro/TestAvroObjectInspectorGenerator.java  |   33 +
 .../hive/serde2/io/TestTimestampWritable.java   |    2 +-
 .../TestObjectInspectorUtils.java               |   22 +
 service/pom.xml                                 |    2 +-
 .../auth/LdapAuthenticationProviderImpl.java    |  460 +-
 .../service/auth/TSetIpAddressProcessor.java    |    4 +-
 .../org/apache/hive/service/cli/CLIService.java |   13 -
 .../service/cli/operation/SQLOperation.java     |   16 +-
 .../hive/service/cli/session/HiveSession.java   |   12 +-
 .../service/cli/session/HiveSessionBase.java    |    2 -
 .../service/cli/session/HiveSessionImpl.java    |  100 +-
 .../cli/session/HiveSessionImplwithUGI.java     |   84 +-
 .../service/cli/session/SessionManager.java     |    6 +-
 .../service/cli/thrift/ThriftCLIService.java    |   25 +-
 .../service/cli/thrift/ThriftHttpServlet.java   |   18 +-
 .../apache/hive/service/server/HiveServer2.java |   15 +-
 .../apache/hive/service/cli/CLIServiceTest.java |  135 +-
 shims/0.20S/pom.xml                             |    2 +-
 .../apache/hadoop/mapred/WebHCatJTShim20S.java  |   10 +
 shims/0.23/pom.xml                              |    2 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |   11 +-
 .../apache/hadoop/mapred/WebHCatJTShim23.java   |   18 +-
 shims/aggregator/pom.xml                        |    2 +-
 shims/common/pom.xml                            |    2 +-
 .../apache/hadoop/hive/shims/HadoopShims.java   |    5 +
 .../hive/thrift/HadoopThriftAuthBridge.java     |    6 +-
 shims/pom.xml                                   |    2 +-
 shims/scheduler/pom.xml                         |    2 +-
 spark-client/pom.xml                            |    4 +-
 testutils/pom.xml                               |    2 +-
 1649 files changed, 95443 insertions(+), 36216 deletions(-)
----------------------------------------------------------------------



[04/52] [abbrv] hive git commit: HIVE-11343 Merge branch 'master' into hbase-metastore

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/itests/hive-unit/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index e5a32fb,39ab9e7..d2177a5
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@@ -299,8 -338,27 +339,28 @@@ public class ObjectStore implements Raw
          " created in the thread with id: " + Thread.currentThread().getId());
    }
  
+   /**
+    * Creates the proxy used to evaluate expressions. This is here to prevent circular
+    * dependency - ql -&gt; metastore client &lt;-&gt metastore server -&gt ql. If server and
+    * client are split, this can be removed.
+    * @param conf Configuration.
+    * @return The partition expression proxy.
+    */
+   private static PartitionExpressionProxy createExpressionProxy(Configuration conf) {
+     String className = HiveConf.getVar(conf, HiveConf.ConfVars.METASTORE_EXPRESSION_PROXY_CLASS);
+     try {
+       @SuppressWarnings("unchecked")
+       Class<? extends PartitionExpressionProxy> clazz =
+           (Class<? extends PartitionExpressionProxy>)MetaStoreUtils.getClass(className);
+       return MetaStoreUtils.newInstance(
+           clazz, new Class<?>[0], new Object[0]);
+     } catch (MetaException e) {
+       LOG.error("Error loading PartitionExpressionProxy", e);
+       throw new RuntimeException("Error loading PartitionExpressionProxy: " + e.getMessage());
+     }
+   }
  
 +
    /**
     * Properties specified in hive-default.xml override the properties specified
     * in jpox.properties.
@@@ -3258,8 -3426,9 +3350,9 @@@
          if (roleMember.size() > 0) {
            pm.deletePersistentAll(roleMember);
          }
+         queryWrapper.close();
          // then remove all the grants
 -        List<MGlobalPrivilege> userGrants = listPrincipalGlobalGrants(
 +        List<MGlobalPrivilege> userGrants = listPrincipalMGlobalGrants(
              mRol.getRoleName(), PrincipalType.ROLE);
          if (userGrants.size() > 0) {
            pm.deletePersistentAll(userGrants);
@@@ -3344,26 -3521,27 +3445,27 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MRoleMap> listRoles(String principalName, PrincipalType principalType) {
 +  public List<MRoleMap> listMRoles(String principalName,
 +      PrincipalType principalType) {
      boolean success = false;
-     List<MRoleMap> mRoleMember = null;
+     Query query = null;
+     List<MRoleMap> mRoleMember = new ArrayList<MRoleMap>();
+ 
      try {
-       openTransaction();
        LOG.debug("Executing listRoles");
-       Query query = pm
-           .newQuery(
-               MRoleMap.class,
-               "principalName == t1 && principalType == t2");
-       query
-           .declareParameters("java.lang.String t1, java.lang.String t2");
+ 
+       openTransaction();
+       query = pm.newQuery(MRoleMap.class, "principalName == t1 && principalType == t2");
+       query.declareParameters("java.lang.String t1, java.lang.String t2");
        query.setUnique(false);
-       mRoleMember = (List<MRoleMap>) query.executeWithArray(
-           principalName, principalType.toString());
-       LOG.debug("Done executing query for listMSecurityUserRoleMap");
-       pm.retrieveAll(mRoleMember);
+       List<MRoleMap> mRoles =
+           (List<MRoleMap>) query.executeWithArray(principalName, principalType.toString());
+       pm.retrieveAll(mRoles);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityUserRoleMap");
+ 
+       mRoleMember.addAll(mRoles);
+ 
+       LOG.debug("Done retrieving all objects for listRoles");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -3372,60 -3553,18 +3477,56 @@@
  
      if (principalType == PrincipalType.USER) {
        // All users belong to public role implicitly, add that role
-       if (mRoleMember == null) {
-         mRoleMember = new ArrayList<MRoleMap>();
-       } else {
-         mRoleMember = new ArrayList<MRoleMap>(mRoleMember);
-       }
        MRole publicRole = new MRole(HiveMetaStore.PUBLIC, 0, HiveMetaStore.PUBLIC);
-       mRoleMember.add(new MRoleMap(principalName, principalType.toString(), publicRole, 0,
-           null, null, false));
+       mRoleMember.add(new MRoleMap(principalName, principalType.toString(), publicRole, 0, null,
+           null, false));
      }
-     return mRoleMember;
  
+     return mRoleMember;
    }
  
 +  @Override
 +  public List<Role> listRoles(String principalName, PrincipalType principalType) {
 +    List<Role> result = new ArrayList<Role>();
 +    List<MRoleMap> roleMaps = listMRoles(principalName, principalType);
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        MRole mrole = roleMap.getRole();
 +        Role role = new Role(mrole.getRoleName(), mrole.getCreateTime(), mrole.getOwnerName());
 +        result.add(role);
 +      }
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public List<RolePrincipalGrant> listRolesWithGrants(String principalName,
 +                                                      PrincipalType principalType) {
 +    List<RolePrincipalGrant> result = new ArrayList<RolePrincipalGrant>();
 +    List<MRoleMap> roleMaps = listMRoles(principalName, principalType);
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        RolePrincipalGrant rolePrinGrant = new RolePrincipalGrant(
 +            roleMap.getRole().getRoleName(),
 +            roleMap.getPrincipalName(),
 +            PrincipalType.valueOf(roleMap.getPrincipalType()),
 +            roleMap.getGrantOption(),
 +            roleMap.getAddTime(),
 +            roleMap.getGrantor(),
 +            // no grantor type for public role, hence the null check
 +            roleMap.getGrantorType() == null ? null
 +                : PrincipalType.valueOf(roleMap.getGrantorType())
 +        );
 +        result.add(rolePrinGrant);
 +      }
 +    }
 +    return result;
 +  }
 +
    @SuppressWarnings("unchecked")
    private List<MRoleMap> listMSecurityPrincipalMembershipRole(final String roleName,
-       final PrincipalType principalType) {
+       final PrincipalType principalType,
+       QueryWrapper queryWrapper) {
      boolean success = false;
      List<MRoleMap> mRoleMemebership = null;
      try {
@@@ -4305,22 -4448,25 +4411,24 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MRoleMap> listRoleMembers(String roleName) {
 +  public List<MRoleMap> listMRoleMembers(String roleName) {
      boolean success = false;
-     List<MRoleMap> mRoleMemeberList = null;
+     Query query = null;
+     List<MRoleMap> mRoleMemeberList = new ArrayList<MRoleMap>();
      try {
+       LOG.debug("Executing listRoleMembers");
+ 
        openTransaction();
-       LOG.debug("Executing listMSecurityUserRoleMember");
-       Query query = pm.newQuery(MRoleMap.class,
-           "role.roleName == t1");
+       query = pm.newQuery(MRoleMap.class, "role.roleName == t1");
        query.declareParameters("java.lang.String t1");
        query.setUnique(false);
-       mRoleMemeberList = (List<MRoleMap>) query.execute(
-           roleName);
-       LOG.debug("Done executing query for listMSecurityUserRoleMember");
-       pm.retrieveAll(mRoleMemeberList);
+       List<MRoleMap> mRoles = (List<MRoleMap>) query.execute(roleName);
+       pm.retrieveAll(mRoles);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityUserRoleMember");
+ 
+       mRoleMemeberList.addAll(mRoles);
+ 
+       LOG.debug("Done retrieving all objects for listRoleMembers");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -4329,48 -4478,27 +4440,51 @@@
      return mRoleMemeberList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MGlobalPrivilege> listPrincipalGlobalGrants(String principalName,
 -      PrincipalType principalType) {
 +  public List<RolePrincipalGrant> listRoleMembers(String roleName) {
 +    List<MRoleMap> roleMaps = listMRoleMembers(roleName);
 +    List<RolePrincipalGrant> rolePrinGrantList = new ArrayList<RolePrincipalGrant>();
 +
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        RolePrincipalGrant rolePrinGrant = new RolePrincipalGrant(
 +            roleMap.getRole().getRoleName(),
 +            roleMap.getPrincipalName(),
 +            PrincipalType.valueOf(roleMap.getPrincipalType()),
 +            roleMap.getGrantOption(),
 +            roleMap.getAddTime(),
 +            roleMap.getGrantor(),
 +            // no grantor type for public role, hence the null check
 +            roleMap.getGrantorType() == null ? null
 +                : PrincipalType.valueOf(roleMap.getGrantorType())
 +        );
 +        rolePrinGrantList.add(rolePrinGrant);
 +
 +      }
 +    }
 +    return rolePrinGrantList;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MGlobalPrivilege> listPrincipalMGlobalGrants(String principalName,
 +                                                           PrincipalType principalType) {
      boolean commited = false;
-     List<MGlobalPrivilege> userNameDbPriv = null;
+     Query query = null;
+     List<MGlobalPrivilege> userNameDbPriv = new ArrayList<MGlobalPrivilege>();
      try {
+       List<MGlobalPrivilege> mPrivs = null;
        openTransaction();
        if (principalName != null) {
-         Query query = pm.newQuery(MGlobalPrivilege.class,
-             "principalName == t1 && principalType == t2 ");
-         query.declareParameters(
-             "java.lang.String t1, java.lang.String t2");
-         userNameDbPriv = (List<MGlobalPrivilege>) query
-             .executeWithArray(principalName, principalType.toString());
-         pm.retrieveAll(userNameDbPriv);
+         query = pm.newQuery(MGlobalPrivilege.class, "principalName == t1 && principalType == t2 ");
+         query.declareParameters("java.lang.String t1, java.lang.String t2");
+         mPrivs = (List<MGlobalPrivilege>) query
+                 .executeWithArray(principalName, principalType.toString());
+         pm.retrieveAll(mPrivs);
        }
        commited = commitTransaction();
+       if (mPrivs != null) {
+         userNameDbPriv.addAll(mPrivs);
+       }
      } finally {
        if (!commited) {
          rollbackTransaction();
@@@ -4380,34 -4511,12 +4497,35 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalGlobalGrants(String principalName,
 +                                                             PrincipalType principalType) {
 +    List<MGlobalPrivilege> mUsers =
 +        listPrincipalMGlobalGrants(principalName, principalType);
 +    if (mUsers.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mUsers.size(); i++) {
 +      MGlobalPrivilege sUsr = mUsers.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.GLOBAL, null, null, null, null);
 +      HiveObjectPrivilege secUser = new HiveObjectPrivilege(
 +          objectRef, sUsr.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sUsr.getPrivilege(), sUsr
 +              .getCreateTime(), sUsr.getGrantor(), PrincipalType
 +              .valueOf(sUsr.getGrantorType()), sUsr.getGrantOption()));
 +      result.add(secUser);
 +    }
 +    return result;
 +  }
 +
 +  @Override
    public List<HiveObjectPrivilege> listGlobalGrantsAll() {
      boolean commited = false;
+     Query query = null;
      try {
        openTransaction();
-       Query query = pm.newQuery(MGlobalPrivilege.class);
+       query = pm.newQuery(MGlobalPrivilege.class);
        List<MGlobalPrivilege> userNameDbPriv = (List<MGlobalPrivilege>) query.execute();
        pm.retrieveAll(userNameDbPriv);
        commited = commitTransaction();
@@@ -4435,23 -4547,28 +4556,27 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MDBPrivilege> listPrincipalDBGrants(String principalName,
 +  public List<MDBPrivilege> listPrincipalMDBGrants(String principalName,
        PrincipalType principalType, String dbName) {
      boolean success = false;
-     List<MDBPrivilege> mSecurityDBList = null;
+     Query query = null;
+     List<MDBPrivilege> mSecurityDBList = new ArrayList<MDBPrivilege>();
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
      try {
-       openTransaction();
        LOG.debug("Executing listPrincipalDBGrants");
-         Query query = pm.newQuery(MDBPrivilege.class,
-             "principalName == t1 && principalType == t2 && database.name == t3");
-         query
-             .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
-         mSecurityDBList = (List<MDBPrivilege>) query.executeWithArray(principalName, principalType.toString(), dbName);
-       LOG.debug("Done executing query for listPrincipalDBGrants");
-       pm.retrieveAll(mSecurityDBList);
+ 
+       openTransaction();
+       query =
+           pm.newQuery(MDBPrivilege.class,
+               "principalName == t1 && principalType == t2 && database.name == t3");
+       query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
+       List<MDBPrivilege> mPrivs =
+           (List<MDBPrivilege>) query.executeWithArray(principalName, principalType.toString(),
+               dbName);
+       pm.retrieveAll(mPrivs);
        success = commitTransaction();
+ 
+       mSecurityDBList.addAll(mPrivs);
        LOG.debug("Done retrieving all objects for listPrincipalDBGrants");
      } finally {
        if (!success) {
@@@ -4462,32 -4582,14 +4590,37 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
 +                                                         PrincipalType principalType,
 +                                                         String dbName) {
 +    List<MDBPrivilege> mDbs = listPrincipalMDBGrants(principalName, principalType, dbName);
 +    if (mDbs.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege>emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mDbs.size(); i++) {
 +      MDBPrivilege sDB = mDbs.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.DATABASE, dbName, null, null, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sDB.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sDB.getPrivilege(), sDB
 +              .getCreateTime(), sDB.getGrantor(), PrincipalType
 +              .valueOf(sDB.getGrantorType()), sDB.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @Override
    public List<HiveObjectPrivilege> listPrincipalDBGrantsAll(
        String principalName, PrincipalType principalType) {
-     return convertDB(listPrincipalAllDBGrant(principalName, principalType));
+     QueryWrapper queryWrapper = new QueryWrapper();
+     try {
+       return convertDB(listPrincipalAllDBGrant(principalName, principalType, queryWrapper));
+     } finally {
+       queryWrapper.close();
+     }
    }
  
    @Override
@@@ -4781,15 -4904,15 +4935,15 @@@
      return new ObjectPair<Query, Object[]>(query, params);
    }
  
 -  @Override
    @SuppressWarnings("unchecked")
 -  public List<MTablePrivilege> listAllTableGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName) {
 +  public List<MTablePrivilege> listAllMTableGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName) {
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
      boolean success = false;
-     List<MTablePrivilege> mSecurityTabPartList = null;
+     Query query = null;
+     List<MTablePrivilege> mSecurityTabPartList = new ArrayList<MTablePrivilege>();
      try {
        openTransaction();
        LOG.debug("Executing listAllTableGrants");
@@@ -4813,57 -4941,35 +4972,60 @@@
      return mSecurityTabPartList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MPartitionPrivilege> listPrincipalPartitionGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String partName) {
 +  public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
 +                                                      PrincipalType principalType,
 +                                                      String dbName,
 +                                                      String tableName) {
 +    List<MTablePrivilege> mTbls =
 +        listAllMTableGrants(principalName, principalType, dbName, tableName);
 +    if (mTbls.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mTbls.size(); i++) {
 +      MTablePrivilege sTbl = mTbls.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.TABLE, dbName, tableName, null, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sTbl.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sTbl.getPrivilege(), sTbl.getCreateTime(), sTbl
 +              .getGrantor(), PrincipalType.valueOf(sTbl
 +              .getGrantorType()), sTbl.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MPartitionPrivilege> listPrincipalMPartitionGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String partName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
-     List<MPartitionPrivilege> mSecurityTabPartList = null;
+     List<MPartitionPrivilege> mSecurityTabPartList = new ArrayList<MPartitionPrivilege>();
      try {
-       openTransaction();
-       LOG.debug("Executing listMSecurityPrincipalPartitionGrant");
-       Query query = pm.newQuery(
-           MPartitionPrivilege.class,
-               "principalName == t1 && principalType == t2 && partition.table.tableName == t3 " +
-               "&& partition.table.database.name == t4 && partition.partitionName == t5");
-       query.declareParameters(
-           "java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4, " +
-           "java.lang.String t5");
-       mSecurityTabPartList = (List<MPartitionPrivilege>) query
-           .executeWithArray(principalName, principalType.toString(), tableName, dbName, partName);
-       LOG.debug("Done executing query for listMSecurityPrincipalPartitionGrant");
+       LOG.debug("Executing listPrincipalPartitionGrants");
  
-       pm.retrieveAll(mSecurityTabPartList);
+       openTransaction();
+       query =
+           pm.newQuery(MPartitionPrivilege.class,
+               "principalName == t1 && principalType == t2 && partition.table.tableName == t3 "
+                   + "&& partition.table.database.name == t4 && partition.partitionName == t5");
+       query
+           .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4, "
+               + "java.lang.String t5");
+       List<MPartitionPrivilege> mPrivs =
+           (List<MPartitionPrivilege>) query.executeWithArray(principalName,
+               principalType.toString(), tableName, dbName, partName);
+       pm.retrieveAll(mPrivs);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityPrincipalPartitionGrant");
+ 
+       mSecurityTabPartList.addAll(mPrivs);
+ 
+       LOG.debug("Done retrieving all objects for listPrincipalPartitionGrants");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -4872,40 -4981,12 +5037,41 @@@
      return mSecurityTabPartList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MTableColumnPrivilege> listPrincipalTableColumnGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String columnName) {
 +  public List<HiveObjectPrivilege> listPrincipalPartitionGrants(String principalName,
 +                                                                PrincipalType principalType,
 +                                                                String dbName,
 +                                                                String tableName,
 +                                                                List<String> partValues,
 +                                                                String partName) {
 +    List<MPartitionPrivilege> mParts = listPrincipalMPartitionGrants(principalName,
 +        principalType, dbName, tableName, partName);
 +    if (mParts.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mParts.size(); i++) {
 +      MPartitionPrivilege sPart = mParts.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.PARTITION, dbName, tableName, partValues, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sPart.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sPart.getPrivilege(), sPart
 +              .getCreateTime(), sPart.getGrantor(), PrincipalType
 +              .valueOf(sPart.getGrantorType()), sPart
 +              .getGrantOption()));
 +
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MTableColumnPrivilege> listPrincipalMTableColumnGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String columnName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
      columnName = HiveStringUtils.normalizeIdentifier(columnName);
@@@ -4935,37 -5022,12 +5107,38 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(String principalName,
 +                                                                  PrincipalType principalType,
 +                                                                  String dbName,
 +                                                                  String tableName,
 +                                                                  String columnName) {
 +    List<MTableColumnPrivilege> mTableCols =
 +        listPrincipalMTableColumnGrants(principalName, principalType, dbName, tableName, columnName);
 +    if (mTableCols.isEmpty()) {
 +      return Collections.emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mTableCols.size(); i++) {
 +      MTableColumnPrivilege sCol = mTableCols.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.COLUMN, dbName, tableName, null, sCol.getColumnName());
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(
 +          objectRef, sCol.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sCol.getPrivilege(), sCol
 +              .getCreateTime(), sCol.getGrantor(), PrincipalType
 +              .valueOf(sCol.getGrantorType()), sCol
 +              .getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
    @SuppressWarnings("unchecked")
 -  public List<MPartitionColumnPrivilege> listPrincipalPartitionColumnGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String partitionName,
 -      String columnName) {
 +  public List<MPartitionColumnPrivilege> listPrincipalMPartitionColumnGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String partitionName, String columnName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
      columnName = HiveStringUtils.normalizeIdentifier(columnName);
@@@ -5001,38 -5063,10 +5174,39 @@@
    }
  
    @Override
 -  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(String principalName,
 -      PrincipalType principalType) {
 +  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(String principalName,
 +                                                                      PrincipalType principalType,
 +                                                                      String dbName,
 +                                                                      String tableName,
 +                                                                      List<String> partValues,
 +                                                                      String partitionName,
 +                                                                      String columnName) {
 +    List<MPartitionColumnPrivilege> mPartitionCols =
 +        listPrincipalMPartitionColumnGrants(principalName, principalType, dbName, tableName,
 +            partitionName, columnName);
 +    if (mPartitionCols.isEmpty()) {
 +      return Collections.emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mPartitionCols.size(); i++) {
 +      MPartitionColumnPrivilege sCol = mPartitionCols.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.COLUMN, dbName, tableName, partValues, sCol.getColumnName());
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sCol.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sCol.getPrivilege(), sCol
 +              .getCreateTime(), sCol.getGrantor(), PrincipalType
 +              .valueOf(sCol.getGrantorType()), sCol.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(
 +      String principalName, PrincipalType principalType) {
      boolean success = false;
+     Query query = null;
      try {
        openTransaction();
        LOG.debug("Executing listPrincipalPartitionColumnGrantsAll");
@@@ -6294,16 -6400,11 +6540,16 @@@
      }.run(true);
    }
  
 -  private List<MPartitionColumnStatistics> getMPartitionColumnStatistics(Table table,
 -      List<String> partNames, List<String> colNames,
 -      QueryWrapper queryWrapper) throws NoSuchObjectException, MetaException {
 +  @Override
 +  public void flushCache() {
 +    // NOP as there's no caching
 +  }
 +
 +  private List<MPartitionColumnStatistics> getMPartitionColumnStatistics(
-       Table table, List<String> partNames, List<String> colNames)
++      Table table, List<String> partNames, List<String> colNames, QueryWrapper queryWrapper)
 +          throws NoSuchObjectException, MetaException {
      boolean committed = false;
-     MPartitionColumnStatistics mStatsObj = null;
+ 
      try {
        openTransaction();
        // We are not going to verify SD for each partition. Just verify for the table.

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 7d763fe,1abf738..e9c054a
--- a/pom.xml
+++ b/pom.xml
@@@ -172,7 -174,7 +174,8 @@@
      <felix.version>2.4.0</felix.version>
      <curator.version>2.6.0</curator.version>
      <jsr305.version>3.0.0</jsr305.version>
 +    <tephra.version>0.4.0</tephra.version>
+     <gson.version>2.2.4</gson.version>
    </properties>
  
    <repositories>

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
----------------------------------------------------------------------


[07/52] [abbrv] hive git commit: HIVE-10950: Unit test against HBase Metastore (Daniel Dai, Vaibhav Gumashta)

Posted by se...@apache.org.
HIVE-10950: Unit test against HBase Metastore (Daniel Dai, Vaibhav Gumashta)


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

Branch: refs/heads/llap
Commit: 5acf458c449bd2464076c243150760797ae57092
Parents: cb37021
Author: Vaibhav Gumashta <vg...@apache.org>
Authored: Fri Jul 24 09:13:02 2015 -0700
Committer: Vaibhav Gumashta <vg...@apache.org>
Committed: Fri Jul 24 09:13:02 2015 -0700

----------------------------------------------------------------------
 data/conf/hbase/hive-site.xml                   | 263 +++++++++++++++++++
 itests/qtest/pom.xml                            |  19 ++
 .../test/resources/testconfiguration.properties |  46 ++++
 itests/util/pom.xml                             |  12 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |  72 ++++-
 .../hive/metastore/hbase/HBaseReadWrite.java    |   8 +-
 6 files changed, 411 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5acf458c/data/conf/hbase/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/hbase/hive-site.xml b/data/conf/hbase/hive-site.xml
new file mode 100644
index 0000000..2cde40f
--- /dev/null
+++ b/data/conf/hbase/hive-site.xml
@@ -0,0 +1,263 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<configuration>
+
+<property>
+  <name>hive.in.test</name>
+  <value>true</value>
+  <description>Internal marker for test. Used for masking env-dependent values</description>
+</property>
+
+<!-- Hive Configuration can either be stored in this file or in the hadoop configuration files  -->
+<!-- that are implied by Hadoop setup variables.                                                -->
+<!-- Aside from Hadoop setup variables - this file is provided as a convenience so that Hive    -->
+<!-- users do not have to edit hadoop configuration files (that may be managed as a centralized -->
+<!-- resource).                                                                                 -->
+
+<!-- Hive Execution Parameters -->
+<property>
+  <name>hadoop.tmp.dir</name>
+  <value>${test.tmp.dir}/hadoop-tmp</value>
+  <description>A base for other temporary directories.</description>
+</property>
+
+<!--
+<property>
+  <name>hive.exec.reducers.max</name>
+  <value>1</value>
+  <description>maximum number of reducers</description>
+</property>
+-->
+
+<property>
+  <name>hive.exec.scratchdir</name>
+  <value>${test.tmp.dir}/scratchdir</value>
+  <description>Scratch space for Hive jobs</description>
+</property>
+
+<property>
+  <name>hive.exec.local.scratchdir</name>
+  <value>${test.tmp.dir}/localscratchdir/</value>
+  <description>Local scratch space for Hive jobs</description>
+</property>
+
+<property>
+  <name>javax.jdo.option.ConnectionURL</name>
+  <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
+</property>
+
+<property>
+  <name>hive.stats.dbconnectionstring</name>
+  <value>jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true</value>
+</property>
+
+
+<property>
+  <name>javax.jdo.option.ConnectionDriverName</name>
+  <value>org.apache.derby.jdbc.EmbeddedDriver</value>
+</property>
+
+<property>
+  <name>javax.jdo.option.ConnectionUserName</name>
+  <value>APP</value>
+</property>
+
+<property>
+  <name>javax.jdo.option.ConnectionPassword</name>
+  <value>mine</value>
+</property>
+
+<property>
+  <!--  this should eventually be deprecated since the metastore should supply this -->
+  <name>hive.metastore.warehouse.dir</name>
+  <value>${test.warehouse.dir}</value>
+  <description></description>
+</property>
+
+<property>
+  <name>hive.metastore.metadb.dir</name>
+  <value>file://${test.tmp.dir}/metadb/</value>
+  <description>
+  Required by metastore server or if the uris argument below is not supplied
+  </description>
+</property>
+
+<property>
+  <name>test.log.dir</name>
+  <value>${test.tmp.dir}/log/</value>
+  <description></description>
+</property>
+
+<property>
+  <name>test.data.files</name>
+  <value>${hive.root}/data/files</value>
+  <description></description>
+</property>
+
+<property>
+  <name>test.data.scripts</name>
+  <value>${hive.root}/data/scripts</value>
+  <description></description>
+</property>
+
+<property>
+  <name>hive.jar.path</name>
+  <value>${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar</value>
+  <description></description>
+</property>
+
+<property>
+  <name>hive.querylog.location</name>
+  <value>${test.tmp.dir}/tmp</value>
+  <description>Location of the structured hive logs</description>
+</property>
+
+<property>
+  <name>hive.exec.pre.hooks</name>
+  <value>org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables</value>
+  <description>Pre Execute Hook for Tests</description>
+</property>
+
+<property>
+  <name>hive.exec.post.hooks</name>
+  <value>org.apache.hadoop.hive.ql.hooks.PostExecutePrinter</value>
+  <description>Post Execute Hook for Tests</description>
+</property>
+
+<property>
+  <name>hive.support.concurrency</name>
+  <value>false</value>
+  <description>Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks.</description>
+</property>
+
+<property>
+  <key>hive.unlock.numretries</key>
+  <value>2</value>
+  <description>The number of times you want to retry to do one unlock</description>
+</property>
+
+<property>
+  <key>hive.lock.sleep.between.retries</key>
+  <value>2</value>
+  <description>The sleep time (in seconds) between various retries</description>
+</property>
+
+
+<property>
+  <name>fs.pfile.impl</name>
+  <value>org.apache.hadoop.fs.ProxyLocalFileSystem</value>
+  <description>A proxy for local file system used for cross file system testing</description>
+</property>
+
+<property>
+  <name>hive.exec.mode.local.auto</name>
+  <value>false</value>
+  <description>
+    Let hive determine whether to run in local mode automatically
+    Disabling this for tests so that minimr is not affected
+  </description>
+</property>
+
+<property>
+  <name>hive.auto.convert.join</name>
+  <value>false</value>
+  <description>Whether Hive enable the optimization about converting common join into mapjoin based on the input file size</description>
+</property>
+
+<property>
+  <name>hive.ignore.mapjoin.hint</name>
+  <value>false</value>
+  <description>Whether Hive ignores the mapjoin hint</description>
+</property>
+
+<property>
+  <name>hive.input.format</name>
+  <value>org.apache.hadoop.hive.ql.io.CombineHiveInputFormat</value>
+  <description>The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. </description>
+</property>
+
+<property>
+  <name>hive.default.rcfile.serde</name>
+  <value>org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe</value>
+  <description>The default SerDe hive will use for the rcfile format</description>
+</property>
+
+<property>
+  <name>hive.stats.dbclass</name>
+  <value>jdbc:derby</value>
+  <description>The storage for temporary stats generated by tasks. Currently, jdbc, hbase and counter types are supported</description>
+</property>
+
+<property>
+  <name>hive.stats.key.prefix.reserve.length</name>
+  <value>0</value>
+</property>
+
+<property>
+  <name>hive.conf.restricted.list</name>
+  <value>dummy.config.value</value>
+  <description>Using dummy config value above because you cannot override config with empty value</description>
+</property>
+
+<property>
+  <name>hive.exec.submit.local.task.via.child</name>
+  <value>false</value>
+</property>
+
+
+<property>
+  <name>hive.dummyparam.test.server.specific.config.override</name>
+  <value>from.hive-site.xml</value>
+  <description>Using dummy param to test server specific configuration</description>
+</property>
+
+<property>
+  <name>hive.dummyparam.test.server.specific.config.hivesite</name>
+  <value>from.hive-site.xml</value>
+  <description>Using dummy param to test server specific configuration</description>
+</property>
+
+<property>
+  <name>hive.ql.log.PerfLogger.level</name>
+  <value>WARN,DRFA</value>
+  <description>Used to change the perflogger level</description>
+</property>
+
+<property>
+  <name>hive.fetch.task.conversion</name>
+  <value>minimal</value>
+</property>
+
+<property>
+  <name>hive.users.in.admin.role</name>
+  <value>hive_admin_user</value>
+</property>
+
+<property>
+  <name>hive.metastore.fastpath</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>hive.metastore.rawstore.impl</name>
+  <value>org.apache.hadoop.hive.metastore.hbase.HBaseStore</value>
+</property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/5acf458c/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index e195bee..a48772e 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -532,6 +532,25 @@
                   <else>
                   </else>
                 </if>
+                <!-- HBase Metastore -->
+                <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
+                          outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hadoop/hive/cli/"
+                          templatePath="${basedir}/${hive.path.to.root}/ql/src/test/templates/" template="TestCliDriver.vm"
+                          queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientpositive/"
+                          queryFile="${qfile}"
+                          excludeQueryFile="${minimr.query.files},${minitez.query.files},${encrypted.query.files}"
+                          includeQueryFile="${miniHbaseMetastore.query.files}"
+                          queryFileRegex="${qfile_regex}"
+                          clusterMode="${clustermode}"
+                          runDisabled="${run_disabled}"
+                          hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/hbase"
+                          resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/" 
+                          className="TestMiniHBaseMetastoreCliDriver"
+                          logFile="${project.build.directory}/testminihbasemetastoreclidrivergen.log"
+                          logDirectory="${project.build.directory}/qfile-results/clientpositive/"
+                          hadoopVersion="${active.hadoop.version}"
+                          initScript="q_test_init.sql"
+                          cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- Negative Minimr -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"

http://git-wip-us.apache.org/repos/asf/hive/blob/5acf458c/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index fbde465..eb986db 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1141,3 +1141,49 @@ miniSparkOnYarn.query.files=auto_sortmerge_join_16.q,\
   temp_table_external.q,\
   truncate_column_buckets.q,\
   uber_reduce.q
+
+miniHbaseMetastore.query.files=join1.q,\
+join2.q,\
+mapjoin1.q,\
+add_part_multiple.q,\
+annotate_stats_join.q,\
+authorization_parts.q,\
+auto_join1.q,\
+bucket1.q,\
+compute_stats_string.q,\
+create_1.q,\
+groupby1.q,\
+groupby12.q,\
+having.q,\
+innerjoin.q,\
+input_part10.q,\
+input20.q,\
+join1.q,\
+join20.q,\
+leftsemijoin.q,\
+mapjoin1.q,\
+multi_insert_gby.q,\
+orc_create.q,\
+orc_merge1.q,\
+show_roles.q,\
+stats0.q,\
+statsfs.q,\
+temp_table.q,\
+union.q,\
+union10.q,\
+alter_partition_change_col,\
+alter1.q,\
+analyze_tbl_part.q,\
+authorization_1.q,\
+columnstats_part_coltype.q,\
+ctas.q,\
+database.q,\
+drop_partition_with_stats.q,\
+drop_table_with_stats.q,\
+inputddl8.q,\
+order2.q,\
+partition_date.q,\
+partition_multilevels.q,\
+show_partitions.q,\
+sort.q,\
+view.q

http://git-wip-us.apache.org/repos/asf/hive/blob/5acf458c/itests/util/pom.xml
----------------------------------------------------------------------
diff --git a/itests/util/pom.xml b/itests/util/pom.xml
index 0743f01..b0818d6 100644
--- a/itests/util/pom.xml
+++ b/itests/util/pom.xml
@@ -170,6 +170,18 @@
           <groupId>org.apache.hbase</groupId>
           <artifactId>hbase-server</artifactId>
           <version>${hbase.hadoop2.version}</version>
+          <type>test-jar</type>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <type>test-jar</type>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop2.version}</version>
         </dependency>
         <dependency>
           <groupId>org.apache.hbase</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/5acf458c/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 39d5d9e..efdebd7 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql;
 
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT;
 import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
 
 import java.io.BufferedInputStream;
@@ -65,6 +66,11 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hive.cli.CliDriver;
 import org.apache.hadoop.hive.cli.CliSessionState;
@@ -75,7 +81,10 @@ import org.apache.hadoop.hive.common.io.SortPrintStream;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite;
+import org.apache.hadoop.hive.metastore.hbase.TephraHBaseConnection;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -162,6 +171,8 @@ public class QTestUtil {
   public interface SuiteAddTestFunctor {
     public void addTestToSuite(TestSuite suite, Object setup, String tName);
   }
+  private HBaseTestingUtility utility;
+  private boolean snapshotTaken = false;
 
   static {
     for (String srcTable : System.getProperty("test.src.tables", "").trim().split(",")) {
@@ -277,10 +288,6 @@ public class QTestUtil {
       conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED, true);
     }
 
-    // Plug verifying metastore in for testing.
-    conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
-      "org.apache.hadoop.hive.metastore.VerifyingObjectStore");
-
     if (mr != null) {
       assert dfs != null;
 
@@ -342,6 +349,51 @@ public class QTestUtil {
     return "jceks://file" + new Path(keyDir, "test.jks").toUri();
   }
 
+  private void rebuildHBase() throws Exception {
+    HBaseAdmin admin = utility.getHBaseAdmin();
+    if (!snapshotTaken) {
+      for (String tableName : HBaseReadWrite.tableNames) {
+        List<byte[]> families = HBaseReadWrite.columnFamilies.get(tableName);
+        HTableDescriptor desc = new HTableDescriptor(
+            TableName.valueOf(tableName));
+        for (byte[] family : families) {
+          HColumnDescriptor columnDesc = new HColumnDescriptor(family);
+          desc.addFamily(columnDesc);
+        }
+        try {
+          admin.disableTable(tableName);
+          admin.deleteTable(tableName);
+        } catch (IOException e) {
+          System.out.println(e.getMessage());
+        }
+        admin.createTable(desc);
+      }
+    } else {
+      for (String tableName : HBaseReadWrite.tableNames) {
+        admin.disableTable(tableName);
+        admin.restoreSnapshot("snapshot_" + tableName);
+        admin.enableTable(tableName);
+      }
+      try {
+        db.createDatabase(new org.apache.hadoop.hive.metastore.api.Database(
+            DEFAULT_DATABASE_NAME, DEFAULT_DATABASE_COMMENT, new Warehouse(conf)
+                .getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString(), null));
+      } catch (Exception e) {
+        // Ignore if default database already exist
+      }
+      SessionState.get().setCurrentDatabase(DEFAULT_DATABASE_NAME);
+    }
+    admin.close();
+  }
+
+  private void startMiniHBaseCluster() throws Exception {
+    utility = new HBaseTestingUtility();
+    utility.startMiniCluster();
+    conf = new HiveConf(utility.getConfiguration(), Driver.class);
+    rebuildHBase();
+    HBaseReadWrite.getInstance(conf);
+  }
+
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
       String confDir, String hadoopVer, String initScript, String cleanupScript)
     throws Exception {
@@ -351,6 +403,7 @@ public class QTestUtil {
       HiveConf.setHiveSiteLocation(new URL("file://"+ new File(confDir).toURI().getPath() + "/hive-site.xml"));
       System.out.println("Setting hive-site: "+HiveConf.getHiveSiteLocation());
     }
+    startMiniHBaseCluster();
     conf = new HiveConf(Driver.class);
     this.hadoopVer = getHadoopMainVersion(hadoopVer);
     qMap = new TreeMap<String, String>();
@@ -444,6 +497,7 @@ public class QTestUtil {
         sparkSession = null;
       }
     }
+    utility.shutdownMiniCluster();
     if (mr != null) {
       mr.shutdown();
       mr = null;
@@ -731,6 +785,8 @@ public class QTestUtil {
       return;
     }
 
+    rebuildHBase();
+
     clearTablesCreatedDuringTests();
     clearKeysCreatedInTests();
 
@@ -820,6 +876,12 @@ public class QTestUtil {
     cliDriver.processLine(initCommands);
 
     conf.setBoolean("hive.test.init.phase", false);
+
+    HBaseAdmin admin = utility.getHBaseAdmin();
+    for (String tableName : HBaseReadWrite.tableNames) {
+      admin.snapshot("snapshot_" + tableName, tableName);
+    }
+    snapshotTaken = true;
   }
 
   public void init() throws Exception {
@@ -1571,7 +1633,7 @@ public class QTestUtil {
       // close it first.
       SessionState ss = SessionState.get();
       if (ss != null && ss.out != null && ss.out != System.out) {
-	ss.out.close();
+  ss.out.close();
       }
 
       String inSorted = inFileName + SORT_SUFFIX;

http://git-wip-us.apache.org/repos/asf/hive/blob/5acf458c/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index ae73feb..ca1582e 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -71,7 +71,7 @@ import java.util.Set;
 /**
  * Class to manage storing object in and reading them from HBase.
  */
-class HBaseReadWrite {
+public class HBaseReadWrite {
 
   @VisibleForTesting final static String AGGR_STATS_TABLE = "HBMS_AGGR_STATS";
   @VisibleForTesting final static String DB_TABLE = "HBMS_DBS";
@@ -90,10 +90,10 @@ class HBaseReadWrite {
   /**
    * List of tables in HBase
    */
-  final static String[] tableNames = { AGGR_STATS_TABLE, DB_TABLE, FUNC_TABLE, GLOBAL_PRIVS_TABLE,
+  public final static String[] tableNames = { AGGR_STATS_TABLE, DB_TABLE, FUNC_TABLE, GLOBAL_PRIVS_TABLE,
                                        PART_TABLE, USER_TO_ROLE_TABLE, ROLE_TABLE, SD_TABLE,
                                        SECURITY_TABLE, SEQUENCES_TABLE, TABLE_TABLE};
-  final static Map<String, List<byte[]>> columnFamilies =
+  public final static Map<String, List<byte[]>> columnFamilies =
       new HashMap<String, List<byte[]>> (tableNames.length);
 
   static {
@@ -178,7 +178,7 @@ class HBaseReadWrite {
    * @param configuration Configuration object
    * @return thread's instance of HBaseReadWrite
    */
-  static HBaseReadWrite getInstance(Configuration configuration) {
+  public static HBaseReadWrite getInstance(Configuration configuration) {
     staticConf = configuration;
     return self.get();
   }


[36/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)


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

Branch: refs/heads/llap
Commit: 129bed52e65b169ddb62f323fc7427df5bb50f19
Parents: 5e16d53
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Aug 24 11:40:09 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Aug 24 11:40:09 2015 -0700

----------------------------------------------------------------------
 metastore/if/hive_metastore.thrift              |   53 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 6792 +++++++------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  556 ++
 .../ThriftHiveMetastore_server.skeleton.cpp     |   20 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 1294 ++-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  371 +
 .../hive/metastore/api/AbortTxnRequest.java     |    2 +-
 .../metastore/api/AddDynamicPartitions.java     |    2 +-
 .../metastore/api/AddPartitionsRequest.java     |    2 +-
 .../hive/metastore/api/AddPartitionsResult.java |    2 +-
 .../hadoop/hive/metastore/api/AggrStats.java    |    2 +-
 .../metastore/api/AlreadyExistsException.java   |    2 +-
 .../metastore/api/BinaryColumnStatsData.java    |    2 +-
 .../metastore/api/BooleanColumnStatsData.java   |    2 +-
 .../hive/metastore/api/CheckLockRequest.java    |    2 +-
 .../metastore/api/ClearFileMetadataRequest.java |  438 +
 .../metastore/api/ClearFileMetadataResult.java  |  283 +
 .../hive/metastore/api/ColumnStatistics.java    |    2 +-
 .../metastore/api/ColumnStatisticsDesc.java     |    2 +-
 .../hive/metastore/api/ColumnStatisticsObj.java |    2 +-
 .../hive/metastore/api/CommitTxnRequest.java    |    2 +-
 .../hive/metastore/api/CompactionRequest.java   |    2 +-
 .../api/ConfigValSecurityException.java         |    2 +-
 .../api/CurrentNotificationEventId.java         |    2 +-
 .../hadoop/hive/metastore/api/Database.java     |    2 +-
 .../apache/hadoop/hive/metastore/api/Date.java  |    2 +-
 .../hive/metastore/api/DateColumnStatsData.java |    2 +-
 .../hadoop/hive/metastore/api/Decimal.java      |    2 +-
 .../metastore/api/DecimalColumnStatsData.java   |    2 +-
 .../metastore/api/DoubleColumnStatsData.java    |    2 +-
 .../hive/metastore/api/DropPartitionsExpr.java  |    2 +-
 .../metastore/api/DropPartitionsRequest.java    |    2 +-
 .../metastore/api/DropPartitionsResult.java     |    2 +-
 .../hive/metastore/api/EnvironmentContext.java  |    2 +-
 .../hadoop/hive/metastore/api/FieldSchema.java  |    2 +-
 .../hive/metastore/api/FireEventRequest.java    |    2 +-
 .../hive/metastore/api/FireEventResponse.java   |    2 +-
 .../hadoop/hive/metastore/api/Function.java     |    2 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   38 +-
 .../api/GetFileMetadataByExprRequest.java       |  548 ++
 .../api/GetFileMetadataByExprResult.java        |  703 ++
 .../metastore/api/GetFileMetadataRequest.java   |  438 +
 .../metastore/api/GetFileMetadataResult.java    |  540 +
 .../metastore/api/GetOpenTxnsInfoResponse.java  |    2 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |    2 +-
 .../api/GetPrincipalsInRoleRequest.java         |    2 +-
 .../api/GetPrincipalsInRoleResponse.java        |    2 +-
 .../api/GetRoleGrantsForPrincipalRequest.java   |    2 +-
 .../api/GetRoleGrantsForPrincipalResponse.java  |    2 +-
 .../api/GrantRevokePrivilegeRequest.java        |    2 +-
 .../api/GrantRevokePrivilegeResponse.java       |    2 +-
 .../metastore/api/GrantRevokeRoleRequest.java   |    2 +-
 .../metastore/api/GrantRevokeRoleResponse.java  |    2 +-
 .../hive/metastore/api/HeartbeatRequest.java    |    2 +-
 .../metastore/api/HeartbeatTxnRangeRequest.java |    2 +-
 .../api/HeartbeatTxnRangeResponse.java          |    2 +-
 .../hive/metastore/api/HiveObjectPrivilege.java |    2 +-
 .../hive/metastore/api/HiveObjectRef.java       |    2 +-
 .../apache/hadoop/hive/metastore/api/Index.java |    2 +-
 .../api/IndexAlreadyExistsException.java        |    2 +-
 .../metastore/api/InsertEventRequestData.java   |    2 +-
 .../metastore/api/InvalidInputException.java    |    2 +-
 .../metastore/api/InvalidObjectException.java   |    2 +-
 .../api/InvalidOperationException.java          |    2 +-
 .../api/InvalidPartitionException.java          |    2 +-
 .../hive/metastore/api/LockComponent.java       |    2 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |    2 +-
 .../hadoop/hive/metastore/api/LockResponse.java |    2 +-
 .../hive/metastore/api/LongColumnStatsData.java |    2 +-
 .../hive/metastore/api/MetaException.java       |    2 +-
 .../hive/metastore/api/MetadataPpdResult.java   |  508 +
 .../hive/metastore/api/NoSuchLockException.java |    2 +-
 .../metastore/api/NoSuchObjectException.java    |    2 +-
 .../hive/metastore/api/NoSuchTxnException.java  |    2 +-
 .../hive/metastore/api/NotificationEvent.java   |    2 +-
 .../metastore/api/NotificationEventRequest.java |    2 +-
 .../api/NotificationEventResponse.java          |    2 +-
 .../hive/metastore/api/OpenTxnRequest.java      |    2 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |    2 +-
 .../apache/hadoop/hive/metastore/api/Order.java |    2 +-
 .../hadoop/hive/metastore/api/Partition.java    |    2 +-
 .../api/PartitionListComposingSpec.java         |    2 +-
 .../hive/metastore/api/PartitionSpec.java       |    2 +-
 .../api/PartitionSpecWithSharedSD.java          |    2 +-
 .../hive/metastore/api/PartitionWithoutSD.java  |    2 +-
 .../metastore/api/PartitionsByExprRequest.java  |    2 +-
 .../metastore/api/PartitionsByExprResult.java   |    2 +-
 .../metastore/api/PartitionsStatsRequest.java   |    2 +-
 .../metastore/api/PartitionsStatsResult.java    |    2 +-
 .../metastore/api/PrincipalPrivilegeSet.java    |    2 +-
 .../hadoop/hive/metastore/api/PrivilegeBag.java |    2 +-
 .../hive/metastore/api/PrivilegeGrantInfo.java  |    2 +-
 .../metastore/api/PutFileMetadataRequest.java   |  588 ++
 .../metastore/api/PutFileMetadataResult.java    |  283 +
 .../hadoop/hive/metastore/api/ResourceUri.java  |    2 +-
 .../apache/hadoop/hive/metastore/api/Role.java  |    2 +-
 .../hive/metastore/api/RolePrincipalGrant.java  |    2 +-
 .../hadoop/hive/metastore/api/Schema.java       |    2 +-
 .../hadoop/hive/metastore/api/SerDeInfo.java    |    2 +-
 .../api/SetPartitionsStatsRequest.java          |    2 +-
 .../hive/metastore/api/ShowCompactRequest.java  |    2 +-
 .../hive/metastore/api/ShowCompactResponse.java |    2 +-
 .../api/ShowCompactResponseElement.java         |    2 +-
 .../hive/metastore/api/ShowLocksRequest.java    |    2 +-
 .../hive/metastore/api/ShowLocksResponse.java   |    2 +-
 .../metastore/api/ShowLocksResponseElement.java |    2 +-
 .../hadoop/hive/metastore/api/SkewedInfo.java   |    2 +-
 .../hive/metastore/api/StorageDescriptor.java   |    2 +-
 .../metastore/api/StringColumnStatsData.java    |    2 +-
 .../apache/hadoop/hive/metastore/api/Table.java |    2 +-
 .../hive/metastore/api/TableStatsRequest.java   |    2 +-
 .../hive/metastore/api/TableStatsResult.java    |    2 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 9282 ++++++++++++------
 .../hive/metastore/api/TxnAbortedException.java |    2 +-
 .../hadoop/hive/metastore/api/TxnInfo.java      |    2 +-
 .../hive/metastore/api/TxnOpenException.java    |    2 +-
 .../apache/hadoop/hive/metastore/api/Type.java  |    2 +-
 .../hive/metastore/api/UnknownDBException.java  |    2 +-
 .../api/UnknownPartitionException.java          |    2 +-
 .../metastore/api/UnknownTableException.java    |    2 +-
 .../hive/metastore/api/UnlockRequest.java       |    2 +-
 .../hadoop/hive/metastore/api/Version.java      |    2 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 2226 +++--
 .../src/gen/thrift/gen-php/metastore/Types.php  | 1009 +-
 .../hive_metastore/ThriftHiveMetastore-remote   |   28 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1432 ++-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  734 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  167 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |  216 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   54 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   11 +
 .../apache/hadoop/hive/metastore/RawStore.java  |    5 +
 .../hive/metastore/hbase/HBaseReadWrite.java    |   84 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   31 +
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |   13 +
 .../DummyRawStoreControlledCommit.java          |   10 +
 .../DummyRawStoreForJdoConnection.java          |    9 +
 .../hadoop/hive/ql/plan/api/Adjacency.java      |    2 +-
 .../apache/hadoop/hive/ql/plan/api/Graph.java   |    2 +-
 .../hadoop/hive/ql/plan/api/Operator.java       |    2 +-
 .../apache/hadoop/hive/ql/plan/api/Query.java   |    2 +-
 .../hadoop/hive/ql/plan/api/QueryPlan.java      |    2 +-
 .../apache/hadoop/hive/ql/plan/api/Stage.java   |    2 +-
 .../apache/hadoop/hive/ql/plan/api/Task.java    |    2 +-
 .../hadoop/hive/serde/test/InnerStruct.java     |    2 +-
 .../hadoop/hive/serde/test/ThriftTestObj.java   |    2 +-
 .../hadoop/hive/serde2/thrift/test/Complex.java |    2 +-
 .../hive/serde2/thrift/test/IntString.java      |    2 +-
 .../hive/serde2/thrift/test/MegaStruct.java     |    2 +-
 .../hive/serde2/thrift/test/MiniStruct.java     |    2 +-
 .../hive/serde2/thrift/test/SetIntString.java   |    2 +-
 .../hadoop/hive/service/HiveClusterStatus.java  |    2 +-
 .../hive/service/HiveServerException.java       |    2 +-
 .../apache/hadoop/hive/service/ThriftHive.java  |    2 +-
 .../service/cli/thrift/TArrayTypeEntry.java     |    2 +-
 .../hive/service/cli/thrift/TBinaryColumn.java  |    2 +-
 .../hive/service/cli/thrift/TBoolColumn.java    |    2 +-
 .../hive/service/cli/thrift/TBoolValue.java     |    2 +-
 .../hive/service/cli/thrift/TByteColumn.java    |    2 +-
 .../hive/service/cli/thrift/TByteValue.java     |    2 +-
 .../hive/service/cli/thrift/TCLIService.java    |    2 +-
 .../cli/thrift/TCancelDelegationTokenReq.java   |    2 +-
 .../cli/thrift/TCancelDelegationTokenResp.java  |    2 +-
 .../service/cli/thrift/TCancelOperationReq.java |    2 +-
 .../cli/thrift/TCancelOperationResp.java        |    2 +-
 .../service/cli/thrift/TCloseOperationReq.java  |    2 +-
 .../service/cli/thrift/TCloseOperationResp.java |    2 +-
 .../service/cli/thrift/TCloseSessionReq.java    |    2 +-
 .../service/cli/thrift/TCloseSessionResp.java   |    2 +-
 .../hive/service/cli/thrift/TColumnDesc.java    |    2 +-
 .../hive/service/cli/thrift/TDoubleColumn.java  |    2 +-
 .../hive/service/cli/thrift/TDoubleValue.java   |    2 +-
 .../cli/thrift/TExecuteStatementReq.java        |    2 +-
 .../cli/thrift/TExecuteStatementResp.java       |    2 +-
 .../service/cli/thrift/TFetchResultsReq.java    |    2 +-
 .../service/cli/thrift/TFetchResultsResp.java   |    2 +-
 .../service/cli/thrift/TGetCatalogsReq.java     |    2 +-
 .../service/cli/thrift/TGetCatalogsResp.java    |    2 +-
 .../hive/service/cli/thrift/TGetColumnsReq.java |    2 +-
 .../service/cli/thrift/TGetColumnsResp.java     |    2 +-
 .../cli/thrift/TGetDelegationTokenReq.java      |    2 +-
 .../cli/thrift/TGetDelegationTokenResp.java     |    2 +-
 .../service/cli/thrift/TGetFunctionsReq.java    |    2 +-
 .../service/cli/thrift/TGetFunctionsResp.java   |    2 +-
 .../hive/service/cli/thrift/TGetInfoReq.java    |    2 +-
 .../hive/service/cli/thrift/TGetInfoResp.java   |    2 +-
 .../cli/thrift/TGetOperationStatusReq.java      |    2 +-
 .../cli/thrift/TGetOperationStatusResp.java     |    2 +-
 .../cli/thrift/TGetResultSetMetadataReq.java    |    2 +-
 .../cli/thrift/TGetResultSetMetadataResp.java   |    2 +-
 .../hive/service/cli/thrift/TGetSchemasReq.java |    2 +-
 .../service/cli/thrift/TGetSchemasResp.java     |    2 +-
 .../service/cli/thrift/TGetTableTypesReq.java   |    2 +-
 .../service/cli/thrift/TGetTableTypesResp.java  |    2 +-
 .../hive/service/cli/thrift/TGetTablesReq.java  |    2 +-
 .../hive/service/cli/thrift/TGetTablesResp.java |    2 +-
 .../service/cli/thrift/TGetTypeInfoReq.java     |    2 +-
 .../service/cli/thrift/TGetTypeInfoResp.java    |    2 +-
 .../service/cli/thrift/THandleIdentifier.java   |    2 +-
 .../hive/service/cli/thrift/TI16Column.java     |    2 +-
 .../hive/service/cli/thrift/TI16Value.java      |    2 +-
 .../hive/service/cli/thrift/TI32Column.java     |    2 +-
 .../hive/service/cli/thrift/TI32Value.java      |    2 +-
 .../hive/service/cli/thrift/TI64Column.java     |    2 +-
 .../hive/service/cli/thrift/TI64Value.java      |    2 +-
 .../hive/service/cli/thrift/TMapTypeEntry.java  |    2 +-
 .../service/cli/thrift/TOpenSessionReq.java     |    2 +-
 .../service/cli/thrift/TOpenSessionResp.java    |    2 +-
 .../service/cli/thrift/TOperationHandle.java    |    2 +-
 .../service/cli/thrift/TPrimitiveTypeEntry.java |    2 +-
 .../cli/thrift/TRenewDelegationTokenReq.java    |    2 +-
 .../cli/thrift/TRenewDelegationTokenResp.java   |    2 +-
 .../apache/hive/service/cli/thrift/TRow.java    |    2 +-
 .../apache/hive/service/cli/thrift/TRowSet.java |    2 +-
 .../hive/service/cli/thrift/TSessionHandle.java |    2 +-
 .../apache/hive/service/cli/thrift/TStatus.java |    2 +-
 .../hive/service/cli/thrift/TStringColumn.java  |    2 +-
 .../hive/service/cli/thrift/TStringValue.java   |    2 +-
 .../service/cli/thrift/TStructTypeEntry.java    |    2 +-
 .../hive/service/cli/thrift/TTableSchema.java   |    2 +-
 .../hive/service/cli/thrift/TTypeDesc.java      |    2 +-
 .../service/cli/thrift/TTypeQualifiers.java     |    2 +-
 .../service/cli/thrift/TUnionTypeEntry.java     |    2 +-
 .../cli/thrift/TUserDefinedTypeEntry.java       |    2 +-
 .../gen-py/hive_service/ThriftHive-remote       |   49 +-
 225 files changed, 22079 insertions(+), 7118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index e4b9fd1..7026a0d 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -714,6 +714,53 @@ struct FireEventResponse {
     // NOP for now, this is just a place holder for future responses
 }
     
+struct MetadataPpdResult {
+  1: required binary metadata,
+  2: required binary includeBitset
+}
+
+// Return type for get_file_metadata_by_expr
+struct GetFileMetadataByExprResult {
+  1: required map<i64, MetadataPpdResult> metadata,
+  2: required bool isSupported,
+  3: required list<i64> unknownFileIds
+}
+
+// Request type for get_file_metadata_by_expr
+struct GetFileMetadataByExprRequest {
+  1: required list<i64> fileIds,
+  2: required binary expr
+}
+
+// Return type for get_file_metadata
+struct GetFileMetadataResult {
+  1: required map<i64, binary> metadata,
+  2: required bool isSupported
+}
+
+// Request type for get_file_metadata
+struct GetFileMetadataRequest {
+  1: required list<i64> fileIds
+}
+
+// Return type for put_file_metadata
+struct PutFileMetadataResult {
+}
+
+// Request type for put_file_metadata
+struct PutFileMetadataRequest {
+  1: required list<i64> fileIds,
+  2: required list<binary> metadata
+}
+
+// Return type for clear_file_metadata
+struct ClearFileMetadataResult {
+}
+
+// Request type for clear_file_metadata
+struct ClearFileMetadataRequest {
+  1: required list<i64> fileIds
+}
 
 struct GetAllFunctionsResponse {
   1: optional list<Function> functions
@@ -1195,6 +1242,12 @@ service ThriftHiveMetastore extends fb303.FacebookService
   CurrentNotificationEventId get_current_notificationEventId()
   FireEventResponse fire_listener_event(1:FireEventRequest rqst)
   void flushCache()
+
+  GetFileMetadataByExprResult get_file_metadata_by_expr(1:GetFileMetadataByExprRequest req)
+  GetFileMetadataResult get_file_metadata(1:GetFileMetadataRequest req)
+  PutFileMetadataResult put_file_metadata(1:PutFileMetadataRequest req)
+  ClearFileMetadataResult clear_file_metadata(1:ClearFileMetadataRequest req)
+
 }
 
 // * Note about the DDL_TIME: When creating or altering a table or a partition,


[06/52] [abbrv] hive git commit: HIVE-11349 Update HBase metastore hbase version to 1.1.1 (gates)

Posted by se...@apache.org.
HIVE-11349 Update HBase metastore hbase version to 1.1.1 (gates)


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

Branch: refs/heads/llap
Commit: cb3702164b027c646dca9ebbf01daeee0dab0d65
Parents: 61db7b8
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jul 22 18:52:17 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jul 22 18:52:17 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/hbase/HBaseIntegrationTests.java    | 7 +++++--
 metastore/pom.xml                                             | 2 +-
 .../hadoop/hive/metastore/hbase/TephraHBaseConnection.java    | 4 ++--
 pom.xml                                                       | 4 ++--
 4 files changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cb370216/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
index 58b1ee9..c369058 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
-import co.cask.tephra.hbase98.coprocessor.TransactionProcessor;
+import co.cask.tephra.hbase10.coprocessor.TransactionProcessor;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -46,7 +46,7 @@ public class HBaseIntegrationTests {
 
   protected static HBaseTestingUtility utility;
   protected static HBaseAdmin admin;
-  protected static Map<String, String> emptyParameters = new HashMap<String, String>();
+  protected static Map<String, String> emptyParameters = new HashMap<>();
   protected static HiveConf conf;
 
   protected HBaseStore store;
@@ -57,6 +57,9 @@ public class HBaseIntegrationTests {
         System.getProperty(HiveConf.ConfVars.METASTORE_HBASE_CONNECTION_CLASS.varname);
     boolean testingTephra =
         connectionClassName != null && connectionClassName.equals(TephraHBaseConnection.class.getName());
+    if (testingTephra) {
+      LOG.info("Testing with Tephra");
+    }
     utility = new HBaseTestingUtility();
     utility.startMiniCluster();
     conf = new HiveConf(utility.getConfiguration(), HBaseIntegrationTests.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/cb370216/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index e3942f8..ccec9f1 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -138,7 +138,7 @@
     </dependency>
     <dependency>
       <groupId>co.cask.tephra</groupId>
-      <artifactId>tephra-hbase-compat-0.98</artifactId>
+      <artifactId>tephra-hbase-compat-1.0</artifactId>
       <version>${tephra.version}</version>
     </dependency>
     <!-- test inter-project -->

http://git-wip-us.apache.org/repos/asf/hive/blob/cb370216/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/TephraHBaseConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/TephraHBaseConnection.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/TephraHBaseConnection.java
index 47c3f11..f9c6e73 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/TephraHBaseConnection.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/TephraHBaseConnection.java
@@ -25,8 +25,8 @@ import co.cask.tephra.TransactionManager;
 import co.cask.tephra.TransactionSystemClient;
 import co.cask.tephra.distributed.ThreadLocalClientProvider;
 import co.cask.tephra.distributed.TransactionServiceClient;
-import co.cask.tephra.hbase98.TransactionAwareHTable;
-import co.cask.tephra.hbase98.coprocessor.TransactionProcessor;
+import co.cask.tephra.hbase10.TransactionAwareHTable;
+import co.cask.tephra.hbase10.coprocessor.TransactionProcessor;
 import co.cask.tephra.inmemory.InMemoryTxSystemClient;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;

http://git-wip-us.apache.org/repos/asf/hive/blob/cb370216/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e9c054a..bb629b2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -125,7 +125,7 @@
     <hadoop-23.version>2.6.0</hadoop-23.version>
     <hadoop.bin.path>${basedir}/${hive.path.to.root}/testutils/hadoop</hadoop.bin.path>
     <hbase.hadoop1.version>0.98.9-hadoop1</hbase.hadoop1.version>
-    <hbase.hadoop2.version>0.98.9-hadoop2</hbase.hadoop2.version>
+    <hbase.hadoop2.version>1.1.1</hbase.hadoop2.version>
     <!-- httpcomponents are not always in version sync -->
     <httpcomponents.client.version>4.4</httpcomponents.client.version>
     <httpcomponents.core.version>4.4</httpcomponents.core.version>
@@ -174,7 +174,7 @@
     <felix.version>2.4.0</felix.version>
     <curator.version>2.6.0</curator.version>
     <jsr305.version>3.0.0</jsr305.version>
-    <tephra.version>0.4.0</tephra.version>
+    <tephra.version>0.5.1-SNAPSHOT</tephra.version>
     <gson.version>2.2.4</gson.version>
   </properties>
 


[08/52] [abbrv] hive git commit: HIVE-11379 Bump Tephra version to 0.6.0 (gates)

Posted by se...@apache.org.
HIVE-11379 Bump Tephra version to 0.6.0 (gates)


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

Branch: refs/heads/llap
Commit: 9d3d4ebfefa96f442150989c8842a1df8a438dc2
Parents: 5acf458
Author: Alan Gates <ga...@hortonworks.com>
Authored: Thu Jul 30 10:00:37 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Thu Jul 30 10:00:37 2015 -0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9d3d4ebf/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bb629b2..d03fb5f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -174,7 +174,7 @@
     <felix.version>2.4.0</felix.version>
     <curator.version>2.6.0</curator.version>
     <jsr305.version>3.0.0</jsr305.version>
-    <tephra.version>0.5.1-SNAPSHOT</tephra.version>
+    <tephra.version>0.6.0</tephra.version>
     <gson.version>2.2.4</gson.version>
   </properties>
 


[17/52] [abbrv] hive git commit: HIVE-11568 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-11568 : merge master into branch (Sergey Shelukhin)


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

Branch: refs/heads/llap
Commit: c528294bc99c8ba05dd0a3c3f39cc06fb27b9473
Parents: 0fa45e4 e8b2c60
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Aug 14 15:58:43 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Aug 14 15:58:43 2015 -0700

----------------------------------------------------------------------
 accumulo-handler/pom.xml                        |     4 -
 .../apache/hadoop/hive/ant/GenVectorCode.java   |   105 +
 .../src/main/resources/beeline-log4j.properties |    24 -
 beeline/src/main/resources/beeline-log4j2.xml   |    40 +
 bin/ext/beeline.sh                              |     2 +-
 bin/hive                                        |     3 +
 .../hadoop/hive/cli/TestOptionsProcessor.java   |     1 -
 common/pom.xml                                  |    27 +-
 .../apache/hadoop/hive/common/JavaUtils.java    |    11 +-
 .../org/apache/hadoop/hive/common/LogUtils.java |    18 +-
 .../hadoop/hive/common/ValidReadTxnList.java    |     2 +-
 .../hadoop/hive/common/type/HiveDecimal.java    |   306 -
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    27 +-
 common/src/main/resources/hive-log4j.properties |    88 -
 common/src/main/resources/hive-log4j2.xml       |   111 +
 .../hadoop/hive/conf/TestHiveLogging.java       |     8 +-
 .../resources/hive-exec-log4j-test.properties   |    59 -
 .../test/resources/hive-exec-log4j2-test.xml    |    86 +
 .../test/resources/hive-log4j-test.properties   |    71 -
 common/src/test/resources/hive-log4j2-test.xml  |    95 +
 data/conf/hive-log4j-old.properties             |    82 -
 data/conf/hive-log4j.properties                 |    97 -
 data/conf/hive-log4j2.xml                       |   148 +
 data/conf/spark/log4j.properties                |    24 -
 data/conf/spark/log4j2.xml                      |    74 +
 docs/xdocs/language_manual/cli.xml              |     2 +-
 .../test/results/positive/hbase_timestamp.q.out |     8 +-
 hcatalog/bin/hcat_server.sh                     |     2 +-
 hcatalog/bin/templeton.cmd                      |     4 +-
 .../mapreduce/DefaultOutputFormatContainer.java |     7 +-
 ...namicPartitionFileRecordWriterContainer.java |     3 +-
 .../mapreduce/FileOutputFormatContainer.java    |     3 +-
 .../hive/hcatalog/mapreduce/PartInfo.java       |    32 +-
 .../hive/hcatalog/mapreduce/SpecialCases.java   |     8 +-
 .../mapreduce/TestHCatMultiOutputFormat.java    |     6 +-
 hcatalog/scripts/hcat_server_start.sh           |     2 +-
 .../content/xdocs/configuration.xml             |     2 +-
 .../src/documentation/content/xdocs/install.xml |     2 +-
 .../deployers/config/hive/hive-log4j.properties |    88 -
 .../deployers/config/hive/hive-log4j2.xml       |   111 +
 .../templeton/deployers/start_hive_services.sh  |     2 +-
 .../webhcat/svr/src/main/bin/webhcat_server.sh  |     4 +-
 .../src/main/config/webhcat-log4j.properties    |    45 -
 .../svr/src/main/config/webhcat-log4j2.xml      |    75 +
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |   164 +-
 .../main/java/org/apache/hive/hplsql/Conn.java  |     6 +
 .../java/org/apache/hive/hplsql/Converter.java  |    41 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |    65 +-
 .../java/org/apache/hive/hplsql/Expression.java |    73 +-
 .../main/java/org/apache/hive/hplsql/Meta.java  |    98 +
 .../main/java/org/apache/hive/hplsql/Query.java |    55 +
 .../java/org/apache/hive/hplsql/Select.java     |    47 +-
 .../java/org/apache/hive/hplsql/Signal.java     |     2 +-
 .../main/java/org/apache/hive/hplsql/Stmt.java  |    97 +-
 .../main/java/org/apache/hive/hplsql/Var.java   |    43 +-
 .../apache/hive/hplsql/functions/Function.java  |    49 +-
 .../hive/hplsql/functions/FunctionDatetime.java |    14 +-
 .../hive/hplsql/functions/FunctionMisc.java     |    22 +-
 .../hive/hplsql/functions/FunctionOra.java      |    31 +-
 .../hive/hplsql/functions/FunctionString.java   |    46 +-
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |     5 +
 .../apache/hive/hplsql/TestHplsqlOffline.java   |    76 +
 .../test/queries/db/create_procedure_mssql.sql  |    52 +
 .../src/test/queries/db/cursor_attributes.sql   |    60 +
 hplsql/src/test/queries/db/map_object.sql       |     9 +
 hplsql/src/test/queries/db/select_into.sql      |    17 +
 .../src/test/queries/db/set_current_schema.sql  |     6 +
 hplsql/src/test/queries/db/sys_refcursor.sql    |    65 +
 hplsql/src/test/queries/db/use.sql              |     2 +
 .../queries/local/exception_divide_by_zero.sql  |    11 +
 .../test/queries/offline/create_table_mssql.sql |    43 +
 .../test/queries/offline/create_table_ora.sql   |     4 +
 .../results/db/create_procedure_mssql.out.txt   |    45 +
 .../test/results/db/cursor_attributes.out.txt   |    33 +
 hplsql/src/test/results/db/map_object.out.txt   |    17 +
 hplsql/src/test/results/db/select_into.out.txt  |    19 +
 .../test/results/db/set_current_schema.out.txt  |    12 +
 .../src/test/results/db/sys_refcursor.out.txt   |    36 +
 hplsql/src/test/results/db/use.out.txt          |     4 +
 .../test/results/local/create_function.out.txt  |     4 +-
 hplsql/src/test/results/local/declare.out.txt   |     4 +-
 .../local/exception_divide_by_zero.out.txt      |     8 +
 .../results/offline/create_table_mssql.out.txt  |    24 +
 .../results/offline/create_table_ora.out.txt    |     4 +
 .../hive/metastore/TestHiveMetaStore.java       |    96 +-
 .../operation/TestOperationLoggingLayout.java   |   136 +
 itests/pom.xml                                  |     2 +-
 itests/qtest-spark/pom.xml                      |    24 +
 itests/qtest/pom.xml                            |    28 +-
 .../test/resources/testconfiguration.properties |    50 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    62 +-
 jdbc/pom.xml                                    |     1 +
 metastore/if/hive_metastore.thrift              |     5 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  5526 ++-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  7970 ++--
 .../ThriftHiveMetastore_server.skeleton.cpp     |     5 +
 .../thrift/gen-cpp/hive_metastore_constants.cpp |     2 +-
 .../thrift/gen-cpp/hive_metastore_constants.h   |     2 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  6204 ++-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  2666 +-
 .../hive/metastore/api/AbortTxnRequest.java     |    24 +-
 .../metastore/api/AddDynamicPartitions.java     |    69 +-
 .../metastore/api/AddPartitionsRequest.java     |    80 +-
 .../hive/metastore/api/AddPartitionsResult.java |    48 +-
 .../hadoop/hive/metastore/api/AggrStats.java    |    54 +-
 .../metastore/api/AlreadyExistsException.java   |    24 +-
 .../metastore/api/BinaryColumnStatsData.java    |    40 +-
 .../metastore/api/BooleanColumnStatsData.java   |    40 +-
 .../hive/metastore/api/CheckLockRequest.java    |    24 +-
 .../hive/metastore/api/ColumnStatistics.java    |    54 +-
 .../metastore/api/ColumnStatisticsData.java     |    20 +-
 .../metastore/api/ColumnStatisticsDesc.java     |    58 +-
 .../hive/metastore/api/ColumnStatisticsObj.java |    40 +-
 .../hive/metastore/api/CommitTxnRequest.java    |    24 +-
 .../hive/metastore/api/CompactionRequest.java   |    62 +-
 .../hive/metastore/api/CompactionType.java      |     2 +-
 .../api/ConfigValSecurityException.java         |    24 +-
 .../api/CurrentNotificationEventId.java         |    24 +-
 .../hadoop/hive/metastore/api/Database.java     |   115 +-
 .../apache/hadoop/hive/metastore/api/Date.java  |    24 +-
 .../hive/metastore/api/DateColumnStatsData.java |    50 +-
 .../hadoop/hive/metastore/api/Decimal.java      |    41 +-
 .../metastore/api/DecimalColumnStatsData.java   |    50 +-
 .../metastore/api/DoubleColumnStatsData.java    |    50 +-
 .../hive/metastore/api/DropPartitionsExpr.java  |    43 +-
 .../metastore/api/DropPartitionsRequest.java    |    82 +-
 .../metastore/api/DropPartitionsResult.java     |    48 +-
 .../hive/metastore/api/EnvironmentContext.java  |    61 +-
 .../hive/metastore/api/EventRequestType.java    |     2 +-
 .../hadoop/hive/metastore/api/FieldSchema.java  |    58 +-
 .../hive/metastore/api/FireEventRequest.java    |    79 +-
 .../metastore/api/FireEventRequestData.java     |    20 +-
 .../hive/metastore/api/FireEventResponse.java   |    16 +-
 .../hadoop/hive/metastore/api/Function.java     |   110 +-
 .../hadoop/hive/metastore/api/FunctionType.java |     2 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   447 +
 .../metastore/api/GetOpenTxnsInfoResponse.java  |    54 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |    53 +-
 .../api/GetPrincipalsInRoleRequest.java         |    24 +-
 .../api/GetPrincipalsInRoleResponse.java        |    46 +-
 .../api/GetRoleGrantsForPrincipalRequest.java   |    36 +-
 .../api/GetRoleGrantsForPrincipalResponse.java  |    46 +-
 .../api/GrantRevokePrivilegeRequest.java        |    46 +-
 .../api/GrantRevokePrivilegeResponse.java       |    26 +-
 .../metastore/api/GrantRevokeRoleRequest.java   |    86 +-
 .../metastore/api/GrantRevokeRoleResponse.java  |    26 +-
 .../hive/metastore/api/GrantRevokeType.java     |     2 +-
 .../hive/metastore/api/HeartbeatRequest.java    |    34 +-
 .../metastore/api/HeartbeatTxnRangeRequest.java |    32 +-
 .../api/HeartbeatTxnRangeResponse.java          |    74 +-
 .../hive/metastore/api/HiveObjectPrivilege.java |    52 +-
 .../hive/metastore/api/HiveObjectRef.java       |    81 +-
 .../hive/metastore/api/HiveObjectType.java      |     2 +-
 .../apache/hadoop/hive/metastore/api/Index.java |   133 +-
 .../api/IndexAlreadyExistsException.java        |    24 +-
 .../metastore/api/InsertEventRequestData.java   |    45 +-
 .../metastore/api/InvalidInputException.java    |    24 +-
 .../metastore/api/InvalidObjectException.java   |    24 +-
 .../api/InvalidOperationException.java          |    24 +-
 .../api/InvalidPartitionException.java          |    24 +-
 .../hive/metastore/api/LockComponent.java       |    66 +-
 .../hadoop/hive/metastore/api/LockLevel.java    |     2 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |    72 +-
 .../hadoop/hive/metastore/api/LockResponse.java |    36 +-
 .../hadoop/hive/metastore/api/LockState.java    |     2 +-
 .../hadoop/hive/metastore/api/LockType.java     |     2 +-
 .../hive/metastore/api/LongColumnStatsData.java |    50 +-
 .../hive/metastore/api/MetaException.java       |    24 +-
 .../hive/metastore/api/NoSuchLockException.java |    24 +-
 .../metastore/api/NoSuchObjectException.java    |    24 +-
 .../hive/metastore/api/NoSuchTxnException.java  |    24 +-
 .../hive/metastore/api/NotificationEvent.java   |    66 +-
 .../metastore/api/NotificationEventRequest.java |    34 +-
 .../api/NotificationEventResponse.java          |    46 +-
 .../hive/metastore/api/OpenTxnRequest.java      |    40 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |    45 +-
 .../apache/hadoop/hive/metastore/api/Order.java |    32 +-
 .../hadoop/hive/metastore/api/Partition.java    |   156 +-
 .../hive/metastore/api/PartitionEventType.java  |     2 +-
 .../api/PartitionListComposingSpec.java         |    46 +-
 .../hive/metastore/api/PartitionSpec.java       |    58 +-
 .../api/PartitionSpecWithSharedSD.java          |    54 +-
 .../hive/metastore/api/PartitionWithoutSD.java  |   124 +-
 .../metastore/api/PartitionsByExprRequest.java  |    67 +-
 .../metastore/api/PartitionsByExprResult.java   |    54 +-
 .../metastore/api/PartitionsStatsRequest.java   |    90 +-
 .../metastore/api/PartitionsStatsResult.java    |    72 +-
 .../metastore/api/PrincipalPrivilegeSet.java    |   184 +-
 .../hive/metastore/api/PrincipalType.java       |     2 +-
 .../hadoop/hive/metastore/api/PrivilegeBag.java |    46 +-
 .../hive/metastore/api/PrivilegeGrantInfo.java  |    60 +-
 .../hive/metastore/api/RequestPartsSpec.java    |    56 +-
 .../hadoop/hive/metastore/api/ResourceType.java |     2 +-
 .../hadoop/hive/metastore/api/ResourceUri.java  |    36 +-
 .../apache/hadoop/hive/metastore/api/Role.java  |    40 +-
 .../hive/metastore/api/RolePrincipalGrant.java  |    80 +-
 .../hadoop/hive/metastore/api/Schema.java       |    91 +-
 .../hadoop/hive/metastore/api/SerDeInfo.java    |    93 +-
 .../api/SetPartitionsStatsRequest.java          |    46 +-
 .../hive/metastore/api/ShowCompactRequest.java  |    16 +-
 .../hive/metastore/api/ShowCompactResponse.java |    46 +-
 .../api/ShowCompactResponseElement.java         |    86 +-
 .../hive/metastore/api/ShowLocksRequest.java    |    16 +-
 .../hive/metastore/api/ShowLocksResponse.java   |    46 +-
 .../metastore/api/ShowLocksResponseElement.java |   114 +-
 .../hadoop/hive/metastore/api/SkewedInfo.java   |   147 +-
 .../hive/metastore/api/StorageDescriptor.java   |   242 +-
 .../metastore/api/StringColumnStatsData.java    |    48 +-
 .../apache/hadoop/hive/metastore/api/Table.java |   189 +-
 .../hive/metastore/api/TableStatsRequest.java   |    61 +-
 .../hive/metastore/api/TableStatsResult.java    |    46 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 33504 +++++++++++------
 .../hive/metastore/api/TxnAbortedException.java |    24 +-
 .../hadoop/hive/metastore/api/TxnInfo.java      |    52 +-
 .../hive/metastore/api/TxnOpenException.java    |    24 +-
 .../hadoop/hive/metastore/api/TxnState.java     |     2 +-
 .../apache/hadoop/hive/metastore/api/Type.java  |    72 +-
 .../hive/metastore/api/UnknownDBException.java  |    24 +-
 .../api/UnknownPartitionException.java          |    24 +-
 .../metastore/api/UnknownTableException.java    |    24 +-
 .../hive/metastore/api/UnlockRequest.java       |    24 +-
 .../hadoop/hive/metastore/api/Version.java      |    32 +-
 .../metastore/api/hive_metastoreConstants.java  |     7 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   |  4599 ++-
 .../src/gen/thrift/gen-php/metastore/Types.php  |  1184 +-
 .../hive_metastore/ThriftHiveMetastore-remote   |   620 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  4624 ++-
 .../thrift/gen-py/hive_metastore/constants.py   |     2 +-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   856 +-
 .../thrift/gen-rb/hive_metastore_constants.rb   |     2 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    18 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |    62 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |    34 +-
 .../hive/metastore/HiveMetaStoreClient.java     |     7 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |     4 +
 .../hadoop/hive/metastore/ObjectStore.java      |    35 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |     7 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    16 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    32 +-
 .../metastore/txn/ValidCompactorTxnList.java    |     2 +-
 .../DummyRawStoreControlledCommit.java          |     7 +
 .../DummyRawStoreForJdoConnection.java          |     6 +
 .../metastore/txn/TestCompactionTxnHandler.java |    40 +-
 .../hive/metastore/txn/TestTxnHandler.java      |    66 +-
 packaging/src/main/assembly/bin.xml             |    17 +-
 pom.xml                                         |    52 +-
 ql/if/queryplan.thrift                          |     1 +
 ql/pom.xml                                      |    24 +-
 .../gen/thrift/gen-cpp/queryplan_constants.cpp  |     2 +-
 ql/src/gen/thrift/gen-cpp/queryplan_constants.h |     2 +-
 ql/src/gen/thrift/gen-cpp/queryplan_types.cpp   |   796 +-
 ql/src/gen/thrift/gen-cpp/queryplan_types.h     |   294 +-
 .../hadoop/hive/ql/plan/api/Adjacency.java      |    65 +-
 .../hadoop/hive/ql/plan/api/AdjacencyType.java  |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Graph.java   |    87 +-
 .../hadoop/hive/ql/plan/api/NodeType.java       |     2 +-
 .../hadoop/hive/ql/plan/api/Operator.java       |   142 +-
 .../hadoop/hive/ql/plan/api/OperatorType.java   |     7 +-
 .../apache/hadoop/hive/ql/plan/api/Query.java   |   176 +-
 .../hadoop/hive/ql/plan/api/QueryPlan.java      |    62 +-
 .../apache/hadoop/hive/ql/plan/api/Stage.java   |   172 +-
 .../hadoop/hive/ql/plan/api/StageType.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Task.java    |   182 +-
 .../hadoop/hive/ql/plan/api/TaskType.java       |     2 +-
 ql/src/gen/thrift/gen-php/Types.php             |   119 +-
 ql/src/gen/thrift/gen-py/queryplan/constants.py |     2 +-
 ql/src/gen/thrift/gen-py/queryplan/ttypes.py    |    87 +-
 ql/src/gen/thrift/gen-rb/queryplan_constants.rb |     2 +-
 ql/src/gen/thrift/gen-rb/queryplan_types.rb     |     7 +-
 ...tringGroupColumnCompareStringGroupColumn.txt |   112 +-
 ...gGroupColumnCompareStringGroupScalarBase.txt |    12 +-
 ...gGroupScalarCompareStringGroupColumnBase.txt |    12 +-
 ...tringGroupColumnCompareStringGroupColumn.txt |   112 +-
 ...gGroupColumnCompareStringGroupScalarBase.txt |    12 +-
 ...gGroupScalarCompareStringGroupColumnBase.txt |    12 +-
 .../java/org/apache/hadoop/hive/ql/Context.java |     1 -
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   209 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |    13 +-
 .../org/apache/hadoop/hive/ql/QueryPlan.java    |    18 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |     2 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |     3 +-
 .../hive/ql/exec/HashTableSinkOperator.java     |     6 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |     4 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |    25 -
 .../hadoop/hive/ql/exec/OperatorFactory.java    |    11 +
 .../ql/exec/SparkHashTableSinkOperator.java     |    17 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |    36 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |    29 +-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |    20 +-
 .../persistence/HybridHashTableContainer.java   |     6 +
 .../persistence/MapJoinTableContainerSerDe.java |    63 +-
 .../hive/ql/exec/spark/HashTableLoader.java     |    26 +-
 .../ql/exec/spark/HiveSparkClientFactory.java   |    10 +-
 .../hive/ql/exec/spark/KryoSerializer.java      |     4 +
 .../ql/exec/spark/RemoteHiveSparkClient.java    |    57 +-
 .../exec/spark/SparkDynamicPartitionPruner.java |   268 +
 .../hadoop/hive/ql/exec/spark/SparkPlan.java    |     3 -
 .../hive/ql/exec/spark/SparkPlanGenerator.java  |    15 +-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |     1 +
 .../hive/ql/exec/spark/SparkUtilities.java      |    56 +
 .../spark/status/impl/LocalSparkJobStatus.java  |     2 +-
 .../spark/status/impl/RemoteSparkJobStatus.java |     2 +-
 .../hive/ql/exec/vector/BytesColumnVector.java  |   347 -
 .../hive/ql/exec/vector/ColumnVector.java       |   178 -
 .../ql/exec/vector/DecimalColumnVector.java     |   125 -
 .../hive/ql/exec/vector/DoubleColumnVector.java |   161 -
 .../hive/ql/exec/vector/LongColumnVector.java   |   205 -
 .../VectorSparkHashTableSinkOperator.java       |   104 +
 ...VectorSparkPartitionPruningSinkOperator.java |    99 +
 .../ql/exec/vector/VectorizationContext.java    |    51 +-
 .../hive/ql/exec/vector/VectorizedRowBatch.java |   186 -
 .../BRoundWithNumDigitsDoubleToDouble.java      |    42 +
 .../ql/exec/vector/expressions/DecimalUtil.java |    18 +
 .../vector/expressions/FilterExprAndExpr.java   |     8 +-
 .../vector/expressions/FilterExprOrExpr.java    |   140 +-
 ...FuncBRoundWithNumDigitsDecimalToDecimal.java |    40 +
 .../FuncRoundWithNumDigitsDecimalToDecimal.java |    14 +-
 .../ql/exec/vector/expressions/MathExpr.java    |    22 +
 .../ql/exec/vector/expressions/StringExpr.java  |    51 +
 .../hadoop/hive/ql/hooks/LineageInfo.java       |     9 +-
 .../hadoop/hive/ql/hooks/LineageLogger.java     |    44 +-
 .../hive/ql/hooks/PostExecOrcFileDump.java      |   120 +
 .../ql/hooks/PostExecTezSummaryPrinter.java     |    72 +
 .../hive/ql/io/CombineHiveInputFormat.java      |    98 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |    46 +-
 .../apache/hadoop/hive/ql/io/orc/FileDump.java  |     4 +
 .../apache/hadoop/hive/ql/io/orc/OrcConf.java   |   129 +-
 .../apache/hadoop/hive/ql/io/orc/OrcFile.java   |   141 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |    38 +-
 .../hadoop/hive/ql/io/orc/OrcOutputFormat.java  |    67 +-
 .../apache/hadoop/hive/ql/io/orc/OrcSerde.java  |     6 +-
 .../apache/hadoop/hive/ql/io/orc/Reader.java    |     7 +
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |     5 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |    48 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |    12 +-
 .../parquet/read/DataWritableReadSupport.java   |    10 +-
 .../read/ParquetFilterPredicateConverter.java   |   148 +
 .../read/ParquetRecordReaderWrapper.java        |   125 +-
 .../ql/io/rcfile/stats/PartialScanTask.java     |    20 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |   439 +
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |    56 -
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |  1027 -
 .../hadoop/hive/ql/lib/PreOrderOnceWalker.java  |    44 +
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |    12 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |    71 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |     8 +
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |    21 +
 .../hive/ql/lockmgr/HiveTxnManagerImpl.java     |    10 +
 .../hadoop/hive/ql/lockmgr/LockException.java   |     8 +-
 .../hadoop/hive/ql/log/HiveEventCounter.java    |   135 +
 .../apache/hadoop/hive/ql/log/NullAppender.java |    63 +
 .../ql/log/PidDailyRollingFileAppender.java     |    33 -
 .../hive/ql/log/PidFilePatternConverter.java    |    62 +
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   107 +-
 .../hadoop/hive/ql/metadata/HiveException.java  |     3 +
 .../hadoop/hive/ql/metadata/Partition.java      |     2 +-
 .../hadoop/hive/ql/metadata/TableIterable.java  |   104 +
 .../hadoop/hive/ql/optimizer/ColumnPruner.java  |     4 +
 .../hive/ql/optimizer/ColumnPrunerProcCtx.java  |    95 +-
 .../ql/optimizer/ColumnPrunerProcFactory.java   |    42 +-
 .../optimizer/ConstantPropagateProcFactory.java |   123 +-
 .../DynamicPartitionPruningOptimization.java    |    44 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |    20 +-
 .../hive/ql/optimizer/GroupByOptimizer.java     |    58 +-
 .../ql/optimizer/OperatorComparatorFactory.java |   552 +
 .../hadoop/hive/ql/optimizer/Optimizer.java     |     8 +-
 .../hive/ql/optimizer/PointLookupOptimizer.java |   280 +
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |     5 +-
 .../SparkRemoveDynamicPruningBySize.java        |    73 +
 .../calcite/rules/HiveJoinToMultiJoinRule.java  |    82 +-
 .../calcite/translator/ExprNodeConverter.java   |    21 +-
 .../calcite/translator/HiveOpConverter.java     |    46 +-
 .../translator/PlanModifierForASTConv.java      |     2 +-
 .../translator/PlanModifierForReturnPath.java   |     6 +-
 .../correlation/AbstractCorrelationProcCtx.java |     7 +
 .../correlation/CorrelationUtilities.java       |    11 +-
 .../correlation/ReduceSinkDeDuplication.java    |     6 +-
 .../ql/optimizer/lineage/ExprProcFactory.java   |     9 +-
 .../hive/ql/optimizer/lineage/LineageCtx.java   |    34 +-
 .../ql/optimizer/lineage/OpProcFactory.java     |    10 +-
 .../ql/optimizer/pcr/PcrExprProcFactory.java    |   154 +-
 .../physical/GenSparkSkewJoinProcessor.java     |    14 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |    24 +
 .../hive/ql/optimizer/ppr/OpProcFactory.java    |     3 +-
 .../hive/ql/optimizer/ppr/PartitionPruner.java  |    69 +-
 .../spark/CombineEquivalentWorkResolver.java    |   292 +
 .../spark/SparkPartitionPruningSinkDesc.java    |   100 +
 .../spark/SparkReduceSinkMapJoinProc.java       |     2 +-
 .../stats/annotation/StatsRulesProcFactory.java |    49 +-
 .../apache/hadoop/hive/ql/parse/ASTNode.java    |   139 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |    13 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    45 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    11 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    70 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |    29 +-
 .../apache/hadoop/hive/ql/parse/ParseUtils.java |     2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    74 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    12 +
 .../hive/ql/parse/TypeCheckProcFactory.java     |    42 +-
 .../ql/parse/spark/GenSparkProcContext.java     |    14 +-
 .../hive/ql/parse/spark/GenSparkUtils.java      |   111 +-
 .../parse/spark/OptimizeSparkProcContext.java   |    16 +-
 .../hive/ql/parse/spark/SparkCompiler.java      |   180 +-
 .../SparkPartitionPruningSinkOperator.java      |   142 +
 .../hive/ql/parse/spark/SplitOpTreeForDPP.java  |   151 +
 .../hive/ql/plan/ExprNodeConstantDesc.java      |    29 +-
 .../apache/hadoop/hive/ql/plan/FilterDesc.java  |    14 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |    32 +-
 .../hadoop/hive/ql/plan/JoinCondDesc.java       |    14 +
 .../apache/hadoop/hive/ql/plan/JoinDesc.java    |     4 +
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |    20 +-
 .../hadoop/hive/ql/plan/PartitionDesc.java      |    39 +-
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |     9 +-
 .../hadoop/hive/ql/plan/ReduceSinkDesc.java     |     1 +
 .../hive/ql/plan/SparkHashTableSinkDesc.java    |    11 +
 .../hadoop/hive/ql/plan/TableScanDesc.java      |     6 +-
 .../hive/ql/ppd/SyntheticJoinPredicate.java     |    14 +-
 .../ql/processors/CommandProcessorResponse.java |    21 +-
 .../hadoop/hive/ql/processors/HiveCommand.java  |     3 +
 .../hadoop/hive/ql/processors/SetProcessor.java |     4 +
 .../authorization/plugin/HiveOperationType.java |     5 +
 .../plugin/sqlstd/Operation2Privilege.java      |    11 +
 .../hadoop/hive/ql/session/SessionState.java    |    44 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |    54 +-
 .../hive/ql/udf/generic/GenericUDAFStd.java     |     2 +
 .../ql/udf/generic/GenericUDAFVariance.java     |     2 +
 .../hive/ql/udf/generic/GenericUDFBRound.java   |    68 +
 .../hive/ql/udf/generic/GenericUDFBridge.java   |     8 +-
 .../hive/ql/udf/generic/GenericUDFIn.java       |    14 +-
 .../hive/ql/udf/generic/GenericUDFOPAnd.java    |    59 +-
 .../hive/ql/udf/generic/GenericUDFOPOr.java     |    59 +-
 .../hive/ql/udf/generic/GenericUDFRound.java    |    41 +-
 .../hive/ql/udf/generic/GenericUDFStruct.java   |    25 +-
 .../hadoop/hive/ql/udf/generic/RoundUtils.java  |    14 +
 .../main/resources/hive-exec-log4j.properties   |    77 -
 ql/src/main/resources/hive-exec-log4j2.xml      |   110 +
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |   473 +
 .../exec/vector/TestVectorizationContext.java   |    93 +
 .../exec/vector/TestVectorizedRowBatchCtx.java  |     6 +-
 .../TestVectorLogicalExpressions.java           |   282 +
 .../hive/ql/io/orc/TestInputOutputFormat.java   |   118 +-
 .../hadoop/hive/ql/io/orc/TestOrcFile.java      |    11 +-
 .../hive/ql/io/orc/TestRecordReaderImpl.java    |    63 +-
 .../hive/ql/io/orc/TestVectorizedORCReader.java |    75 +-
 .../parquet/TestParquetRecordReaderWrapper.java |   155 +
 .../read/TestParquetFilterPredicate.java        |    51 +
 .../ql/io/sarg/TestConvertAstToSearchArg.java   |  2856 ++
 .../hive/ql/io/sarg/TestSearchArgumentImpl.java |  2891 +-
 .../hive/ql/lockmgr/TestDbTxnManager.java       |    55 +-
 .../hive/ql/lockmgr/TestDbTxnManager2.java      |     2 +-
 .../hadoop/hive/ql/log/TestLog4j2Appenders.java |    95 +
 .../hadoop/hive/ql/metadata/StringAppender.java |   128 +
 .../hadoop/hive/ql/metadata/TestHive.java       |    50 +-
 .../positive/TestTransactionStatement.java      |   102 +
 .../hive/ql/session/TestSessionState.java       |     2 +-
 .../ql/udf/generic/TestGenericUDFBRound.java    |   202 +
 .../queries/clientnegative/ctas_noemptyfolder.q |    10 +
 .../clientnegative/mismatch_columns_insertion.q |     4 +
 .../annotate_stats_deep_filters.q               |    67 +
 .../clientpositive/authorization_1_sql_std.q    |     4 +
 .../clientpositive/cast_tinyint_to_double.q     |     7 +
 ql/src/test/queries/clientpositive/cbo_rp_gby.q |    24 +
 .../queries/clientpositive/cbo_rp_gby_empty.q   |    30 +
 .../test/queries/clientpositive/cbo_rp_insert.q |    17 +
 .../test/queries/clientpositive/cbo_rp_join.q   |    65 +
 .../test/queries/clientpositive/cbo_rp_limit.q  |    16 +
 .../queries/clientpositive/cbo_rp_semijoin.q    |    17 +
 .../clientpositive/cbo_rp_simple_select.q       |    56 +
 .../test/queries/clientpositive/cbo_rp_stats.q  |    10 +
 .../queries/clientpositive/cbo_rp_subq_exists.q |    67 +
 .../queries/clientpositive/cbo_rp_subq_in.q     |    56 +
 .../queries/clientpositive/cbo_rp_subq_not_in.q |    81 +
 .../queries/clientpositive/cbo_rp_udf_udaf.q    |    20 +
 .../test/queries/clientpositive/cbo_rp_union.q  |    14 +
 .../test/queries/clientpositive/cbo_rp_views.q  |    46 +
 .../queries/clientpositive/cbo_rp_windowing.q   |    21 +
 .../queries/clientpositive/compustat_avro.q     |     8 +-
 .../test/queries/clientpositive/create_like.q   |    12 +
 .../queries/clientpositive/dynamic_rdd_cache.q  |   111 +
 .../queries/clientpositive/flatten_and_or.q     |    17 +
 ql/src/test/queries/clientpositive/groupby5.q   |     2 +
 .../clientpositive/insertoverwrite_bucket.q     |    28 +
 ql/src/test/queries/clientpositive/lineage3.q   |    22 +-
 .../clientpositive/load_dyn_part14_win.q        |    18 +-
 .../queries/clientpositive/macro_duplicate.q    |    10 +
 .../test/queries/clientpositive/orc_file_dump.q |    57 +
 .../test/queries/clientpositive/orc_ppd_basic.q |   177 +
 .../clientpositive/parquet_predicate_pushdown.q |     9 +
 .../spark_dynamic_partition_pruning.q           |   180 +
 .../spark_dynamic_partition_pruning_2.q         |   118 +
 ...spark_vectorized_dynamic_partition_pruning.q |   192 +
 .../test/queries/clientpositive/stats_ppr_all.q |    24 +
 ql/src/test/queries/clientpositive/structin.q   |    17 +
 ql/src/test/queries/clientpositive/udf_bround.q |    44 +
 .../clientpositive/udf_from_utc_timestamp.q     |    30 +-
 .../queries/clientpositive/udf_percentile.q     |     2 +
 .../clientpositive/udf_to_utc_timestamp.q       |    30 +-
 .../clientpositive/unionall_unbalancedppd.q     |   192 +
 .../test/queries/clientpositive/vector_acid3.q  |    17 +
 .../test/queries/clientpositive/vector_bround.q |    14 +
 .../clientnegative/ctas_noemptyfolder.q.out     |    19 +
 .../clientnegative/exchange_partition.q.out     |     2 +-
 .../mismatch_columns_insertion.q.out            |     9 +
 .../groupby2_map_skew_multi_distinct.q.out      |     9 +
 .../spark/groupby2_multi_distinct.q.out         |     9 +
 .../groupby3_map_skew_multi_distinct.q.out      |     9 +
 .../spark/groupby3_multi_distinct.q.out         |     9 +
 .../spark/groupby_grouping_sets7.q.out          |     9 +
 .../alter_partition_coltype.q.out               |    12 +-
 .../annotate_stats_deep_filters.q.out           |   244 +
 .../clientpositive/annotate_stats_filter.q.out  |     8 +-
 .../authorization_1_sql_std.q.out               |    11 +
 .../clientpositive/cast_tinyint_to_double.q.out |    38 +
 .../results/clientpositive/cbo_rp_gby.q.out     |   124 +
 .../clientpositive/cbo_rp_gby_empty.q.out       |    77 +
 .../results/clientpositive/cbo_rp_insert.q.out  |    89 +
 .../results/clientpositive/cbo_rp_join.q.out    | 15028 ++++++++
 .../results/clientpositive/cbo_rp_limit.q.out   |    90 +
 .../clientpositive/cbo_rp_semijoin.q.out        |   440 +
 .../clientpositive/cbo_rp_simple_select.q.out   |   755 +
 .../results/clientpositive/cbo_rp_stats.q.out   |    14 +
 .../clientpositive/cbo_rp_subq_exists.q.out     |   297 +
 .../results/clientpositive/cbo_rp_subq_in.q.out |   151 +
 .../clientpositive/cbo_rp_subq_not_in.q.out     |   365 +
 .../clientpositive/cbo_rp_udf_udaf.q.out        |   125 +
 .../results/clientpositive/cbo_rp_union.q.out   |   920 +
 .../results/clientpositive/cbo_rp_views.q.out   |   237 +
 .../clientpositive/cbo_rp_windowing.q.out       |   293 +
 .../results/clientpositive/compustat_avro.q.out |     8 +-
 .../clientpositive/convert_enum_to_string.q.out |     9 +-
 .../results/clientpositive/create_like.q.out    |    66 +
 .../clientpositive/dynamic_rdd_cache.q.out      |  1428 +
 .../clientpositive/exchange_partition.q.out     |     4 +-
 .../clientpositive/exchange_partition2.q.out    |     4 +-
 .../clientpositive/exchange_partition3.q.out    |     4 +-
 .../results/clientpositive/flatten_and_or.q.out |    66 +
 .../test/results/clientpositive/groupby5.q.out  |     8 +-
 .../groupby_multi_single_reducer2.q.out         |     2 +-
 .../groupby_multi_single_reducer3.q.out         |    12 +-
 .../clientpositive/groupby_sort_1_23.q.out      |    56 +-
 .../clientpositive/groupby_sort_skew_1_23.q.out |    56 +-
 .../clientpositive/infer_const_type.q.out       |     7 +-
 .../clientpositive/input_testxpath4.q.out       |     2 +-
 .../clientpositive/insertoverwrite_bucket.q.out |   104 +
 .../join_cond_pushdown_unqual4.q.out            |     2 +-
 .../test/results/clientpositive/lineage3.q.out  |    65 +-
 .../clientpositive/load_dyn_part14_win.q.out    |   167 +-
 .../clientpositive/macro_duplicate.q.out        |    56 +
 .../results/clientpositive/multi_insert.q.out   |     8 +-
 .../clientpositive/multi_insert_gby.q.out       |     2 +-
 .../multi_insert_lateral_view.q.out             |     4 +-
 ...i_insert_move_tasks_share_dependencies.q.out |   360 +-
 .../test/results/clientpositive/null_cast.q.out |     6 +-
 .../results/clientpositive/orc_file_dump.q.out  |   447 +
 .../clientpositive/orc_predicate_pushdown.q.out |    36 +-
 .../parquet_predicate_pushdown.q.out            |    47 +
 ql/src/test/results/clientpositive/pcr.q.out    |    12 +-
 .../results/clientpositive/ppd_gby_join.q.out   |     4 +-
 .../test/results/clientpositive/ppd_join.q.out  |     4 +-
 .../test/results/clientpositive/ppd_join2.q.out |    22 +-
 .../test/results/clientpositive/ppd_join3.q.out |    52 +-
 .../clientpositive/ppd_outer_join4.q.out        |     2 +-
 .../results/clientpositive/ppd_transform.q.out  |    12 +-
 ql/src/test/results/clientpositive/ptf.q.out    |    27 +-
 .../results/clientpositive/show_functions.q.out |     1 +
 .../clientpositive/spark/auto_join18.q.out      |    24 +-
 .../clientpositive/spark/auto_join30.q.out      |    51 +-
 .../clientpositive/spark/auto_join32.q.out      |    24 +-
 .../spark/auto_smb_mapjoin_14.q.out             |    30 +-
 .../spark/auto_sortmerge_join_10.q.out          |    23 +-
 .../results/clientpositive/spark/bucket2.q.out  |     3 -
 .../results/clientpositive/spark/bucket3.q.out  |     3 -
 .../results/clientpositive/spark/bucket4.q.out  |     3 -
 .../spark/column_access_stats.q.out             |     4 -
 .../spark/dynamic_rdd_cache.q.out               |  1073 +
 .../clientpositive/spark/groupby10.q.out        |    32 +-
 .../clientpositive/spark/groupby1_map.q.out     |   412 +
 .../spark/groupby1_map_nomap.q.out              |   408 +
 .../spark/groupby1_map_skew.q.out               |   427 +
 .../clientpositive/spark/groupby1_noskew.q.out  |   406 +
 .../clientpositive/spark/groupby2_map.q.out     |   118 +
 .../spark/groupby2_map_multi_distinct.q.out     |   232 +
 .../spark/groupby2_map_skew.q.out               |   129 +
 .../clientpositive/spark/groupby2_noskew.q.out  |   111 +
 .../spark/groupby2_noskew_multi_distinct.q.out  |   114 +
 .../clientpositive/spark/groupby4_map.q.out     |    93 +
 .../spark/groupby4_map_skew.q.out               |    93 +
 .../clientpositive/spark/groupby4_noskew.q.out  |   104 +
 .../results/clientpositive/spark/groupby5.q.out |   433 +
 .../clientpositive/spark/groupby5_map.q.out     |    95 +
 .../spark/groupby5_map_skew.q.out               |    95 +
 .../clientpositive/spark/groupby5_noskew.q.out  |   418 +
 .../results/clientpositive/spark/groupby6.q.out |   113 +
 .../clientpositive/spark/groupby6_map.q.out     |   109 +
 .../spark/groupby6_map_skew.q.out               |   122 +
 .../clientpositive/spark/groupby6_noskew.q.out  |   104 +
 .../clientpositive/spark/groupby7_map.q.out     |    23 +-
 .../spark/groupby7_map_skew.q.out               |    38 +-
 .../clientpositive/spark/groupby7_noskew.q.out  |    17 +-
 .../groupby7_noskew_multi_single_reducer.q.out  |    18 +-
 .../results/clientpositive/spark/groupby8.q.out |    62 +-
 .../spark/groupby8_map_skew.q.out               |    37 +-
 .../spark/groupby_grouping_id2.q.out            |   230 +
 .../spark/groupby_multi_single_reducer2.q.out   |     2 +-
 .../spark/groupby_multi_single_reducer3.q.out   |    12 +-
 .../clientpositive/spark/groupby_position.q.out |    37 +-
 .../spark/groupby_ppr_multi_distinct.q.out      |   346 +
 .../spark/groupby_resolution.q.out              |   796 +
 .../spark/groupby_sort_1_23.q.out               |    90 +-
 .../spark/groupby_sort_skew_1_23.q.out          |    90 +-
 .../clientpositive/spark/insert_into3.q.out     |    33 +-
 .../results/clientpositive/spark/join18.q.out   |    24 +-
 .../results/clientpositive/spark/join22.q.out   |    19 +-
 .../spark/join_cond_pushdown_unqual4.q.out      |     2 +-
 .../spark/limit_partition_metadataonly.q.out    |     2 -
 .../clientpositive/spark/limit_pushdown.q.out   |    31 +-
 .../spark/list_bucket_dml_2.q.java1.7.out       |     3 -
 .../clientpositive/spark/load_dyn_part14.q.out  |    30 +-
 .../clientpositive/spark/multi_insert.q.out     |     8 +-
 .../clientpositive/spark/multi_insert_gby.q.out |     2 +-
 .../spark/multi_insert_lateral_view.q.out       |     4 +-
 ...i_insert_move_tasks_share_dependencies.q.out |   536 +-
 .../clientpositive/spark/nullgroup.q.out        |   265 +
 .../clientpositive/spark/nullgroup2.q.out       |   300 +
 .../clientpositive/spark/nullgroup4.q.out       |   292 +
 .../spark/nullgroup4_multi_distinct.q.out       |   133 +
 .../spark/optimize_nullscan.q.out               |     3 -
 .../test/results/clientpositive/spark/pcr.q.out |    18 +-
 .../clientpositive/spark/ppd_gby_join.q.out     |     4 +-
 .../results/clientpositive/spark/ppd_join.q.out |     4 +-
 .../clientpositive/spark/ppd_join2.q.out        |    22 +-
 .../clientpositive/spark/ppd_join3.q.out        |    52 +-
 .../clientpositive/spark/ppd_outer_join4.q.out  |     2 +-
 .../clientpositive/spark/ppd_transform.q.out    |    12 +-
 .../test/results/clientpositive/spark/ptf.q.out |    17 +-
 .../results/clientpositive/spark/sample3.q.out  |     3 -
 .../results/clientpositive/spark/sample9.q.out  |     3 -
 .../clientpositive/spark/skewjoinopt11.q.out    |    60 +-
 .../clientpositive/spark/skewjoinopt9.q.out     |    20 +-
 .../clientpositive/spark/smb_mapjoin_11.q.out   |     6 -
 .../spark/spark_dynamic_partition_pruning.q.out |  5573 +++
 .../spark_dynamic_partition_pruning_2.q.out     |  1015 +
 ...k_vectorized_dynamic_partition_pruning.q.out |  5822 +++
 .../clientpositive/spark/temp_table_gb1.q.out   |    67 +
 .../clientpositive/spark/udaf_collect_set.q.out |   212 +
 .../clientpositive/spark/udf_example_add.q.out  |     3 -
 .../clientpositive/spark/udf_in_file.q.out      |     3 -
 .../results/clientpositive/spark/udf_max.q.out  |    62 +
 .../results/clientpositive/spark/udf_min.q.out  |    62 +
 .../clientpositive/spark/udf_percentile.q.out   |   450 +
 .../results/clientpositive/spark/union10.q.out  |    36 +-
 .../results/clientpositive/spark/union11.q.out  |    38 +-
 .../results/clientpositive/spark/union15.q.out  |    23 +-
 .../results/clientpositive/spark/union16.q.out  |   450 +-
 .../results/clientpositive/spark/union2.q.out   |    36 +-
 .../results/clientpositive/spark/union20.q.out  |    18 +-
 .../results/clientpositive/spark/union25.q.out  |    21 +-
 .../results/clientpositive/spark/union28.q.out  |    21 +-
 .../results/clientpositive/spark/union3.q.out   |    45 +-
 .../results/clientpositive/spark/union30.q.out  |    21 +-
 .../results/clientpositive/spark/union4.q.out   |    18 +-
 .../results/clientpositive/spark/union5.q.out   |    20 +-
 .../results/clientpositive/spark/union9.q.out   |    54 +-
 .../clientpositive/spark/union_remove_1.q.out   |    23 +-
 .../clientpositive/spark/union_remove_15.q.out  |    23 +-
 .../clientpositive/spark/union_remove_16.q.out  |    23 +-
 .../clientpositive/spark/union_remove_18.q.out  |    23 +-
 .../clientpositive/spark/union_remove_19.q.out  |    75 +-
 .../clientpositive/spark/union_remove_20.q.out  |    23 +-
 .../clientpositive/spark/union_remove_21.q.out  |    21 +-
 .../clientpositive/spark/union_remove_22.q.out  |    46 +-
 .../clientpositive/spark/union_remove_24.q.out  |    23 +-
 .../clientpositive/spark/union_remove_25.q.out  |    59 +-
 .../clientpositive/spark/union_remove_4.q.out   |    23 +-
 .../clientpositive/spark/union_remove_6.q.out   |    23 +-
 .../spark/union_remove_6_subq.q.out             |    84 +-
 .../clientpositive/spark/union_remove_7.q.out   |    23 +-
 .../clientpositive/spark/union_top_level.q.out  |    59 +-
 .../clientpositive/spark/union_view.q.out       |    33 +-
 .../spark/vector_count_distinct.q.out           |    31 +-
 .../spark/vector_decimal_mapjoin.q.out          |     1 +
 .../clientpositive/spark/vector_elt.q.out       |     7 -
 .../spark/vector_left_outer_join.q.out          |     2 +
 .../spark/vector_mapjoin_reduce.q.out           |     1 +
 .../spark/vector_string_concat.q.out            |     3 -
 .../clientpositive/spark/vectorization_0.q.out  |     2 +-
 .../clientpositive/spark/vectorization_13.q.out |     4 +-
 .../clientpositive/spark/vectorization_15.q.out |     2 +-
 .../clientpositive/spark/vectorization_17.q.out |    12 +-
 .../spark/vectorization_decimal_date.q.out      |     4 -
 .../spark/vectorization_div0.q.out              |     3 -
 .../spark/vectorization_short_regress.q.out     |    40 +-
 .../clientpositive/spark/vectorized_case.q.out  |     5 +-
 .../spark/vectorized_mapjoin.q.out              |     1 +
 .../spark/vectorized_math_funcs.q.out           |     3 -
 .../spark/vectorized_nested_mapjoin.q.out       |     2 +
 .../clientpositive/spark/vectorized_ptf.q.out   |    21 +-
 .../spark/vectorized_string_funcs.q.out         |     3 -
 .../results/clientpositive/stats_ppr_all.q.out  |   284 +
 .../test/results/clientpositive/structin.q.out  |    66 +
 .../clientpositive/tez/explainuser_1.q.out      |    71 +-
 .../clientpositive/tez/orc_ppd_basic.q.out      |   701 +
 .../test/results/clientpositive/tez/ptf.q.out   |    15 +-
 .../results/clientpositive/tez/union2.q.out     |    28 +-
 .../results/clientpositive/tez/union9.q.out     |    40 +-
 .../clientpositive/tez/vector_acid3.q.out       |    31 +
 .../tez/vector_mr_diff_schema_alias.q.out       |     2 +-
 .../tez/vector_null_projection.q.out            |     4 +
 .../clientpositive/tez/vectorization_0.q.out    |     2 +-
 .../clientpositive/tez/vectorization_13.q.out   |     4 +-
 .../clientpositive/tez/vectorization_15.q.out   |     2 +-
 .../clientpositive/tez/vectorization_17.q.out   |    12 +-
 .../clientpositive/tez/vectorization_7.q.out    |     4 +-
 .../clientpositive/tez/vectorization_8.q.out    |     4 +-
 .../tez/vectorization_short_regress.q.out       |    40 +-
 .../clientpositive/tez/vectorized_case.q.out    |     2 +-
 .../clientpositive/tez/vectorized_ptf.q.out     |    19 +-
 .../results/clientpositive/udf_bround.q.out     |   119 +
 .../clientpositive/udf_from_utc_timestamp.q.out |    66 +-
 .../results/clientpositive/udf_inline.q.out     |     8 +-
 ql/src/test/results/clientpositive/udf_or.q.out |     4 +-
 .../results/clientpositive/udf_percentile.q.out |   104 +-
 .../results/clientpositive/udf_struct.q.out     |     2 +-
 .../clientpositive/udf_to_utc_timestamp.q.out   |    66 +-
 .../test/results/clientpositive/udf_union.q.out |     2 +-
 .../test/results/clientpositive/union16.q.out   |   354 +-
 ql/src/test/results/clientpositive/union2.q.out |    32 +-
 ql/src/test/results/clientpositive/union9.q.out |    46 +-
 .../clientpositive/union_remove_6_subq.q.out    |    34 +-
 .../results/clientpositive/union_view.q.out     |    24 +
 .../clientpositive/unionall_unbalancedppd.q.out |   653 +
 .../results/clientpositive/vector_acid3.q.out   |    31 +
 .../results/clientpositive/vector_bround.q.out  |    86 +
 .../vector_mr_diff_schema_alias.q.out           |     2 +-
 .../clientpositive/vector_null_projection.q.out |     4 +
 .../clientpositive/vectorization_0.q.out        |     2 +-
 .../clientpositive/vectorization_13.q.out       |     4 +-
 .../clientpositive/vectorization_15.q.out       |     2 +-
 .../clientpositive/vectorization_17.q.out       |    12 +-
 .../clientpositive/vectorization_7.q.out        |     4 +-
 .../clientpositive/vectorization_8.q.out        |     4 +-
 .../vectorization_short_regress.q.out           |    40 +-
 .../clientpositive/vectorized_case.q.out        |     9 +-
 .../results/clientpositive/vectorized_ptf.q.out |    67 +-
 .../gen/thrift/gen-cpp/complex_constants.cpp    |     2 +-
 .../src/gen/thrift/gen-cpp/complex_constants.h  |     2 +-
 serde/src/gen/thrift/gen-cpp/complex_types.cpp  |   442 +-
 serde/src/gen/thrift/gen-cpp/complex_types.h    |   174 +-
 .../gen/thrift/gen-cpp/megastruct_constants.cpp |     2 +-
 .../gen/thrift/gen-cpp/megastruct_constants.h   |     2 +-
 .../src/gen/thrift/gen-cpp/megastruct_types.cpp |   585 +-
 serde/src/gen/thrift/gen-cpp/megastruct_types.h |   175 +-
 .../src/gen/thrift/gen-cpp/serde_constants.cpp  |     2 +-
 serde/src/gen/thrift/gen-cpp/serde_constants.h  |     2 +-
 serde/src/gen/thrift/gen-cpp/serde_types.cpp    |     5 +-
 serde/src/gen/thrift/gen-cpp/serde_types.h      |     5 +-
 .../gen/thrift/gen-cpp/testthrift_constants.cpp |     2 +-
 .../gen/thrift/gen-cpp/testthrift_constants.h   |     2 +-
 .../src/gen/thrift/gen-cpp/testthrift_types.cpp |    95 +-
 serde/src/gen/thrift/gen-cpp/testthrift_types.h |    45 +-
 .../hadoop/hive/serde/serdeConstants.java       |     7 +-
 .../hadoop/hive/serde/test/InnerStruct.java     |    24 +-
 .../hadoop/hive/serde/test/ThriftTestObj.java   |    62 +-
 .../hadoop/hive/serde2/thrift/test/Complex.java |   279 +-
 .../hive/serde2/thrift/test/IntString.java      |    40 +-
 .../hive/serde2/thrift/test/MegaStruct.java     |   521 +-
 .../hive/serde2/thrift/test/MiniStruct.java     |    38 +-
 .../hadoop/hive/serde2/thrift/test/MyEnum.java  |     2 +-
 .../hive/serde2/thrift/test/PropValueUnion.java |    60 +-
 .../hive/serde2/thrift/test/SetIntString.java   |    54 +-
 serde/src/gen/thrift/gen-php/Types.php          |    15 +-
 .../org/apache/hadoop/hive/serde/Types.php      |   373 +-
 .../src/gen/thrift/gen-py/complex/constants.py  |     2 +-
 serde/src/gen/thrift/gen-py/complex/ttypes.py   |    50 +-
 .../gen/thrift/gen-py/megastruct/constants.py   |     2 +-
 .../src/gen/thrift/gen-py/megastruct/ttypes.py  |    44 +-
 .../org_apache_hadoop_hive_serde/constants.py   |     2 +-
 .../org_apache_hadoop_hive_serde/ttypes.py      |     2 +-
 .../gen/thrift/gen-py/testthrift/constants.py   |     2 +-
 .../src/gen/thrift/gen-py/testthrift/ttypes.py  |    14 +-
 .../src/gen/thrift/gen-rb/complex_constants.rb  |     2 +-
 serde/src/gen/thrift/gen-rb/complex_types.rb    |     2 +-
 .../gen/thrift/gen-rb/megastruct_constants.rb   |     2 +-
 serde/src/gen/thrift/gen-rb/megastruct_types.rb |     2 +-
 serde/src/gen/thrift/gen-rb/serde_constants.rb  |     2 +-
 serde/src/gen/thrift/gen-rb/serde_types.rb      |     2 +-
 .../gen/thrift/gen-rb/testthrift_constants.rb   |     2 +-
 serde/src/gen/thrift/gen-rb/testthrift_types.rb |     2 +-
 .../hadoop/hive/ql/io/sarg/ExpressionTree.java  |   157 -
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   |    87 -
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  |   278 -
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |    14 +-
 .../apache/hadoop/hive/serde2/WriteBuffers.java |     2 +-
 .../hadoop/hive/serde2/avro/InstanceCache.java  |     9 +-
 .../hive/serde2/io/HiveDecimalWritable.java     |   185 -
 .../hive/serde2/lazy/LazyHiveDecimal.java       |     3 +-
 .../lazy/fast/LazySimpleDeserializeRead.java    |    16 +-
 .../lazybinary/LazyBinaryHiveDecimal.java       |     2 +-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java |    26 +-
 .../fast/LazyBinaryDeserializeRead.java         |     6 +-
 .../fast/LazyBinarySerializeWrite.java          |     5 +-
 .../objectinspector/ObjectInspectorFactory.java |     7 +-
 .../objectinspector/ObjectInspectorUtils.java   |    19 +
 .../StandardConstantStructObjectInspector.java  |    51 +
 .../hive/serde2/typeinfo/HiveDecimalUtils.java  |    35 +-
 .../hive/serde2/typeinfo/TypeInfoUtils.java     |     2 +-
 .../hive/serde2/avro/TestInstanceCache.java     |    40 +-
 service/src/gen/thrift/gen-cpp/TCLIService.cpp  |   458 +-
 service/src/gen/thrift/gen-cpp/TCLIService.h    |   821 +-
 .../thrift/gen-cpp/TCLIService_constants.cpp    |     2 +-
 .../gen/thrift/gen-cpp/TCLIService_constants.h  |     2 +-
 .../gen/thrift/gen-cpp/TCLIService_types.cpp    |  3250 +-
 .../src/gen/thrift/gen-cpp/TCLIService_types.h  |  1482 +-
 service/src/gen/thrift/gen-cpp/ThriftHive.cpp   |   286 +-
 service/src/gen/thrift/gen-cpp/ThriftHive.h     |   389 +-
 .../thrift/gen-cpp/hive_service_constants.cpp   |     2 +-
 .../gen/thrift/gen-cpp/hive_service_constants.h |     2 +-
 .../gen/thrift/gen-cpp/hive_service_types.cpp   |   110 +-
 .../src/gen/thrift/gen-cpp/hive_service_types.h |    75 +-
 .../hadoop/hive/service/HiveClusterStatus.java  |    68 +-
 .../hive/service/HiveServerException.java       |    40 +-
 .../hadoop/hive/service/JobTrackerState.java    |     2 +-
 .../apache/hadoop/hive/service/ThriftHive.java  |   914 +-
 .../service/cli/thrift/TArrayTypeEntry.java     |    24 +-
 .../hive/service/cli/thrift/TBinaryColumn.java  |    64 +-
 .../hive/service/cli/thrift/TBoolColumn.java    |    62 +-
 .../hive/service/cli/thrift/TBoolValue.java     |    26 +-
 .../hive/service/cli/thrift/TByteColumn.java    |    62 +-
 .../hive/service/cli/thrift/TByteValue.java     |    26 +-
 .../hive/service/cli/thrift/TCLIService.java    |  1734 +-
 .../cli/thrift/TCLIServiceConstants.java        |     7 +-
 .../cli/thrift/TCancelDelegationTokenReq.java   |    32 +-
 .../cli/thrift/TCancelDelegationTokenResp.java  |    24 +-
 .../service/cli/thrift/TCancelOperationReq.java |    24 +-
 .../cli/thrift/TCancelOperationResp.java        |    24 +-
 .../service/cli/thrift/TCloseOperationReq.java  |    24 +-
 .../service/cli/thrift/TCloseOperationResp.java |    24 +-
 .../service/cli/thrift/TCloseSessionReq.java    |    24 +-
 .../service/cli/thrift/TCloseSessionResp.java   |    24 +-
 .../apache/hive/service/cli/thrift/TColumn.java |    20 +-
 .../hive/service/cli/thrift/TColumnDesc.java    |    50 +-
 .../hive/service/cli/thrift/TColumnValue.java   |    20 +-
 .../hive/service/cli/thrift/TDoubleColumn.java  |    62 +-
 .../hive/service/cli/thrift/TDoubleValue.java   |    26 +-
 .../cli/thrift/TExecuteStatementReq.java        |    87 +-
 .../cli/thrift/TExecuteStatementResp.java       |    34 +-
 .../service/cli/thrift/TFetchOrientation.java   |     2 +-
 .../service/cli/thrift/TFetchResultsReq.java    |    54 +-
 .../service/cli/thrift/TFetchResultsResp.java   |    42 +-
 .../service/cli/thrift/TGetCatalogsReq.java     |    24 +-
 .../service/cli/thrift/TGetCatalogsResp.java    |    34 +-
 .../hive/service/cli/thrift/TGetColumnsReq.java |    58 +-
 .../service/cli/thrift/TGetColumnsResp.java     |    34 +-
 .../cli/thrift/TGetDelegationTokenReq.java      |    40 +-
 .../cli/thrift/TGetDelegationTokenResp.java     |    34 +-
 .../service/cli/thrift/TGetFunctionsReq.java    |    50 +-
 .../service/cli/thrift/TGetFunctionsResp.java   |    34 +-
 .../hive/service/cli/thrift/TGetInfoReq.java    |    36 +-
 .../hive/service/cli/thrift/TGetInfoResp.java   |    32 +-
 .../hive/service/cli/thrift/TGetInfoType.java   |     2 +-
 .../hive/service/cli/thrift/TGetInfoValue.java  |    20 +-
 .../cli/thrift/TGetOperationStatusReq.java      |    24 +-
 .../cli/thrift/TGetOperationStatusResp.java     |    62 +-
 .../cli/thrift/TGetResultSetMetadataReq.java    |    24 +-
 .../cli/thrift/TGetResultSetMetadataResp.java   |    34 +-
 .../hive/service/cli/thrift/TGetSchemasReq.java |    42 +-
 .../service/cli/thrift/TGetSchemasResp.java     |    34 +-
 .../service/cli/thrift/TGetTableTypesReq.java   |    24 +-
 .../service/cli/thrift/TGetTableTypesResp.java  |    34 +-
 .../hive/service/cli/thrift/TGetTablesReq.java  |    79 +-
 .../hive/service/cli/thrift/TGetTablesResp.java |    34 +-
 .../service/cli/thrift/TGetTypeInfoReq.java     |    24 +-
 .../service/cli/thrift/TGetTypeInfoResp.java    |    34 +-
 .../service/cli/thrift/THandleIdentifier.java   |    50 +-
 .../hive/service/cli/thrift/TI16Column.java     |    62 +-
 .../hive/service/cli/thrift/TI16Value.java      |    26 +-
 .../hive/service/cli/thrift/TI32Column.java     |    62 +-
 .../hive/service/cli/thrift/TI32Value.java      |    26 +-
 .../hive/service/cli/thrift/TI64Column.java     |    62 +-
 .../hive/service/cli/thrift/TI64Value.java      |    26 +-
 .../hive/service/cli/thrift/TMapTypeEntry.java  |    32 +-
 .../service/cli/thrift/TOpenSessionReq.java     |    91 +-
 .../service/cli/thrift/TOpenSessionResp.java    |    91 +-
 .../service/cli/thrift/TOperationHandle.java    |    54 +-
 .../service/cli/thrift/TOperationState.java     |     2 +-
 .../hive/service/cli/thrift/TOperationType.java |     2 +-
 .../service/cli/thrift/TPrimitiveTypeEntry.java |    38 +-
 .../service/cli/thrift/TProtocolVersion.java    |     2 +-
 .../cli/thrift/TRenewDelegationTokenReq.java    |    32 +-
 .../cli/thrift/TRenewDelegationTokenResp.java   |    24 +-
 .../apache/hive/service/cli/thrift/TRow.java    |    46 +-
 .../apache/hive/service/cli/thrift/TRowSet.java |    86 +-
 .../hive/service/cli/thrift/TSessionHandle.java |    24 +-
 .../apache/hive/service/cli/thrift/TStatus.java |    83 +-
 .../hive/service/cli/thrift/TStatusCode.java    |     2 +-
 .../hive/service/cli/thrift/TStringColumn.java  |    62 +-
 .../hive/service/cli/thrift/TStringValue.java   |    26 +-
 .../service/cli/thrift/TStructTypeEntry.java    |    50 +-
 .../hive/service/cli/thrift/TTableSchema.java   |    46 +-
 .../hive/service/cli/thrift/TTypeDesc.java      |    46 +-
 .../hive/service/cli/thrift/TTypeEntry.java     |    20 +-
 .../apache/hive/service/cli/thrift/TTypeId.java |     2 +-
 .../service/cli/thrift/TTypeQualifierValue.java |    20 +-
 .../service/cli/thrift/TTypeQualifiers.java     |    54 +-
 .../service/cli/thrift/TUnionTypeEntry.java     |    50 +-
 .../cli/thrift/TUserDefinedTypeEntry.java       |    24 +-
 service/src/gen/thrift/gen-php/TCLIService.php  |   269 +-
 service/src/gen/thrift/gen-php/ThriftHive.php   |   125 +-
 service/src/gen/thrift/gen-php/Types.php        |    30 +-
 .../gen-py/TCLIService/TCLIService-remote       |   102 +-
 .../thrift/gen-py/TCLIService/TCLIService.py    |   439 +-
 .../gen/thrift/gen-py/TCLIService/constants.py  |     2 +-
 .../src/gen/thrift/gen-py/TCLIService/ttypes.py |   478 +-
 .../gen-py/hive_service/ThriftHive-remote       |  1021 +-
 .../thrift/gen-py/hive_service/ThriftHive.py    |   266 +-
 .../gen/thrift/gen-py/hive_service/constants.py |     2 +-
 .../gen/thrift/gen-py/hive_service/ttypes.py    |    19 +-
 .../gen/thrift/gen-rb/hive_service_constants.rb |     2 +-
 .../src/gen/thrift/gen-rb/hive_service_types.rb |     2 +-
 .../src/gen/thrift/gen-rb/t_c_l_i_service.rb    |     2 +-
 .../thrift/gen-rb/t_c_l_i_service_constants.rb  |     2 +-
 .../gen/thrift/gen-rb/t_c_l_i_service_types.rb  |     2 +-
 service/src/gen/thrift/gen-rb/thrift_hive.rb    |     2 +-
 .../hive/service/cli/CLIServiceUtils.java       |     7 -
 .../cli/operation/GetColumnsOperation.java      |    10 +-
 .../cli/operation/GetTablesOperation.java       |     7 +-
 .../cli/operation/LogDivertAppender.java        |   223 +-
 .../service/cli/operation/OperationManager.java |    17 +-
 .../service/cli/session/SessionManager.java     |    42 +-
 .../session/TestPluggableHiveSessionImpl.java   |    55 +
 shims/common/pom.xml                            |    17 +-
 .../hadoop/hive/shims/HiveEventCounter.java     |   102 -
 spark-client/pom.xml                            |     5 +
 .../hive/spark/client/SparkClientImpl.java      |    20 +-
 .../hive/spark/client/SparkClientUtilities.java |    13 +-
 .../hive/spark/client/TestSparkClient.java      |     4 +-
 .../src/test/resources/log4j.properties         |    23 -
 spark-client/src/test/resources/log4j2.xml      |    39 +
 storage-api/pom.xml                             |    78 +
 .../hadoop/hive/common/type/HiveDecimal.java    |   313 +
 .../hive/ql/exec/vector/BytesColumnVector.java  |   322 +
 .../hive/ql/exec/vector/ColumnVector.java       |   173 +
 .../ql/exec/vector/DecimalColumnVector.java     |   106 +
 .../hive/ql/exec/vector/DoubleColumnVector.java |   143 +
 .../hive/ql/exec/vector/LongColumnVector.java   |   189 +
 .../hive/ql/exec/vector/VectorizedRowBatch.java |   186 +
 .../hadoop/hive/ql/io/sarg/ExpressionTree.java  |   156 +
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   |   104 +
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  |   287 +
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |    28 +
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |   687 +
 .../hive/serde2/io/HiveDecimalWritable.java     |   174 +
 testutils/ptest2/pom.xml                        |    20 +
 .../ptest2/src/main/resources/log4j.properties  |    37 -
 testutils/ptest2/src/main/resources/log4j2.xml  |    79 +
 955 files changed, 130154 insertions(+), 43331 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/itests/qtest/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --cc itests/src/test/resources/testconfiguration.properties
index eb986db,bed621d..c877f85
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@@ -1142,48 -1184,8 +1184,54 @@@ miniSparkOnYarn.query.files=auto_sortme
    truncate_column_buckets.q,\
    uber_reduce.q
  
 +miniHbaseMetastore.query.files=join1.q,\
 +join2.q,\
 +mapjoin1.q,\
 +add_part_multiple.q,\
 +annotate_stats_join.q,\
 +authorization_parts.q,\
 +auto_join1.q,\
 +bucket1.q,\
 +compute_stats_string.q,\
 +create_1.q,\
 +groupby1.q,\
 +groupby12.q,\
 +having.q,\
 +innerjoin.q,\
 +input_part10.q,\
 +input20.q,\
 +join1.q,\
 +join20.q,\
 +leftsemijoin.q,\
 +mapjoin1.q,\
 +multi_insert_gby.q,\
 +orc_create.q,\
 +orc_merge1.q,\
 +show_roles.q,\
 +stats0.q,\
 +statsfs.q,\
 +temp_table.q,\
 +union.q,\
 +union10.q,\
 +alter_partition_change_col,\
 +alter1.q,\
 +analyze_tbl_part.q,\
 +authorization_1.q,\
 +columnstats_part_coltype.q,\
 +ctas.q,\
 +database.q,\
 +drop_partition_with_stats.q,\
 +drop_table_with_stats.q,\
 +inputddl8.q,\
 +order2.q,\
 +partition_date.q,\
 +partition_multilevels.q,\
 +show_partitions.q,\
 +sort.q,\
 +view.q
++
+ spark.query.negative.files=groupby2_map_skew_multi_distinct.q,\
+   groupby2_multi_distinct.q,\
+   groupby3_map_skew_multi_distinct.q,\
+   groupby3_multi_distinct.q,\
+   groupby_grouping_sets7.q

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index fbc8400,0354fe1..fcc4f0b
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@@ -28313,116 -31450,6 +31450,140 @@@ uint32_t ThriftHiveMetastore_fire_liste
    return xfer;
  }
  
++
++ThriftHiveMetastore_flushCache_args::~ThriftHiveMetastore_flushCache_args() throw() {
++}
++
++
 +uint32_t ThriftHiveMetastore_flushCache_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 +
 +  uint32_t xfer = 0;
 +  std::string fname;
 +  ::apache::thrift::protocol::TType ftype;
 +  int16_t fid;
 +
 +  xfer += iprot->readStructBegin(fname);
 +
 +  using ::apache::thrift::protocol::TProtocolException;
 +
 +
 +  while (true)
 +  {
 +    xfer += iprot->readFieldBegin(fname, ftype, fid);
 +    if (ftype == ::apache::thrift::protocol::T_STOP) {
 +      break;
 +    }
 +    xfer += iprot->skip(ftype);
 +    xfer += iprot->readFieldEnd();
 +  }
 +
 +  xfer += iprot->readStructEnd();
 +
 +  return xfer;
 +}
 +
 +uint32_t ThriftHiveMetastore_flushCache_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
 +  uint32_t xfer = 0;
++  oprot->incrementRecursionDepth();
 +  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_args");
 +
 +  xfer += oprot->writeFieldStop();
 +  xfer += oprot->writeStructEnd();
++  oprot->decrementRecursionDepth();
 +  return xfer;
 +}
 +
++
++ThriftHiveMetastore_flushCache_pargs::~ThriftHiveMetastore_flushCache_pargs() throw() {
++}
++
++
 +uint32_t ThriftHiveMetastore_flushCache_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
 +  uint32_t xfer = 0;
++  oprot->incrementRecursionDepth();
 +  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_pargs");
 +
 +  xfer += oprot->writeFieldStop();
 +  xfer += oprot->writeStructEnd();
++  oprot->decrementRecursionDepth();
 +  return xfer;
 +}
 +
++
++ThriftHiveMetastore_flushCache_result::~ThriftHiveMetastore_flushCache_result() throw() {
++}
++
++
 +uint32_t ThriftHiveMetastore_flushCache_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 +
 +  uint32_t xfer = 0;
 +  std::string fname;
 +  ::apache::thrift::protocol::TType ftype;
 +  int16_t fid;
 +
 +  xfer += iprot->readStructBegin(fname);
 +
 +  using ::apache::thrift::protocol::TProtocolException;
 +
 +
 +  while (true)
 +  {
 +    xfer += iprot->readFieldBegin(fname, ftype, fid);
 +    if (ftype == ::apache::thrift::protocol::T_STOP) {
 +      break;
 +    }
 +    xfer += iprot->skip(ftype);
 +    xfer += iprot->readFieldEnd();
 +  }
 +
 +  xfer += iprot->readStructEnd();
 +
 +  return xfer;
 +}
 +
 +uint32_t ThriftHiveMetastore_flushCache_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 +
 +  uint32_t xfer = 0;
 +
 +  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_result");
 +
 +  xfer += oprot->writeFieldStop();
 +  xfer += oprot->writeStructEnd();
 +  return xfer;
 +}
 +
++
++ThriftHiveMetastore_flushCache_presult::~ThriftHiveMetastore_flushCache_presult() throw() {
++}
++
++
 +uint32_t ThriftHiveMetastore_flushCache_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 +
 +  uint32_t xfer = 0;
 +  std::string fname;
 +  ::apache::thrift::protocol::TType ftype;
 +  int16_t fid;
 +
 +  xfer += iprot->readStructBegin(fname);
 +
 +  using ::apache::thrift::protocol::TProtocolException;
 +
 +
 +  while (true)
 +  {
 +    xfer += iprot->readFieldBegin(fname, ftype, fid);
 +    if (ftype == ::apache::thrift::protocol::T_STOP) {
 +      break;
 +    }
 +    xfer += iprot->skip(ftype);
 +    xfer += iprot->readFieldEnd();
 +  }
 +
 +  xfer += iprot->readStructEnd();
 +
 +  return xfer;
 +}
 +
  void ThriftHiveMetastoreClient::getMetaConf(std::string& _return, const std::string& key)
  {
    send_getMetaConf(key);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index a534c2e,520c6e3..baa28e3
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@@ -17333,82 -18190,9 +18194,99 @@@ class ThriftHiveMetastore_fire_listener
  
    uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
  
+   friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_fire_listener_event_presult& obj);
  };
  
 +
 +class ThriftHiveMetastore_flushCache_args {
 + public:
 +
++  static const char* ascii_fingerprint; // = "99914B932BD37A50B983C5E7C90AE93B";
++  static const uint8_t binary_fingerprint[16]; // = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B};
++
++  ThriftHiveMetastore_flushCache_args(const ThriftHiveMetastore_flushCache_args&);
++  ThriftHiveMetastore_flushCache_args& operator=(const ThriftHiveMetastore_flushCache_args&);
 +  ThriftHiveMetastore_flushCache_args() {
 +  }
 +
-   virtual ~ThriftHiveMetastore_flushCache_args() throw() {}
- 
++  virtual ~ThriftHiveMetastore_flushCache_args() throw();
 +
 +  bool operator == (const ThriftHiveMetastore_flushCache_args & /* rhs */) const
 +  {
 +    return true;
 +  }
 +  bool operator != (const ThriftHiveMetastore_flushCache_args &rhs) const {
 +    return !(*this == rhs);
 +  }
 +
 +  bool operator < (const ThriftHiveMetastore_flushCache_args & ) const;
 +
 +  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
 +  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
 +
++  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_flushCache_args& obj);
 +};
 +
 +
 +class ThriftHiveMetastore_flushCache_pargs {
 + public:
 +
++  static const char* ascii_fingerprint; // = "99914B932BD37A50B983C5E7C90AE93B";
++  static const uint8_t binary_fingerprint[16]; // = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B};
 +
-   virtual ~ThriftHiveMetastore_flushCache_pargs() throw() {}
 +
++  virtual ~ThriftHiveMetastore_flushCache_pargs() throw();
 +
 +  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
 +
++  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_flushCache_pargs& obj);
 +};
 +
 +
 +class ThriftHiveMetastore_flushCache_result {
 + public:
 +
++  static const char* ascii_fingerprint; // = "99914B932BD37A50B983C5E7C90AE93B";
++  static const uint8_t binary_fingerprint[16]; // = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B};
++
++  ThriftHiveMetastore_flushCache_result(const ThriftHiveMetastore_flushCache_result&);
++  ThriftHiveMetastore_flushCache_result& operator=(const ThriftHiveMetastore_flushCache_result&);
 +  ThriftHiveMetastore_flushCache_result() {
 +  }
 +
-   virtual ~ThriftHiveMetastore_flushCache_result() throw() {}
- 
++  virtual ~ThriftHiveMetastore_flushCache_result() throw();
 +
 +  bool operator == (const ThriftHiveMetastore_flushCache_result & /* rhs */) const
 +  {
 +    return true;
 +  }
 +  bool operator != (const ThriftHiveMetastore_flushCache_result &rhs) const {
 +    return !(*this == rhs);
 +  }
 +
 +  bool operator < (const ThriftHiveMetastore_flushCache_result & ) const;
 +
 +  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
 +  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
 +
++  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_flushCache_result& obj);
 +};
 +
 +
 +class ThriftHiveMetastore_flushCache_presult {
 + public:
 +
++  static const char* ascii_fingerprint; // = "99914B932BD37A50B983C5E7C90AE93B";
++  static const uint8_t binary_fingerprint[16]; // = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B};
 +
-   virtual ~ThriftHiveMetastore_flushCache_presult() throw() {}
 +
++  virtual ~ThriftHiveMetastore_flushCache_presult() throw();
 +
 +  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
 +
++  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_flushCache_presult& obj);
 +};
 +
  class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public  ::facebook::fb303::FacebookServiceClient {
   public:
    ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java
index 5d44585,37e5bf1..ae12142
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class AbortTxnRequest implements org.apache.thrift.TBase<AbortTxnRequest, AbortTxnRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class AbortTxnRequest implements org.apache.thrift.TBase<AbortTxnRequest, AbortTxnRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AbortTxnRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AbortTxnRequest");
  
    private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index afb2b79,9c78c49..f60521f
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicPartitions, AddDynamicPartitions._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicPartitions, AddDynamicPartitions._Fields>, java.io.Serializable, Cloneable, Comparable<AddDynamicPartitions> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddDynamicPartitions");
  
    private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
index 7bb72ba,fcfaaf3..00a7236
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitionsRequest, AddPartitionsRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitionsRequest, AddPartitionsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AddPartitionsRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddPartitionsRequest");
  
    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
index 875e04a,9022019..7150e68
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartitionsResult, AddPartitionsResult._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartitionsResult, AddPartitionsResult._Fields>, java.io.Serializable, Cloneable, Comparable<AddPartitionsResult> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddPartitionsResult");
  
    private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
index 0e1dbcb,917cec0..bf14ac0
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class AggrStats implements org.apache.thrift.TBase<AggrStats, AggrStats._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class AggrStats implements org.apache.thrift.TBase<AggrStats, AggrStats._Fields>, java.io.Serializable, Cloneable, Comparable<AggrStats> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AggrStats");
  
    private static final org.apache.thrift.protocol.TField COL_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("colStats", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java
index 2a5e58f,d7a317b..16f2cb7
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class AlreadyExistsException extends TException implements org.apache.thrift.TBase<AlreadyExistsException, AlreadyExistsException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class AlreadyExistsException extends TException implements org.apache.thrift.TBase<AlreadyExistsException, AlreadyExistsException._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExistsException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExistsException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java
index 1516b25,00b312d..7ddb91a
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class BinaryColumnStatsData implements org.apache.thrift.TBase<BinaryColumnStatsData, BinaryColumnStatsData._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class BinaryColumnStatsData implements org.apache.thrift.TBase<BinaryColumnStatsData, BinaryColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<BinaryColumnStatsData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BinaryColumnStatsData");
  
    private static final org.apache.thrift.protocol.TField MAX_COL_LEN_FIELD_DESC = new org.apache.thrift.protocol.TField("maxColLen", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java
index 9ef9c0f,a0f3ab8..f98e56b
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class BooleanColumnStatsData implements org.apache.thrift.TBase<BooleanColumnStatsData, BooleanColumnStatsData._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class BooleanColumnStatsData implements org.apache.thrift.TBase<BooleanColumnStatsData, BooleanColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<BooleanColumnStatsData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BooleanColumnStatsData");
  
    private static final org.apache.thrift.protocol.TField NUM_TRUES_FIELD_DESC = new org.apache.thrift.protocol.TField("numTrues", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java
index ae34203,82e3031..667d12e
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class CheckLockRequest implements org.apache.thrift.TBase<CheckLockRequest, CheckLockRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class CheckLockRequest implements org.apache.thrift.TBase<CheckLockRequest, CheckLockRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CheckLockRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CheckLockRequest");
  
    private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
index b34619f,510dace..dd9aeb7
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ColumnStatistics implements org.apache.thrift.TBase<ColumnStatistics, ColumnStatistics._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ColumnStatistics implements org.apache.thrift.TBase<ColumnStatistics, ColumnStatistics._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnStatistics> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnStatistics");
  
    private static final org.apache.thrift.protocol.TField STATS_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("statsDesc", org.apache.thrift.protocol.TType.STRUCT, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
index e951d04,cf967b6..09f925c
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ColumnStatisticsDesc implements org.apache.thrift.TBase<ColumnStatisticsDesc, ColumnStatisticsDesc._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ColumnStatisticsDesc implements org.apache.thrift.TBase<ColumnStatisticsDesc, ColumnStatisticsDesc._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnStatisticsDesc> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnStatisticsDesc");
  
    private static final org.apache.thrift.protocol.TField IS_TBL_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("isTblLevel", org.apache.thrift.protocol.TType.BOOL, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java
index f1257fd,3c2b123..2be715f
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ColumnStatisticsObj implements org.apache.thrift.TBase<ColumnStatisticsObj, ColumnStatisticsObj._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ColumnStatisticsObj implements org.apache.thrift.TBase<ColumnStatisticsObj, ColumnStatisticsObj._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnStatisticsObj> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnStatisticsObj");
  
    private static final org.apache.thrift.protocol.TField COL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("colName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
index aeb2421,5e8cd04..91483e2
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class CommitTxnRequest implements org.apache.thrift.TBase<CommitTxnRequest, CommitTxnRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class CommitTxnRequest implements org.apache.thrift.TBase<CommitTxnRequest, CommitTxnRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CommitTxnRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CommitTxnRequest");
  
    private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)1);


[24/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 0b80390..7fcdd7e 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -9750,6 +9750,726 @@ class FireEventResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class MetadataPpdResult:
+  """
+  Attributes:
+   - metadata
+   - includeBitset
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'metadata', None, None, ), # 1
+    (2, TType.STRING, 'includeBitset', None, None, ), # 2
+  )
+
+  def __init__(self, metadata=None, includeBitset=None,):
+    self.metadata = metadata
+    self.includeBitset = includeBitset
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.metadata = iprot.readString();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.includeBitset = iprot.readString();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('MetadataPpdResult')
+    if self.metadata is not None:
+      oprot.writeFieldBegin('metadata', TType.STRING, 1)
+      oprot.writeString(self.metadata)
+      oprot.writeFieldEnd()
+    if self.includeBitset is not None:
+      oprot.writeFieldBegin('includeBitset', TType.STRING, 2)
+      oprot.writeString(self.includeBitset)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.metadata is None:
+      raise TProtocol.TProtocolException(message='Required field metadata is unset!')
+    if self.includeBitset is None:
+      raise TProtocol.TProtocolException(message='Required field includeBitset is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.metadata)
+    value = (value * 31) ^ hash(self.includeBitset)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetFileMetadataByExprResult:
+  """
+  Attributes:
+   - metadata
+   - isSupported
+   - unknownFileIds
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'metadata', (TType.I64,None,TType.STRUCT,(MetadataPpdResult, MetadataPpdResult.thrift_spec)), None, ), # 1
+    (2, TType.BOOL, 'isSupported', None, None, ), # 2
+    (3, TType.LIST, 'unknownFileIds', (TType.I64,None), None, ), # 3
+  )
+
+  def __init__(self, metadata=None, isSupported=None, unknownFileIds=None,):
+    self.metadata = metadata
+    self.isSupported = isSupported
+    self.unknownFileIds = unknownFileIds
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.metadata = {}
+          (_ktype463, _vtype464, _size462 ) = iprot.readMapBegin()
+          for _i466 in xrange(_size462):
+            _key467 = iprot.readI64();
+            _val468 = MetadataPpdResult()
+            _val468.read(iprot)
+            self.metadata[_key467] = _val468
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.BOOL:
+          self.isSupported = iprot.readBool();
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.LIST:
+          self.unknownFileIds = []
+          (_etype472, _size469) = iprot.readListBegin()
+          for _i473 in xrange(_size469):
+            _elem474 = iprot.readI64();
+            self.unknownFileIds.append(_elem474)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetFileMetadataByExprResult')
+    if self.metadata is not None:
+      oprot.writeFieldBegin('metadata', TType.MAP, 1)
+      oprot.writeMapBegin(TType.I64, TType.STRUCT, len(self.metadata))
+      for kiter475,viter476 in self.metadata.items():
+        oprot.writeI64(kiter475)
+        viter476.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.isSupported is not None:
+      oprot.writeFieldBegin('isSupported', TType.BOOL, 2)
+      oprot.writeBool(self.isSupported)
+      oprot.writeFieldEnd()
+    if self.unknownFileIds is not None:
+      oprot.writeFieldBegin('unknownFileIds', TType.LIST, 3)
+      oprot.writeListBegin(TType.I64, len(self.unknownFileIds))
+      for iter477 in self.unknownFileIds:
+        oprot.writeI64(iter477)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.metadata is None:
+      raise TProtocol.TProtocolException(message='Required field metadata is unset!')
+    if self.isSupported is None:
+      raise TProtocol.TProtocolException(message='Required field isSupported is unset!')
+    if self.unknownFileIds is None:
+      raise TProtocol.TProtocolException(message='Required field unknownFileIds is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.metadata)
+    value = (value * 31) ^ hash(self.isSupported)
+    value = (value * 31) ^ hash(self.unknownFileIds)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetFileMetadataByExprRequest:
+  """
+  Attributes:
+   - fileIds
+   - expr
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'fileIds', (TType.I64,None), None, ), # 1
+    (2, TType.STRING, 'expr', None, None, ), # 2
+  )
+
+  def __init__(self, fileIds=None, expr=None,):
+    self.fileIds = fileIds
+    self.expr = expr
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.fileIds = []
+          (_etype481, _size478) = iprot.readListBegin()
+          for _i482 in xrange(_size478):
+            _elem483 = iprot.readI64();
+            self.fileIds.append(_elem483)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.expr = iprot.readString();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetFileMetadataByExprRequest')
+    if self.fileIds is not None:
+      oprot.writeFieldBegin('fileIds', TType.LIST, 1)
+      oprot.writeListBegin(TType.I64, len(self.fileIds))
+      for iter484 in self.fileIds:
+        oprot.writeI64(iter484)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.expr is not None:
+      oprot.writeFieldBegin('expr', TType.STRING, 2)
+      oprot.writeString(self.expr)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.fileIds is None:
+      raise TProtocol.TProtocolException(message='Required field fileIds is unset!')
+    if self.expr is None:
+      raise TProtocol.TProtocolException(message='Required field expr is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.fileIds)
+    value = (value * 31) ^ hash(self.expr)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetFileMetadataResult:
+  """
+  Attributes:
+   - metadata
+   - isSupported
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'metadata', (TType.I64,None,TType.STRING,None), None, ), # 1
+    (2, TType.BOOL, 'isSupported', None, None, ), # 2
+  )
+
+  def __init__(self, metadata=None, isSupported=None,):
+    self.metadata = metadata
+    self.isSupported = isSupported
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.metadata = {}
+          (_ktype486, _vtype487, _size485 ) = iprot.readMapBegin()
+          for _i489 in xrange(_size485):
+            _key490 = iprot.readI64();
+            _val491 = iprot.readString();
+            self.metadata[_key490] = _val491
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.BOOL:
+          self.isSupported = iprot.readBool();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetFileMetadataResult')
+    if self.metadata is not None:
+      oprot.writeFieldBegin('metadata', TType.MAP, 1)
+      oprot.writeMapBegin(TType.I64, TType.STRING, len(self.metadata))
+      for kiter492,viter493 in self.metadata.items():
+        oprot.writeI64(kiter492)
+        oprot.writeString(viter493)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.isSupported is not None:
+      oprot.writeFieldBegin('isSupported', TType.BOOL, 2)
+      oprot.writeBool(self.isSupported)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.metadata is None:
+      raise TProtocol.TProtocolException(message='Required field metadata is unset!')
+    if self.isSupported is None:
+      raise TProtocol.TProtocolException(message='Required field isSupported is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.metadata)
+    value = (value * 31) ^ hash(self.isSupported)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetFileMetadataRequest:
+  """
+  Attributes:
+   - fileIds
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'fileIds', (TType.I64,None), None, ), # 1
+  )
+
+  def __init__(self, fileIds=None,):
+    self.fileIds = fileIds
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.fileIds = []
+          (_etype497, _size494) = iprot.readListBegin()
+          for _i498 in xrange(_size494):
+            _elem499 = iprot.readI64();
+            self.fileIds.append(_elem499)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetFileMetadataRequest')
+    if self.fileIds is not None:
+      oprot.writeFieldBegin('fileIds', TType.LIST, 1)
+      oprot.writeListBegin(TType.I64, len(self.fileIds))
+      for iter500 in self.fileIds:
+        oprot.writeI64(iter500)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.fileIds is None:
+      raise TProtocol.TProtocolException(message='Required field fileIds is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.fileIds)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class PutFileMetadataResult:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('PutFileMetadataResult')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class PutFileMetadataRequest:
+  """
+  Attributes:
+   - fileIds
+   - metadata
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'fileIds', (TType.I64,None), None, ), # 1
+    (2, TType.LIST, 'metadata', (TType.STRING,None), None, ), # 2
+  )
+
+  def __init__(self, fileIds=None, metadata=None,):
+    self.fileIds = fileIds
+    self.metadata = metadata
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.fileIds = []
+          (_etype504, _size501) = iprot.readListBegin()
+          for _i505 in xrange(_size501):
+            _elem506 = iprot.readI64();
+            self.fileIds.append(_elem506)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.LIST:
+          self.metadata = []
+          (_etype510, _size507) = iprot.readListBegin()
+          for _i511 in xrange(_size507):
+            _elem512 = iprot.readString();
+            self.metadata.append(_elem512)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('PutFileMetadataRequest')
+    if self.fileIds is not None:
+      oprot.writeFieldBegin('fileIds', TType.LIST, 1)
+      oprot.writeListBegin(TType.I64, len(self.fileIds))
+      for iter513 in self.fileIds:
+        oprot.writeI64(iter513)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.metadata is not None:
+      oprot.writeFieldBegin('metadata', TType.LIST, 2)
+      oprot.writeListBegin(TType.STRING, len(self.metadata))
+      for iter514 in self.metadata:
+        oprot.writeString(iter514)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.fileIds is None:
+      raise TProtocol.TProtocolException(message='Required field fileIds is unset!')
+    if self.metadata is None:
+      raise TProtocol.TProtocolException(message='Required field metadata is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.fileIds)
+    value = (value * 31) ^ hash(self.metadata)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ClearFileMetadataResult:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ClearFileMetadataResult')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ClearFileMetadataRequest:
+  """
+  Attributes:
+   - fileIds
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'fileIds', (TType.I64,None), None, ), # 1
+  )
+
+  def __init__(self, fileIds=None,):
+    self.fileIds = fileIds
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.fileIds = []
+          (_etype518, _size515) = iprot.readListBegin()
+          for _i519 in xrange(_size515):
+            _elem520 = iprot.readI64();
+            self.fileIds.append(_elem520)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ClearFileMetadataRequest')
+    if self.fileIds is not None:
+      oprot.writeFieldBegin('fileIds', TType.LIST, 1)
+      oprot.writeListBegin(TType.I64, len(self.fileIds))
+      for iter521 in self.fileIds:
+        oprot.writeI64(iter521)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.fileIds is None:
+      raise TProtocol.TProtocolException(message='Required field fileIds is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.fileIds)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class GetAllFunctionsResponse:
   """
   Attributes:
@@ -9776,11 +10496,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype465, _size462) = iprot.readListBegin()
-          for _i466 in xrange(_size462):
-            _elem467 = Function()
-            _elem467.read(iprot)
-            self.functions.append(_elem467)
+          (_etype525, _size522) = iprot.readListBegin()
+          for _i526 in xrange(_size522):
+            _elem527 = Function()
+            _elem527.read(iprot)
+            self.functions.append(_elem527)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9797,8 +10517,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter468 in self.functions:
-        iter468.write(oprot)
+      for iter528 in self.functions:
+        iter528.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 4bd4302..771de51 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2231,6 +2231,173 @@ class FireEventResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class MetadataPpdResult
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  METADATA = 1
+  INCLUDEBITSET = 2
+
+  FIELDS = {
+    METADATA => {:type => ::Thrift::Types::STRING, :name => 'metadata', :binary => true},
+    INCLUDEBITSET => {:type => ::Thrift::Types::STRING, :name => 'includeBitset', :binary => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field metadata is unset!') unless @metadata
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field includeBitset is unset!') unless @includeBitset
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetFileMetadataByExprResult
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  METADATA = 1
+  ISSUPPORTED = 2
+  UNKNOWNFILEIDS = 3
+
+  FIELDS = {
+    METADATA => {:type => ::Thrift::Types::MAP, :name => 'metadata', :key => {:type => ::Thrift::Types::I64}, :value => {:type => ::Thrift::Types::STRUCT, :class => ::MetadataPpdResult}},
+    ISSUPPORTED => {:type => ::Thrift::Types::BOOL, :name => 'isSupported'},
+    UNKNOWNFILEIDS => {:type => ::Thrift::Types::LIST, :name => 'unknownFileIds', :element => {:type => ::Thrift::Types::I64}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field metadata is unset!') unless @metadata
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field isSupported is unset!') if @isSupported.nil?
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field unknownFileIds is unset!') unless @unknownFileIds
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetFileMetadataByExprRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  FILEIDS = 1
+  EXPR = 2
+
+  FIELDS = {
+    FILEIDS => {:type => ::Thrift::Types::LIST, :name => 'fileIds', :element => {:type => ::Thrift::Types::I64}},
+    EXPR => {:type => ::Thrift::Types::STRING, :name => 'expr', :binary => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field fileIds is unset!') unless @fileIds
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field expr is unset!') unless @expr
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetFileMetadataResult
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  METADATA = 1
+  ISSUPPORTED = 2
+
+  FIELDS = {
+    METADATA => {:type => ::Thrift::Types::MAP, :name => 'metadata', :key => {:type => ::Thrift::Types::I64}, :value => {:type => ::Thrift::Types::STRING, :binary => true}},
+    ISSUPPORTED => {:type => ::Thrift::Types::BOOL, :name => 'isSupported'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field metadata is unset!') unless @metadata
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field isSupported is unset!') if @isSupported.nil?
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetFileMetadataRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  FILEIDS = 1
+
+  FIELDS = {
+    FILEIDS => {:type => ::Thrift::Types::LIST, :name => 'fileIds', :element => {:type => ::Thrift::Types::I64}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field fileIds is unset!') unless @fileIds
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class PutFileMetadataResult
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+
+  FIELDS = {
+
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class PutFileMetadataRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  FILEIDS = 1
+  METADATA = 2
+
+  FIELDS = {
+    FILEIDS => {:type => ::Thrift::Types::LIST, :name => 'fileIds', :element => {:type => ::Thrift::Types::I64}},
+    METADATA => {:type => ::Thrift::Types::LIST, :name => 'metadata', :element => {:type => ::Thrift::Types::STRING, :binary => true}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field fileIds is unset!') unless @fileIds
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field metadata is unset!') unless @metadata
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class ClearFileMetadataResult
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+
+  FIELDS = {
+
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class ClearFileMetadataRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  FILEIDS = 1
+
+  FIELDS = {
+    FILEIDS => {:type => ::Thrift::Types::LIST, :name => 'fileIds', :element => {:type => ::Thrift::Types::I64}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field fileIds is unset!') unless @fileIds
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class GetAllFunctionsResponse
   include ::Thrift::Struct, ::Thrift::Struct_Union
   FUNCTIONS = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 90eb902..8625c7b 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2104,6 +2104,66 @@ module ThriftHiveMetastore
       return
     end
 
+    def get_file_metadata_by_expr(req)
+      send_get_file_metadata_by_expr(req)
+      return recv_get_file_metadata_by_expr()
+    end
+
+    def send_get_file_metadata_by_expr(req)
+      send_message('get_file_metadata_by_expr', Get_file_metadata_by_expr_args, :req => req)
+    end
+
+    def recv_get_file_metadata_by_expr()
+      result = receive_message(Get_file_metadata_by_expr_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_file_metadata_by_expr failed: unknown result')
+    end
+
+    def get_file_metadata(req)
+      send_get_file_metadata(req)
+      return recv_get_file_metadata()
+    end
+
+    def send_get_file_metadata(req)
+      send_message('get_file_metadata', Get_file_metadata_args, :req => req)
+    end
+
+    def recv_get_file_metadata()
+      result = receive_message(Get_file_metadata_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_file_metadata failed: unknown result')
+    end
+
+    def put_file_metadata(req)
+      send_put_file_metadata(req)
+      return recv_put_file_metadata()
+    end
+
+    def send_put_file_metadata(req)
+      send_message('put_file_metadata', Put_file_metadata_args, :req => req)
+    end
+
+    def recv_put_file_metadata()
+      result = receive_message(Put_file_metadata_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'put_file_metadata failed: unknown result')
+    end
+
+    def clear_file_metadata(req)
+      send_clear_file_metadata(req)
+      return recv_clear_file_metadata()
+    end
+
+    def send_clear_file_metadata(req)
+      send_message('clear_file_metadata', Clear_file_metadata_args, :req => req)
+    end
+
+    def recv_clear_file_metadata()
+      result = receive_message(Clear_file_metadata_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'clear_file_metadata failed: unknown result')
+    end
+
   end
 
   class Processor < ::FacebookService::Processor 
@@ -3702,6 +3762,34 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'flushCache', seqid)
     end
 
+    def process_get_file_metadata_by_expr(seqid, iprot, oprot)
+      args = read_args(iprot, Get_file_metadata_by_expr_args)
+      result = Get_file_metadata_by_expr_result.new()
+      result.success = @handler.get_file_metadata_by_expr(args.req)
+      write_result(result, oprot, 'get_file_metadata_by_expr', seqid)
+    end
+
+    def process_get_file_metadata(seqid, iprot, oprot)
+      args = read_args(iprot, Get_file_metadata_args)
+      result = Get_file_metadata_result.new()
+      result.success = @handler.get_file_metadata(args.req)
+      write_result(result, oprot, 'get_file_metadata', seqid)
+    end
+
+    def process_put_file_metadata(seqid, iprot, oprot)
+      args = read_args(iprot, Put_file_metadata_args)
+      result = Put_file_metadata_result.new()
+      result.success = @handler.put_file_metadata(args.req)
+      write_result(result, oprot, 'put_file_metadata', seqid)
+    end
+
+    def process_clear_file_metadata(seqid, iprot, oprot)
+      args = read_args(iprot, Clear_file_metadata_args)
+      result = Clear_file_metadata_result.new()
+      result.success = @handler.clear_file_metadata(args.req)
+      write_result(result, oprot, 'clear_file_metadata', seqid)
+    end
+
   end
 
   # HELPER FUNCTIONS AND STRUCTURES
@@ -8466,5 +8554,133 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_file_metadata_by_expr_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::GetFileMetadataByExprRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_file_metadata_by_expr_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetFileMetadataByExprResult}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_file_metadata_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::GetFileMetadataRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_file_metadata_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetFileMetadataResult}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Put_file_metadata_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::PutFileMetadataRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Put_file_metadata_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::PutFileMetadataResult}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Clear_file_metadata_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::ClearFileMetadataRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Clear_file_metadata_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::ClearFileMetadataResult}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
 end
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index da3d278..a06efc6 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -52,6 +52,8 @@ import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
+import org.apache.hadoop.hive.metastore.api.ClearFileMetadataRequest;
+import org.apache.hadoop.hive.metastore.api.ClearFileMetadataResult;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
@@ -69,6 +71,10 @@ import org.apache.hadoop.hive.metastore.api.FireEventRequest;
 import org.apache.hadoop.hive.metastore.api.FireEventResponse;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
+import org.apache.hadoop.hive.metastore.api.GetFileMetadataByExprRequest;
+import org.apache.hadoop.hive.metastore.api.GetFileMetadataByExprResult;
+import org.apache.hadoop.hive.metastore.api.GetFileMetadataRequest;
+import org.apache.hadoop.hive.metastore.api.GetFileMetadataResult;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
@@ -114,6 +120,8 @@ import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
+import org.apache.hadoop.hive.metastore.api.PutFileMetadataRequest;
+import org.apache.hadoop.hive.metastore.api.PutFileMetadataResult;
 import org.apache.hadoop.hive.metastore.api.RequestPartsSpec;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
@@ -167,14 +175,6 @@ import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
-import org.apache.hadoop.hive.metastore.model.MDBPrivilege;
-import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege;
-import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege;
-import org.apache.hadoop.hive.metastore.model.MPartitionPrivilege;
-import org.apache.hadoop.hive.metastore.model.MRole;
-import org.apache.hadoop.hive.metastore.model.MRoleMap;
-import org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege;
-import org.apache.hadoop.hive.metastore.model.MTablePrivilege;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.txn.TxnHandler;
 import org.apache.hadoop.hive.serde2.Deserializer;
@@ -208,6 +208,7 @@ import org.apache.thrift.transport.TTransportFactory;
 import javax.jdo.JDOException;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.AbstractMap;
@@ -292,8 +293,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
   }
 
-  public static class HMSHandler extends FacebookBase implements
-      IHMSHandler {
+  public static class HMSHandler extends FacebookBase implements IHMSHandler {
     public static final Log LOG = HiveMetaStore.LOG;
     private String rawStoreClassName;
     private final HiveConf hiveConf; // stores datastore (jpox) properties,
@@ -5700,6 +5700,40 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
 
     }
+
+    @Override
+    public GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req)
+        throws TException {
+      throw new UnsupportedOperationException("Not implemented yet");
+    }
+
+    @Override
+    public GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req) throws TException {
+      List<Long> fileIds = req.getFileIds();
+      ByteBuffer[] metadatas = getMS().getFileMetadata(fileIds);
+      GetFileMetadataResult result = new GetFileMetadataResult();
+      result.setIsSupported(metadatas != null);
+      if (metadatas != null) {
+        assert metadatas.length == fileIds.size();
+        for (int i = 0; i < metadatas.length; ++i) {
+          result.putToMetadata(fileIds.get(i), metadatas[i]);
+        }
+      }
+      return result;
+    }
+
+    @Override
+    public PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req) throws TException {
+      getMS().putFileMetadata(req.getFileIds(), req.getMetadata());
+      return new PutFileMetadataResult();
+    }
+
+    @Override
+    public ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req)
+        throws TException {
+      getMS().putFileMetadata(req.getFileIds(), null);
+      return new ClearFileMetadataResult();
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 59378db..1b2700a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -23,6 +23,7 @@ import static org.apache.commons.lang.StringUtils.join;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.URI;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -7615,4 +7616,14 @@ public class ObjectStore implements RawStore, Configurable {
     event.setMessage((dbEvent.getMessage()));
     return event;
   }
+
+  @Override
+  public ByteBuffer[] getFileMetadata(List<Long> fileIds) {
+    return null; // Not supported for now; callers have to handle this accordingly.
+  }
+
+  @Override
+  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) {
+    // Not supported for now.
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
index 9db1907..1968256 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -22,6 +22,7 @@ import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
+import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Map;
 
@@ -592,4 +593,8 @@ public interface RawStore extends Configurable {
    * flush statistics objects.  This should be called at the beginning of each query.
    */
   public void flushCache();
+
+  ByteBuffer[] getFileMetadata(List<Long> fileIds) throws MetaException;
+
+  void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws MetaException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index 66c46a5..f1336dc 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -27,17 +27,20 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.RegexStringComparator;
 import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
@@ -58,6 +61,7 @@ import org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator;
 import org.apache.hive.common.util.BloomFilter;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
@@ -88,6 +92,7 @@ public class HBaseReadWrite {
   @VisibleForTesting final static String SEQUENCES_TABLE = "HBMS_SEQUENCES";
   @VisibleForTesting final static String TABLE_TABLE = "HBMS_TBLS";
   @VisibleForTesting final static String USER_TO_ROLE_TABLE = "HBMS_USER_TO_ROLE";
+  @VisibleForTesting final static String FILE_METADATA_TABLE = "HBMS_FILE_METADATA";
   @VisibleForTesting final static byte[] CATALOG_CF = "c".getBytes(HBaseUtils.ENCODING);
   @VisibleForTesting final static byte[] STATS_CF = "s".getBytes(HBaseUtils.ENCODING);
   @VisibleForTesting final static String NO_CACHE_CONF = "no.use.cache";
@@ -96,7 +101,8 @@ public class HBaseReadWrite {
    */
   public final static String[] tableNames = { AGGR_STATS_TABLE, DB_TABLE, FUNC_TABLE, GLOBAL_PRIVS_TABLE,
                                        PART_TABLE, USER_TO_ROLE_TABLE, ROLE_TABLE, SD_TABLE,
-                                       SECURITY_TABLE, SEQUENCES_TABLE, TABLE_TABLE};
+                                       SECURITY_TABLE, SEQUENCES_TABLE, TABLE_TABLE,
+                                       FILE_METADATA_TABLE };
   public final static Map<String, List<byte[]>> columnFamilies =
       new HashMap<String, List<byte[]>> (tableNames.length);
 
@@ -112,6 +118,8 @@ public class HBaseReadWrite {
     columnFamilies.put(SECURITY_TABLE, Arrays.asList(CATALOG_CF));
     columnFamilies.put(SEQUENCES_TABLE, Arrays.asList(CATALOG_CF));
     columnFamilies.put(TABLE_TABLE, Arrays.asList(CATALOG_CF, STATS_CF));
+    // Stats CF will contain PPD stats.
+    columnFamilies.put(FILE_METADATA_TABLE, Arrays.asList(CATALOG_CF, STATS_CF));
   }
 
   /**
@@ -1714,6 +1722,37 @@ public class HBaseReadWrite {
   }
 
   /**********************************************************************************************
+   * File metadata related methods
+   *********************************************************************************************/
+
+  /**
+   * @param fileIds file ID list.
+   * @return Serialized file metadata.
+   */
+  ByteBuffer[] getFileMetadata(List<Long> fileIds) throws IOException {
+    byte[][] keys = new byte[fileIds.size()][];
+    for (int i = 0; i < fileIds.size(); ++i) {
+      keys[i] = HBaseUtils.makeLongKey(fileIds.get(i));
+    }
+    ByteBuffer[] result = new ByteBuffer[keys.length];
+    multiRead(FILE_METADATA_TABLE, CATALOG_CF, CATALOG_COL, keys, result);
+    return result;
+  }
+
+  /**
+   * @param fileIds file ID list.
+   * @param metadata Serialized file metadata.
+   */
+  void storeFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata)
+      throws IOException, InterruptedException {
+    byte[][] keys = new byte[fileIds.size()][];
+    for (int i = 0; i < fileIds.size(); ++i) {
+      keys[i] = HBaseUtils.makeLongKey(fileIds.get(i));
+    }
+    multiModify(FILE_METADATA_TABLE, keys, CATALOG_CF, CATALOG_COL, metadata);
+  }
+
+  /**********************************************************************************************
    * Security related methods
    *********************************************************************************************/
 
@@ -1899,6 +1938,49 @@ public class HBaseReadWrite {
     return res.getValue(colFam, colName);
   }
 
+  private void multiRead(String table, byte[] colFam, byte[] colName,
+      byte[][] keys, ByteBuffer[] resultDest) throws IOException {
+    assert keys.length == resultDest.length;
+    @SuppressWarnings("deprecation")
+    HTableInterface htab = conn.getHBaseTable(table);
+    List<Get> gets = new ArrayList<>(keys.length);
+    for (byte[] key : keys) {
+      Get g = new Get(key);
+      g.addColumn(colFam, colName);
+      gets.add(g);
+    }
+    Result[] results = htab.get(gets);
+    for (int i = 0; i < results.length; ++i) {
+      Result r = results[i];
+      resultDest[i] = (r.isEmpty() ? null : r.getValueAsByteBuffer(colFam, colName));
+    }
+  }
+
+  private void multiModify(String table, byte[][] keys, byte[] colFam,
+      byte[] colName, List<ByteBuffer> values) throws IOException, InterruptedException {
+    assert values == null || keys.length == values.size();
+    // HBase APIs are weird. To supply bytebuffer value, you have to also have bytebuffer
+    // column name, but not column family. So there. Perhaps we should add these to constants too.
+    ByteBuffer colNameBuf = ByteBuffer.wrap(colName);
+    @SuppressWarnings("deprecation")
+    HTableInterface htab = conn.getHBaseTable(table);
+    List<Row> actions = new ArrayList<>(keys.length);
+    for (int i = 0; i < keys.length; ++i) {
+      ByteBuffer value = (values != null) ? values.get(i) : null;
+      if (value == null) {
+        actions.add(new Delete(keys[i]));
+      } else {
+        Put p = new Put(keys[i]);
+        p.addColumn(colFam, colNameBuf, HConstants.LATEST_TIMESTAMP, value);
+        actions.add(p);
+      }
+    }
+    Object[] results = new Object[keys.length];
+    htab.batch(actions, results);
+    // TODO: should we check results array? we don't care about partial results
+    conn.flush(htab);
+  }
+
   private Result read(String table, byte[] key, byte[] colFam, byte[][] colNames)
       throws IOException {
     HTableInterface htab = conn.getHBaseTable(table);

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index 717e094..f30fcab 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.thrift.TException;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -2239,4 +2240,34 @@ public class HBaseStore implements RawStore {
   @VisibleForTesting HBaseReadWrite backdoor() {
     return getHBase();
   }
+
+  @Override
+  public ByteBuffer[] getFileMetadata(List<Long> fileIds) throws MetaException {
+    openTransaction();
+    boolean commit = true;
+    try {
+      return getHBase().getFileMetadata(fileIds);
+    } catch (IOException e) {
+      commit = false;
+      LOG.error("Unable to get file metadata", e);
+      throw new MetaException("Error reading file metadata " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
+  }
+
+  @Override
+  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws MetaException {
+    openTransaction();
+    boolean commit = false;
+    try {
+      getHBase().storeFileMetadata(fileIds, metadata);
+      commit = true;
+    } catch (IOException | InterruptedException e) {
+      LOG.error("Unable to store file metadata", e);
+      throw new MetaException("Error storing file metadata " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index b6fa591..841afd4 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -1302,4 +1302,17 @@ class HBaseUtils {
     keyEnd[keyEnd.length - 1]++;
     return keyEnd;
   }
+
+  static byte[] makeLongKey(long v) {
+    byte[] b = new byte[8];
+    b[0] = (byte)(v >>> 56);
+    b[1] = (byte)(v >>> 48);
+    b[2] = (byte)(v >>> 40);
+    b[3] = (byte)(v >>> 32);
+    b[4] = (byte)(v >>> 24);
+    b[5] = (byte)(v >>> 16);
+    b[6] = (byte)(v >>>  8);
+    b[7] = (byte)(v >>>  0);
+    return b;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 6efadba..0f3331a 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -759,4 +760,13 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   public void flushCache() {
     objectStore.flushCache();
   }
+
+  @Override
+  public ByteBuffer[] getFileMetadata(List<Long> fileIds) {
+    return null;
+  }
+
+  @Override
+  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 00fca8c..126a2c2 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -777,6 +778,14 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   }
 
+  @Override
+  public ByteBuffer[] getFileMetadata(List<Long> fileIds) {
+    return null;
+  }
+
+  @Override
+  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) {
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java
----------------------------------------------------------------------
diff --git a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java
index b26ab96..2153f0e 100644
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Adjacency.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Adjacency implements org.apache.thrift.TBase<Adjacency, Adjacency._Fields>, java.io.Serializable, Cloneable, Comparable<Adjacency> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Adjacency");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java
----------------------------------------------------------------------
diff --git a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java
index 0a13175..f864c18 100644
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Graph.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Graph implements org.apache.thrift.TBase<Graph, Graph._Fields>, java.io.Serializable, Cloneable, Comparable<Graph> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Graph");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java
----------------------------------------------------------------------
diff --git a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java
index 991974c..a7ec4e4 100644
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Operator implements org.apache.thrift.TBase<Operator, Operator._Fields>, java.io.Serializable, Cloneable, Comparable<Operator> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Operator");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java
----------------------------------------------------------------------
diff --git a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java
index f98a7e1..2f64123 100644
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Query implements org.apache.thrift.TBase<Query, Query._Fields>, java.io.Serializable, Cloneable, Comparable<Query> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Query");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java
----------------------------------------------------------------------
diff --git a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java
index 0994fda..5ccceb1 100644
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/QueryPlan.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class QueryPlan implements org.apache.thrift.TBase<QueryPlan, QueryPlan._Fields>, java.io.Serializable, Cloneable, Comparable<QueryPlan> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("QueryPlan");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java
----------------------------------------------------------------------
diff --git a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java
index e0cd86c..706e335 100644
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Stage implements org.apache.thrift.TBase<Stage, Stage._Fields>, java.io.Serializable, Cloneable, Comparable<Stage> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Stage");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java
----------------------------------------------------------------------
diff --git a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java
index 66e5e30..2d55d7a 100644
--- a/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java
+++ b/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Task implements org.apache.thrift.TBase<Task, Task._Fields>, java.io.Serializable, Cloneable, Comparable<Task> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Task");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java
index 68bb885..eed53fa 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/InnerStruct.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class InnerStruct implements org.apache.thrift.TBase<InnerStruct, InnerStruct._Fields>, java.io.Serializable, Cloneable, Comparable<InnerStruct> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InnerStruct");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java
index 208fa82..4410307 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ThriftTestObj implements org.apache.thrift.TBase<ThriftTestObj, ThriftTestObj._Fields>, java.io.Serializable, Cloneable, Comparable<ThriftTestObj> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ThriftTestObj");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java
index 6d32947..59a1f7e 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Complex implements org.apache.thrift.TBase<Complex, Complex._Fields>, java.io.Serializable, Cloneable, Comparable<Complex> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Complex");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java
index 26e7b38..901fc4b 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/IntString.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class IntString implements org.apache.thrift.TBase<IntString, IntString._Fields>, java.io.Serializable, Cloneable, Comparable<IntString> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IntString");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java
index d937a9c..cc3f375 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class MegaStruct implements org.apache.thrift.TBase<MegaStruct, MegaStruct._Fields>, java.io.Serializable, Cloneable, Comparable<MegaStruct> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MegaStruct");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java
index c25156a..e7498f4 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MiniStruct.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class MiniStruct implements org.apache.thrift.TBase<MiniStruct, MiniStruct._Fields>, java.io.Serializable, Cloneable, Comparable<MiniStruct> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MiniStruct");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java
index d1bd61d..a2cbda2 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/SetIntString.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class SetIntString implements org.apache.thrift.TBase<SetIntString, SetIntString._Fields>, java.io.Serializable, Cloneable, Comparable<SetIntString> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SetIntString");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java
index d4b6972..7396d02 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class HiveClusterStatus implements org.apache.thrift.TBase<HiveClusterStatus, HiveClusterStatus._Fields>, java.io.Serializable, Cloneable, Comparable<HiveClusterStatus> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HiveClusterStatus");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java
index 760c81e..e15a9e0 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class HiveServerException extends TException implements org.apache.thrift.TBase<HiveServerException, HiveServerException._Fields>, java.io.Serializable, Cloneable, Comparable<HiveServerException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HiveServerException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java
index df793b1..2a7fd9b 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ThriftHive {
 
   public interface Iface extends org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface {

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java
index 5625516..841139b 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TArrayTypeEntry implements org.apache.thrift.TBase<TArrayTypeEntry, TArrayTypeEntry._Fields>, java.io.Serializable, Cloneable, Comparable<TArrayTypeEntry> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TArrayTypeEntry");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBinaryColumn.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBinaryColumn.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBinaryColumn.java
index 202399a..bfea569 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBinaryColumn.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBinaryColumn.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TBinaryColumn implements org.apache.thrift.TBase<TBinaryColumn, TBinaryColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TBinaryColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBinaryColumn");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolColumn.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolColumn.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolColumn.java
index 921e9de..5c10fde 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolColumn.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolColumn.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TBoolColumn implements org.apache.thrift.TBase<TBoolColumn, TBoolColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TBoolColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolColumn");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolValue.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolValue.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolValue.java
index 201c9fb..86b5ce3 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolValue.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolValue.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TBoolValue implements org.apache.thrift.TBase<TBoolValue, TBoolValue._Fields>, java.io.Serializable, Cloneable, Comparable<TBoolValue> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolValue");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteColumn.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteColumn.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteColumn.java
index cd9b6da..3d42927 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteColumn.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteColumn.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TByteColumn implements org.apache.thrift.TBase<TByteColumn, TByteColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TByteColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteColumn");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteValue.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteValue.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteValue.java
index 42b5bd5..04f8e7c 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteValue.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteValue.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TByteValue implements org.apache.thrift.TBase<TByteValue, TByteValue._Fields>, java.io.Serializable, Cloneable, Comparable<TByteValue> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteValue");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIService.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIService.java
index 6bdd53d..2630215 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIService.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIService.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCLIService {
 
   public interface Iface {

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java
index 1097869..cdabe7d 100644
--- a/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TCancelDelegationTokenReq implements org.apache.thrift.TBase<TCancelDelegationTokenReq, TCancelDelegationTokenReq._Fields>, java.io.Serializable, Cloneable, Comparable<TCancelDelegationTokenReq> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenReq");
 


[26/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index 949a6e9..0baeef3 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -13861,6 +13861,995 @@ class FireEventResponse {
 
 }
 
+class MetadataPpdResult {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $metadata = null;
+  /**
+   * @var string
+   */
+  public $includeBitset = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'metadata',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'includeBitset',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['metadata'])) {
+        $this->metadata = $vals['metadata'];
+      }
+      if (isset($vals['includeBitset'])) {
+        $this->includeBitset = $vals['includeBitset'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'MetadataPpdResult';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->metadata);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->includeBitset);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('MetadataPpdResult');
+    if ($this->metadata !== null) {
+      $xfer += $output->writeFieldBegin('metadata', TType::STRING, 1);
+      $xfer += $output->writeString($this->metadata);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->includeBitset !== null) {
+      $xfer += $output->writeFieldBegin('includeBitset', TType::STRING, 2);
+      $xfer += $output->writeString($this->includeBitset);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetFileMetadataByExprResult {
+  static $_TSPEC;
+
+  /**
+   * @var array
+   */
+  public $metadata = null;
+  /**
+   * @var bool
+   */
+  public $isSupported = null;
+  /**
+   * @var int[]
+   */
+  public $unknownFileIds = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'metadata',
+          'type' => TType::MAP,
+          'ktype' => TType::I64,
+          'vtype' => TType::STRUCT,
+          'key' => array(
+            'type' => TType::I64,
+          ),
+          'val' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\MetadataPpdResult',
+            ),
+          ),
+        2 => array(
+          'var' => 'isSupported',
+          'type' => TType::BOOL,
+          ),
+        3 => array(
+          'var' => 'unknownFileIds',
+          'type' => TType::LST,
+          'etype' => TType::I64,
+          'elem' => array(
+            'type' => TType::I64,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['metadata'])) {
+        $this->metadata = $vals['metadata'];
+      }
+      if (isset($vals['isSupported'])) {
+        $this->isSupported = $vals['isSupported'];
+      }
+      if (isset($vals['unknownFileIds'])) {
+        $this->unknownFileIds = $vals['unknownFileIds'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetFileMetadataByExprResult';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::MAP) {
+            $this->metadata = array();
+            $_size465 = 0;
+            $_ktype466 = 0;
+            $_vtype467 = 0;
+            $xfer += $input->readMapBegin($_ktype466, $_vtype467, $_size465);
+            for ($_i469 = 0; $_i469 < $_size465; ++$_i469)
+            {
+              $key470 = 0;
+              $val471 = new \metastore\MetadataPpdResult();
+              $xfer += $input->readI64($key470);
+              $val471 = new \metastore\MetadataPpdResult();
+              $xfer += $val471->read($input);
+              $this->metadata[$key470] = $val471;
+            }
+            $xfer += $input->readMapEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->isSupported);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::LST) {
+            $this->unknownFileIds = array();
+            $_size472 = 0;
+            $_etype475 = 0;
+            $xfer += $input->readListBegin($_etype475, $_size472);
+            for ($_i476 = 0; $_i476 < $_size472; ++$_i476)
+            {
+              $elem477 = null;
+              $xfer += $input->readI64($elem477);
+              $this->unknownFileIds []= $elem477;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetFileMetadataByExprResult');
+    if ($this->metadata !== null) {
+      if (!is_array($this->metadata)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('metadata', TType::MAP, 1);
+      {
+        $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata));
+        {
+          foreach ($this->metadata as $kiter478 => $viter479)
+          {
+            $xfer += $output->writeI64($kiter478);
+            $xfer += $viter479->write($output);
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->isSupported !== null) {
+      $xfer += $output->writeFieldBegin('isSupported', TType::BOOL, 2);
+      $xfer += $output->writeBool($this->isSupported);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->unknownFileIds !== null) {
+      if (!is_array($this->unknownFileIds)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('unknownFileIds', TType::LST, 3);
+      {
+        $output->writeListBegin(TType::I64, count($this->unknownFileIds));
+        {
+          foreach ($this->unknownFileIds as $iter480)
+          {
+            $xfer += $output->writeI64($iter480);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetFileMetadataByExprRequest {
+  static $_TSPEC;
+
+  /**
+   * @var int[]
+   */
+  public $fileIds = null;
+  /**
+   * @var string
+   */
+  public $expr = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'fileIds',
+          'type' => TType::LST,
+          'etype' => TType::I64,
+          'elem' => array(
+            'type' => TType::I64,
+            ),
+          ),
+        2 => array(
+          'var' => 'expr',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['fileIds'])) {
+        $this->fileIds = $vals['fileIds'];
+      }
+      if (isset($vals['expr'])) {
+        $this->expr = $vals['expr'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetFileMetadataByExprRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::LST) {
+            $this->fileIds = array();
+            $_size481 = 0;
+            $_etype484 = 0;
+            $xfer += $input->readListBegin($_etype484, $_size481);
+            for ($_i485 = 0; $_i485 < $_size481; ++$_i485)
+            {
+              $elem486 = null;
+              $xfer += $input->readI64($elem486);
+              $this->fileIds []= $elem486;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->expr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetFileMetadataByExprRequest');
+    if ($this->fileIds !== null) {
+      if (!is_array($this->fileIds)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fileIds', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::I64, count($this->fileIds));
+        {
+          foreach ($this->fileIds as $iter487)
+          {
+            $xfer += $output->writeI64($iter487);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->expr !== null) {
+      $xfer += $output->writeFieldBegin('expr', TType::STRING, 2);
+      $xfer += $output->writeString($this->expr);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetFileMetadataResult {
+  static $_TSPEC;
+
+  /**
+   * @var array
+   */
+  public $metadata = null;
+  /**
+   * @var bool
+   */
+  public $isSupported = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'metadata',
+          'type' => TType::MAP,
+          'ktype' => TType::I64,
+          'vtype' => TType::STRING,
+          'key' => array(
+            'type' => TType::I64,
+          ),
+          'val' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        2 => array(
+          'var' => 'isSupported',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['metadata'])) {
+        $this->metadata = $vals['metadata'];
+      }
+      if (isset($vals['isSupported'])) {
+        $this->isSupported = $vals['isSupported'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetFileMetadataResult';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::MAP) {
+            $this->metadata = array();
+            $_size488 = 0;
+            $_ktype489 = 0;
+            $_vtype490 = 0;
+            $xfer += $input->readMapBegin($_ktype489, $_vtype490, $_size488);
+            for ($_i492 = 0; $_i492 < $_size488; ++$_i492)
+            {
+              $key493 = 0;
+              $val494 = '';
+              $xfer += $input->readI64($key493);
+              $xfer += $input->readString($val494);
+              $this->metadata[$key493] = $val494;
+            }
+            $xfer += $input->readMapEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->isSupported);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetFileMetadataResult');
+    if ($this->metadata !== null) {
+      if (!is_array($this->metadata)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('metadata', TType::MAP, 1);
+      {
+        $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata));
+        {
+          foreach ($this->metadata as $kiter495 => $viter496)
+          {
+            $xfer += $output->writeI64($kiter495);
+            $xfer += $output->writeString($viter496);
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->isSupported !== null) {
+      $xfer += $output->writeFieldBegin('isSupported', TType::BOOL, 2);
+      $xfer += $output->writeBool($this->isSupported);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetFileMetadataRequest {
+  static $_TSPEC;
+
+  /**
+   * @var int[]
+   */
+  public $fileIds = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'fileIds',
+          'type' => TType::LST,
+          'etype' => TType::I64,
+          'elem' => array(
+            'type' => TType::I64,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['fileIds'])) {
+        $this->fileIds = $vals['fileIds'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetFileMetadataRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::LST) {
+            $this->fileIds = array();
+            $_size497 = 0;
+            $_etype500 = 0;
+            $xfer += $input->readListBegin($_etype500, $_size497);
+            for ($_i501 = 0; $_i501 < $_size497; ++$_i501)
+            {
+              $elem502 = null;
+              $xfer += $input->readI64($elem502);
+              $this->fileIds []= $elem502;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetFileMetadataRequest');
+    if ($this->fileIds !== null) {
+      if (!is_array($this->fileIds)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fileIds', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::I64, count($this->fileIds));
+        {
+          foreach ($this->fileIds as $iter503)
+          {
+            $xfer += $output->writeI64($iter503);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class PutFileMetadataResult {
+  static $_TSPEC;
+
+
+  public function __construct() {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        );
+    }
+  }
+
+  public function getName() {
+    return 'PutFileMetadataResult';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('PutFileMetadataResult');
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class PutFileMetadataRequest {
+  static $_TSPEC;
+
+  /**
+   * @var int[]
+   */
+  public $fileIds = null;
+  /**
+   * @var string[]
+   */
+  public $metadata = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'fileIds',
+          'type' => TType::LST,
+          'etype' => TType::I64,
+          'elem' => array(
+            'type' => TType::I64,
+            ),
+          ),
+        2 => array(
+          'var' => 'metadata',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['fileIds'])) {
+        $this->fileIds = $vals['fileIds'];
+      }
+      if (isset($vals['metadata'])) {
+        $this->metadata = $vals['metadata'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'PutFileMetadataRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::LST) {
+            $this->fileIds = array();
+            $_size504 = 0;
+            $_etype507 = 0;
+            $xfer += $input->readListBegin($_etype507, $_size504);
+            for ($_i508 = 0; $_i508 < $_size504; ++$_i508)
+            {
+              $elem509 = null;
+              $xfer += $input->readI64($elem509);
+              $this->fileIds []= $elem509;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::LST) {
+            $this->metadata = array();
+            $_size510 = 0;
+            $_etype513 = 0;
+            $xfer += $input->readListBegin($_etype513, $_size510);
+            for ($_i514 = 0; $_i514 < $_size510; ++$_i514)
+            {
+              $elem515 = null;
+              $xfer += $input->readString($elem515);
+              $this->metadata []= $elem515;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('PutFileMetadataRequest');
+    if ($this->fileIds !== null) {
+      if (!is_array($this->fileIds)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fileIds', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::I64, count($this->fileIds));
+        {
+          foreach ($this->fileIds as $iter516)
+          {
+            $xfer += $output->writeI64($iter516);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->metadata !== null) {
+      if (!is_array($this->metadata)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('metadata', TType::LST, 2);
+      {
+        $output->writeListBegin(TType::STRING, count($this->metadata));
+        {
+          foreach ($this->metadata as $iter517)
+          {
+            $xfer += $output->writeString($iter517);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ClearFileMetadataResult {
+  static $_TSPEC;
+
+
+  public function __construct() {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        );
+    }
+  }
+
+  public function getName() {
+    return 'ClearFileMetadataResult';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ClearFileMetadataResult');
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ClearFileMetadataRequest {
+  static $_TSPEC;
+
+  /**
+   * @var int[]
+   */
+  public $fileIds = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'fileIds',
+          'type' => TType::LST,
+          'etype' => TType::I64,
+          'elem' => array(
+            'type' => TType::I64,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['fileIds'])) {
+        $this->fileIds = $vals['fileIds'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ClearFileMetadataRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::LST) {
+            $this->fileIds = array();
+            $_size518 = 0;
+            $_etype521 = 0;
+            $xfer += $input->readListBegin($_etype521, $_size518);
+            for ($_i522 = 0; $_i522 < $_size518; ++$_i522)
+            {
+              $elem523 = null;
+              $xfer += $input->readI64($elem523);
+              $this->fileIds []= $elem523;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ClearFileMetadataRequest');
+    if ($this->fileIds !== null) {
+      if (!is_array($this->fileIds)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fileIds', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::I64, count($this->fileIds));
+        {
+          foreach ($this->fileIds as $iter524)
+          {
+            $xfer += $output->writeI64($iter524);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class GetAllFunctionsResponse {
   static $_TSPEC;
 
@@ -13912,15 +14901,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size465 = 0;
-            $_etype468 = 0;
-            $xfer += $input->readListBegin($_etype468, $_size465);
-            for ($_i469 = 0; $_i469 < $_size465; ++$_i469)
+            $_size525 = 0;
+            $_etype528 = 0;
+            $xfer += $input->readListBegin($_etype528, $_size525);
+            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
             {
-              $elem470 = null;
-              $elem470 = new \metastore\Function();
-              $xfer += $elem470->read($input);
-              $this->functions []= $elem470;
+              $elem530 = null;
+              $elem530 = new \metastore\Function();
+              $xfer += $elem530->read($input);
+              $this->functions []= $elem530;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13948,9 +14937,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter471)
+          foreach ($this->functions as $iter531)
           {
-            $xfer += $iter471->write($output);
+            $xfer += $iter531->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 6bd2728..466063e 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -149,6 +149,10 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  CurrentNotificationEventId get_current_notificationEventId()')
   print('  FireEventResponse fire_listener_event(FireEventRequest rqst)')
   print('  void flushCache()')
+  print('  GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req)')
+  print('  GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req)')
+  print('  PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req)')
+  print('  ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req)')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -968,6 +972,30 @@ elif cmd == 'flushCache':
     sys.exit(1)
   pp.pprint(client.flushCache())
 
+elif cmd == 'get_file_metadata_by_expr':
+  if len(args) != 1:
+    print('get_file_metadata_by_expr requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_file_metadata_by_expr(eval(args[0]),))
+
+elif cmd == 'get_file_metadata':
+  if len(args) != 1:
+    print('get_file_metadata requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_file_metadata(eval(args[0]),))
+
+elif cmd == 'put_file_metadata':
+  if len(args) != 1:
+    print('put_file_metadata requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.put_file_metadata(eval(args[0]),))
+
+elif cmd == 'clear_file_metadata':
+  if len(args) != 1:
+    print('clear_file_metadata requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.clear_file_metadata(eval(args[0]),))
+
 elif cmd == 'getName':
   if len(args) != 0:
     print('getName requires 0 args')


[48/52] [abbrv] hive git commit: HIVE-11711: Merge hbase-metastore branch to trunk

Posted by se...@apache.org.
HIVE-11711: Merge hbase-metastore branch to trunk


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

Branch: refs/heads/llap
Commit: 52383033822b87b02853eaaf15db1b2904617615
Parents: 514ab79 4c17ecf
Author: Daniel Dai <da...@hortonworks.com>
Authored: Mon Sep 21 22:02:22 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Mon Sep 21 22:02:22 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ant/QTestGenTask.java    |    11 +
 bin/ext/hbaseimport.cmd                         |    35 +
 bin/ext/hbaseimport.sh                          |    27 +
 bin/ext/hbaseschematool.sh                      |    27 +
 .../apache/hadoop/hive/common/ObjectPair.java   |     5 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    98 +-
 .../apache/hive/common/util/BloomFilter.java    |    20 +-
 data/conf/tez/hive-site.xml                     |    10 +
 itests/hive-unit/pom.xml                        |    35 +
 .../hadoop/hive/metastore/TestAdminUser.java    |     4 +-
 .../hive/metastore/TestHiveMetaStore.java       |     3 +
 .../metastore/hbase/HBaseIntegrationTests.java  |   117 +
 .../TestHBaseAggrStatsCacheIntegration.java     |   691 +
 .../hive/metastore/hbase/TestHBaseImport.java   |   650 +
 .../metastore/hbase/TestHBaseMetastoreSql.java  |   223 +
 .../hbase/TestHBaseStoreIntegration.java        |  1794 +
 .../hbase/TestStorageDescriptorSharing.java     |   191 +
 itests/qtest/pom.xml                            |    10 +-
 itests/util/pom.xml                             |    32 +
 .../metastore/hbase/HBaseStoreTestUtil.java     |    45 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    41 +-
 metastore/if/hive_metastore.thrift              |    54 +
 metastore/pom.xml                               |    82 +
 .../metastore/hbase/HbaseMetastoreProto.java    | 34901 +++++++++++++++++
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  6919 ++--
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |   664 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    25 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  1294 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   371 +
 .../hive/metastore/api/AbortTxnRequest.java     |     2 +-
 .../metastore/api/AddDynamicPartitions.java     |     2 +-
 .../metastore/api/AddPartitionsRequest.java     |     2 +-
 .../hive/metastore/api/AddPartitionsResult.java |     2 +-
 .../hadoop/hive/metastore/api/AggrStats.java    |     2 +-
 .../metastore/api/AlreadyExistsException.java   |     2 +-
 .../metastore/api/BinaryColumnStatsData.java    |     2 +-
 .../metastore/api/BooleanColumnStatsData.java   |     2 +-
 .../hive/metastore/api/CheckLockRequest.java    |     2 +-
 .../metastore/api/ClearFileMetadataRequest.java |   438 +
 .../metastore/api/ClearFileMetadataResult.java  |   283 +
 .../hive/metastore/api/ColumnStatistics.java    |     2 +-
 .../metastore/api/ColumnStatisticsDesc.java     |     2 +-
 .../hive/metastore/api/ColumnStatisticsObj.java |     2 +-
 .../hive/metastore/api/CommitTxnRequest.java    |     2 +-
 .../hive/metastore/api/CompactionRequest.java   |     2 +-
 .../api/ConfigValSecurityException.java         |     2 +-
 .../api/CurrentNotificationEventId.java         |     2 +-
 .../hadoop/hive/metastore/api/Database.java     |     2 +-
 .../apache/hadoop/hive/metastore/api/Date.java  |     2 +-
 .../hive/metastore/api/DateColumnStatsData.java |     2 +-
 .../hadoop/hive/metastore/api/Decimal.java      |     2 +-
 .../metastore/api/DecimalColumnStatsData.java   |     2 +-
 .../metastore/api/DoubleColumnStatsData.java    |     2 +-
 .../hive/metastore/api/DropPartitionsExpr.java  |     2 +-
 .../metastore/api/DropPartitionsRequest.java    |     2 +-
 .../metastore/api/DropPartitionsResult.java     |     2 +-
 .../hive/metastore/api/EnvironmentContext.java  |     2 +-
 .../hadoop/hive/metastore/api/FieldSchema.java  |     2 +-
 .../hive/metastore/api/FireEventRequest.java    |     2 +-
 .../hive/metastore/api/FireEventResponse.java   |     2 +-
 .../hadoop/hive/metastore/api/Function.java     |     2 +-
 .../metastore/api/GetAllFunctionsResponse.java  |    38 +-
 .../api/GetFileMetadataByExprRequest.java       |   548 +
 .../api/GetFileMetadataByExprResult.java        |   703 +
 .../metastore/api/GetFileMetadataRequest.java   |   438 +
 .../metastore/api/GetFileMetadataResult.java    |   540 +
 .../metastore/api/GetOpenTxnsInfoResponse.java  |     2 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |     2 +-
 .../api/GetPrincipalsInRoleRequest.java         |     2 +-
 .../api/GetPrincipalsInRoleResponse.java        |     2 +-
 .../api/GetRoleGrantsForPrincipalRequest.java   |     2 +-
 .../api/GetRoleGrantsForPrincipalResponse.java  |     2 +-
 .../api/GrantRevokePrivilegeRequest.java        |     2 +-
 .../api/GrantRevokePrivilegeResponse.java       |     2 +-
 .../metastore/api/GrantRevokeRoleRequest.java   |     2 +-
 .../metastore/api/GrantRevokeRoleResponse.java  |     2 +-
 .../hive/metastore/api/HeartbeatRequest.java    |     2 +-
 .../metastore/api/HeartbeatTxnRangeRequest.java |     2 +-
 .../api/HeartbeatTxnRangeResponse.java          |     2 +-
 .../hive/metastore/api/HiveObjectPrivilege.java |     2 +-
 .../hive/metastore/api/HiveObjectRef.java       |     2 +-
 .../apache/hadoop/hive/metastore/api/Index.java |     2 +-
 .../api/IndexAlreadyExistsException.java        |     2 +-
 .../metastore/api/InsertEventRequestData.java   |     2 +-
 .../metastore/api/InvalidInputException.java    |     2 +-
 .../metastore/api/InvalidObjectException.java   |     2 +-
 .../api/InvalidOperationException.java          |     2 +-
 .../api/InvalidPartitionException.java          |     2 +-
 .../hive/metastore/api/LockComponent.java       |     2 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |     2 +-
 .../hadoop/hive/metastore/api/LockResponse.java |     2 +-
 .../hive/metastore/api/LongColumnStatsData.java |     2 +-
 .../hive/metastore/api/MetaException.java       |     2 +-
 .../hive/metastore/api/MetadataPpdResult.java   |   508 +
 .../hive/metastore/api/NoSuchLockException.java |     2 +-
 .../metastore/api/NoSuchObjectException.java    |     2 +-
 .../hive/metastore/api/NoSuchTxnException.java  |     2 +-
 .../hive/metastore/api/NotificationEvent.java   |     2 +-
 .../metastore/api/NotificationEventRequest.java |     2 +-
 .../api/NotificationEventResponse.java          |     2 +-
 .../hive/metastore/api/OpenTxnRequest.java      |     2 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Order.java |     2 +-
 .../hadoop/hive/metastore/api/Partition.java    |     2 +-
 .../api/PartitionListComposingSpec.java         |     2 +-
 .../hive/metastore/api/PartitionSpec.java       |     2 +-
 .../api/PartitionSpecWithSharedSD.java          |     2 +-
 .../hive/metastore/api/PartitionWithoutSD.java  |     2 +-
 .../metastore/api/PartitionsByExprRequest.java  |     2 +-
 .../metastore/api/PartitionsByExprResult.java   |     2 +-
 .../metastore/api/PartitionsStatsRequest.java   |     2 +-
 .../metastore/api/PartitionsStatsResult.java    |     2 +-
 .../metastore/api/PrincipalPrivilegeSet.java    |     2 +-
 .../hadoop/hive/metastore/api/PrivilegeBag.java |     2 +-
 .../hive/metastore/api/PrivilegeGrantInfo.java  |     2 +-
 .../metastore/api/PutFileMetadataRequest.java   |   588 +
 .../metastore/api/PutFileMetadataResult.java    |   283 +
 .../hadoop/hive/metastore/api/ResourceUri.java  |     2 +-
 .../apache/hadoop/hive/metastore/api/Role.java  |     2 +-
 .../hive/metastore/api/RolePrincipalGrant.java  |     2 +-
 .../hadoop/hive/metastore/api/Schema.java       |     2 +-
 .../hadoop/hive/metastore/api/SerDeInfo.java    |     2 +-
 .../api/SetPartitionsStatsRequest.java          |     2 +-
 .../hive/metastore/api/ShowCompactRequest.java  |     2 +-
 .../hive/metastore/api/ShowCompactResponse.java |     2 +-
 .../api/ShowCompactResponseElement.java         |     2 +-
 .../hive/metastore/api/ShowLocksRequest.java    |     2 +-
 .../hive/metastore/api/ShowLocksResponse.java   |     2 +-
 .../metastore/api/ShowLocksResponseElement.java |     2 +-
 .../hadoop/hive/metastore/api/SkewedInfo.java   |     2 +-
 .../hive/metastore/api/StorageDescriptor.java   |     2 +-
 .../metastore/api/StringColumnStatsData.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Table.java |     2 +-
 .../hive/metastore/api/TableStatsRequest.java   |     2 +-
 .../hive/metastore/api/TableStatsResult.java    |     2 +-
 .../hive/metastore/api/ThriftHiveMetastore.java |  8422 ++--
 .../hive/metastore/api/TxnAbortedException.java |     2 +-
 .../hadoop/hive/metastore/api/TxnInfo.java      |     2 +-
 .../hive/metastore/api/TxnOpenException.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Type.java  |     2 +-
 .../hive/metastore/api/UnknownDBException.java  |     2 +-
 .../api/UnknownPartitionException.java          |     2 +-
 .../metastore/api/UnknownTableException.java    |     2 +-
 .../hive/metastore/api/UnlockRequest.java       |     2 +-
 .../hadoop/hive/metastore/api/Version.java      |     2 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   |  2810 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |  1009 +-
 .../hive_metastore/ThriftHiveMetastore-remote   |    49 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  1563 +-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   734 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   167 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   267 +
 .../hadoop/hive/metastore/HiveAlterHandler.java |    38 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   266 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   112 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |     6 +
 .../hadoop/hive/metastore/ObjectStore.java      |   418 +-
 .../hive/metastore/PartFilterExprUtil.java      |   149 +
 .../apache/hadoop/hive/metastore/RawStore.java  |    66 +-
 .../hadoop/hive/metastore/RawStoreProxy.java    |     5 +-
 .../hbase/AggrStatsInvalidatorFilter.java       |   121 +
 .../hadoop/hive/metastore/hbase/Counter.java    |    53 +
 .../hive/metastore/hbase/HBaseConnection.java   |    96 +
 .../metastore/hbase/HBaseFilterPlanUtil.java    |   612 +
 .../hive/metastore/hbase/HBaseImport.java       |   535 +
 .../hive/metastore/hbase/HBaseReadWrite.java    |  2106 +
 .../hive/metastore/hbase/HBaseSchemaTool.java   |   239 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |  2387 ++
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |  1340 +
 .../hive/metastore/hbase/ObjectCache.java       |    81 +
 .../hive/metastore/hbase/PartitionCache.java    |   168 +
 .../metastore/hbase/PartitionKeyComparator.java |   292 +
 .../hbase/SharedStorageDescriptor.java          |   251 +
 .../hadoop/hive/metastore/hbase/StatsCache.java |   326 +
 .../metastore/hbase/TephraHBaseConnection.java  |   127 +
 .../metastore/hbase/VanillaHBaseConnection.java |   137 +
 .../stats/BinaryColumnStatsAggregator.java      |    35 +
 .../stats/BooleanColumnStatsAggregator.java     |    35 +
 .../hbase/stats/ColumnStatsAggregator.java      |    26 +
 .../stats/ColumnStatsAggregatorFactory.java     |    94 +
 .../stats/DecimalColumnStatsAggregator.java     |    43 +
 .../stats/DoubleColumnStatsAggregator.java      |    36 +
 .../hbase/stats/LongColumnStatsAggregator.java  |    36 +
 .../stats/StringColumnStatsAggregator.java      |    36 +
 .../hive/metastore/parser/ExpressionTree.java   |     9 +-
 .../metastore/hbase/hbase_metastore_proto.proto |   282 +
 .../DummyRawStoreControlledCommit.java          |    56 +-
 .../DummyRawStoreForJdoConnection.java          |    50 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |    43 +-
 .../hadoop/hive/metastore/hbase/MockUtils.java  |   199 +
 .../hbase/TestHBaseAggregateStatsCache.java     |   316 +
 .../hbase/TestHBaseFilterPlanUtil.java          |   483 +
 .../hive/metastore/hbase/TestHBaseStore.java    |  1307 +
 .../metastore/hbase/TestHBaseStoreCached.java   |   378 +
 .../hbase/TestSharedStorageDescriptor.java      |   153 +
 pom.xml                                         |     3 +-
 .../hadoop/hive/ql/plan/api/Adjacency.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Graph.java   |     2 +-
 .../hadoop/hive/ql/plan/api/Operator.java       |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Query.java   |     2 +-
 .../hadoop/hive/ql/plan/api/QueryPlan.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Stage.java   |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Task.java    |     2 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |     6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    64 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |     9 +-
 .../hadoop/hive/ql/metadata/Partition.java      |    29 +-
 .../AuthorizationPreEventListener.java          |     2 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |     3 +
 .../hive/metastore/TestMetastoreExpr.java       |     2 +-
 .../dynpart_sort_opt_vectorization.q            |     2 +
 .../clientpositive/dynpart_sort_optimization.q  |     2 +
 .../dynpart_sort_opt_vectorization.q.out        |    12 +-
 .../dynpart_sort_optimization.q.out             |    12 +-
 .../tez/dynpart_sort_opt_vectorization.q.out    |    12 +-
 .../tez/dynpart_sort_optimization.q.out         |    12 +-
 ql/src/test/templates/TestCliDriver.vm          |     3 +-
 .../hadoop/hive/serde/test/InnerStruct.java     |     2 +-
 .../hadoop/hive/serde/test/ThriftTestObj.java   |     2 +-
 .../hadoop/hive/serde2/thrift/test/Complex.java |     2 +-
 .../hive/serde2/thrift/test/IntString.java      |     2 +-
 .../hive/serde2/thrift/test/MegaStruct.java     |     2 +-
 .../hive/serde2/thrift/test/MiniStruct.java     |     2 +-
 .../hive/serde2/thrift/test/SetIntString.java   |     2 +-
 .../BinarySortableSerDeWithEndPrefix.java       |    41 +
 .../hadoop/hive/service/HiveClusterStatus.java  |     2 +-
 .../hive/service/HiveServerException.java       |     2 +-
 .../apache/hadoop/hive/service/ThriftHive.java  |     2 +-
 .../service/cli/thrift/TArrayTypeEntry.java     |     2 +-
 .../hive/service/cli/thrift/TBinaryColumn.java  |     2 +-
 .../hive/service/cli/thrift/TBoolColumn.java    |     2 +-
 .../hive/service/cli/thrift/TBoolValue.java     |     2 +-
 .../hive/service/cli/thrift/TByteColumn.java    |     2 +-
 .../hive/service/cli/thrift/TByteValue.java     |     2 +-
 .../hive/service/cli/thrift/TCLIService.java    |     2 +-
 .../cli/thrift/TCancelDelegationTokenReq.java   |     2 +-
 .../cli/thrift/TCancelDelegationTokenResp.java  |     2 +-
 .../service/cli/thrift/TCancelOperationReq.java |     2 +-
 .../cli/thrift/TCancelOperationResp.java        |     2 +-
 .../service/cli/thrift/TCloseOperationReq.java  |     2 +-
 .../service/cli/thrift/TCloseOperationResp.java |     2 +-
 .../service/cli/thrift/TCloseSessionReq.java    |     2 +-
 .../service/cli/thrift/TCloseSessionResp.java   |     2 +-
 .../hive/service/cli/thrift/TColumnDesc.java    |     2 +-
 .../hive/service/cli/thrift/TDoubleColumn.java  |     2 +-
 .../hive/service/cli/thrift/TDoubleValue.java   |     2 +-
 .../cli/thrift/TExecuteStatementReq.java        |     2 +-
 .../cli/thrift/TExecuteStatementResp.java       |     2 +-
 .../service/cli/thrift/TFetchResultsReq.java    |     2 +-
 .../service/cli/thrift/TFetchResultsResp.java   |     2 +-
 .../service/cli/thrift/TGetCatalogsReq.java     |     2 +-
 .../service/cli/thrift/TGetCatalogsResp.java    |     2 +-
 .../hive/service/cli/thrift/TGetColumnsReq.java |     2 +-
 .../service/cli/thrift/TGetColumnsResp.java     |     2 +-
 .../cli/thrift/TGetDelegationTokenReq.java      |     2 +-
 .../cli/thrift/TGetDelegationTokenResp.java     |     2 +-
 .../service/cli/thrift/TGetFunctionsReq.java    |     2 +-
 .../service/cli/thrift/TGetFunctionsResp.java   |     2 +-
 .../hive/service/cli/thrift/TGetInfoReq.java    |     2 +-
 .../hive/service/cli/thrift/TGetInfoResp.java   |     2 +-
 .../cli/thrift/TGetOperationStatusReq.java      |     2 +-
 .../cli/thrift/TGetOperationStatusResp.java     |     2 +-
 .../cli/thrift/TGetResultSetMetadataReq.java    |     2 +-
 .../cli/thrift/TGetResultSetMetadataResp.java   |     2 +-
 .../hive/service/cli/thrift/TGetSchemasReq.java |     2 +-
 .../service/cli/thrift/TGetSchemasResp.java     |     2 +-
 .../service/cli/thrift/TGetTableTypesReq.java   |     2 +-
 .../service/cli/thrift/TGetTableTypesResp.java  |     2 +-
 .../hive/service/cli/thrift/TGetTablesReq.java  |     2 +-
 .../hive/service/cli/thrift/TGetTablesResp.java |     2 +-
 .../service/cli/thrift/TGetTypeInfoReq.java     |     2 +-
 .../service/cli/thrift/TGetTypeInfoResp.java    |     2 +-
 .../service/cli/thrift/THandleIdentifier.java   |     2 +-
 .../hive/service/cli/thrift/TI16Column.java     |     2 +-
 .../hive/service/cli/thrift/TI16Value.java      |     2 +-
 .../hive/service/cli/thrift/TI32Column.java     |     2 +-
 .../hive/service/cli/thrift/TI32Value.java      |     2 +-
 .../hive/service/cli/thrift/TI64Column.java     |     2 +-
 .../hive/service/cli/thrift/TI64Value.java      |     2 +-
 .../hive/service/cli/thrift/TMapTypeEntry.java  |     2 +-
 .../service/cli/thrift/TOpenSessionReq.java     |     2 +-
 .../service/cli/thrift/TOpenSessionResp.java    |     2 +-
 .../service/cli/thrift/TOperationHandle.java    |     2 +-
 .../service/cli/thrift/TPrimitiveTypeEntry.java |     2 +-
 .../cli/thrift/TRenewDelegationTokenReq.java    |     2 +-
 .../cli/thrift/TRenewDelegationTokenResp.java   |     2 +-
 .../apache/hive/service/cli/thrift/TRow.java    |     2 +-
 .../apache/hive/service/cli/thrift/TRowSet.java |     2 +-
 .../hive/service/cli/thrift/TSessionHandle.java |     2 +-
 .../apache/hive/service/cli/thrift/TStatus.java |     2 +-
 .../hive/service/cli/thrift/TStringColumn.java  |     2 +-
 .../hive/service/cli/thrift/TStringValue.java   |     2 +-
 .../service/cli/thrift/TStructTypeEntry.java    |     2 +-
 .../hive/service/cli/thrift/TTableSchema.java   |     2 +-
 .../hive/service/cli/thrift/TTypeDesc.java      |     2 +-
 .../service/cli/thrift/TTypeQualifiers.java     |     2 +-
 .../service/cli/thrift/TUnionTypeEntry.java     |     2 +-
 .../cli/thrift/TUserDefinedTypeEntry.java       |     2 +-
 .../gen-py/hive_service/ThriftHive-remote       |    49 +-
 299 files changed, 74878 insertions(+), 7084 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/data/conf/tez/hive-site.xml
----------------------------------------------------------------------
diff --cc data/conf/tez/hive-site.xml
index b4abe90,bcda3ea..c4c2a12
--- a/data/conf/tez/hive-site.xml
+++ b/data/conf/tez/hive-site.xml
@@@ -254,13 -254,13 +254,23 @@@
  </property>
  
  <property>
 +  <name>hive.tez.java.opts</name>
 +  <value> -Dlog4j.configurationFile=tez-container-log4j2.xml -Dtez.container.log.level=INFO -Dtez.container.root.logger=CLA </value>
 +</property>
 +
 +<property>
 +  <name>tez.am.launch.cmd-opts</name>
 +  <value> -Dlog4j.configurationFile=tez-container-log4j2.xml -Dtez.container.log.level=INFO -Dtez.container.root.logger=CLA </value>
 +</property>
 +
++<property>
+   <name>hive.metastore.fastpath</name>
+   <value>true</value>
+ </property>
+ 
+ <property>
+   <name>hive.metastore.rawstore.impl</name>
+   <value>org.apache.hadoop.hive.metastore.hbase.HBaseStore</value>
+ </property>
+ 
  </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52383033/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
----------------------------------------------------------------------


[02/52] [abbrv] hive git commit: HIVE-11300 HBase metastore: Support token and master key methods (gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/a310524c/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
index 314fc7f..5c5818a 100644
--- a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
+++ b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
@@ -10222,73 +10222,43 @@ public final class HbaseMetastoreProto {
     // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.Database)
   }
 
-  public interface FieldSchemaOrBuilder
+  public interface DelegationTokenOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required string name = 1;
-    /**
-     * <code>required string name = 1;</code>
-     */
-    boolean hasName();
-    /**
-     * <code>required string name = 1;</code>
-     */
-    java.lang.String getName();
-    /**
-     * <code>required string name = 1;</code>
-     */
-    com.google.protobuf.ByteString
-        getNameBytes();
-
-    // required string type = 2;
-    /**
-     * <code>required string type = 2;</code>
-     */
-    boolean hasType();
-    /**
-     * <code>required string type = 2;</code>
-     */
-    java.lang.String getType();
-    /**
-     * <code>required string type = 2;</code>
-     */
-    com.google.protobuf.ByteString
-        getTypeBytes();
-
-    // optional string comment = 3;
+    // required string token_str = 1;
     /**
-     * <code>optional string comment = 3;</code>
+     * <code>required string token_str = 1;</code>
      */
-    boolean hasComment();
+    boolean hasTokenStr();
     /**
-     * <code>optional string comment = 3;</code>
+     * <code>required string token_str = 1;</code>
      */
-    java.lang.String getComment();
+    java.lang.String getTokenStr();
     /**
-     * <code>optional string comment = 3;</code>
+     * <code>required string token_str = 1;</code>
      */
     com.google.protobuf.ByteString
-        getCommentBytes();
+        getTokenStrBytes();
   }
   /**
-   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.FieldSchema}
+   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.DelegationToken}
    */
-  public static final class FieldSchema extends
+  public static final class DelegationToken extends
       com.google.protobuf.GeneratedMessage
-      implements FieldSchemaOrBuilder {
-    // Use FieldSchema.newBuilder() to construct.
-    private FieldSchema(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements DelegationTokenOrBuilder {
+    // Use DelegationToken.newBuilder() to construct.
+    private DelegationToken(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private FieldSchema(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private DelegationToken(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final FieldSchema defaultInstance;
-    public static FieldSchema getDefaultInstance() {
+    private static final DelegationToken defaultInstance;
+    public static DelegationToken getDefaultInstance() {
       return defaultInstance;
     }
 
-    public FieldSchema getDefaultInstanceForType() {
+    public DelegationToken getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -10298,7 +10268,7 @@ public final class HbaseMetastoreProto {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private FieldSchema(
+    private DelegationToken(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -10323,17 +10293,7 @@ public final class HbaseMetastoreProto {
             }
             case 10: {
               bitField0_ |= 0x00000001;
-              name_ = input.readBytes();
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              type_ = input.readBytes();
-              break;
-            }
-            case 26: {
-              bitField0_ |= 0x00000004;
-              comment_ = input.readBytes();
+              tokenStr_ = input.readBytes();
               break;
             }
           }
@@ -10350,132 +10310,46 @@ public final class HbaseMetastoreProto {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_descriptor;
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_DelegationToken_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_fieldAccessorTable
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_DelegationToken_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.Builder.class);
+              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<FieldSchema> PARSER =
-        new com.google.protobuf.AbstractParser<FieldSchema>() {
-      public FieldSchema parsePartialFrom(
+    public static com.google.protobuf.Parser<DelegationToken> PARSER =
+        new com.google.protobuf.AbstractParser<DelegationToken>() {
+      public DelegationToken parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new FieldSchema(input, extensionRegistry);
+        return new DelegationToken(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<FieldSchema> getParserForType() {
+    public com.google.protobuf.Parser<DelegationToken> getParserForType() {
       return PARSER;
     }
 
     private int bitField0_;
-    // required string name = 1;
-    public static final int NAME_FIELD_NUMBER = 1;
-    private java.lang.Object name_;
+    // required string token_str = 1;
+    public static final int TOKEN_STR_FIELD_NUMBER = 1;
+    private java.lang.Object tokenStr_;
     /**
-     * <code>required string name = 1;</code>
+     * <code>required string token_str = 1;</code>
      */
-    public boolean hasName() {
+    public boolean hasTokenStr() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required string name = 1;</code>
-     */
-    public java.lang.String getName() {
-      java.lang.Object ref = name_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          name_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string name = 1;</code>
-     */
-    public com.google.protobuf.ByteString
-        getNameBytes() {
-      java.lang.Object ref = name_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        name_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // required string type = 2;
-    public static final int TYPE_FIELD_NUMBER = 2;
-    private java.lang.Object type_;
-    /**
-     * <code>required string type = 2;</code>
-     */
-    public boolean hasType() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required string type = 2;</code>
-     */
-    public java.lang.String getType() {
-      java.lang.Object ref = type_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          type_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string type = 2;</code>
-     */
-    public com.google.protobuf.ByteString
-        getTypeBytes() {
-      java.lang.Object ref = type_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        type_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional string comment = 3;
-    public static final int COMMENT_FIELD_NUMBER = 3;
-    private java.lang.Object comment_;
-    /**
-     * <code>optional string comment = 3;</code>
-     */
-    public boolean hasComment() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional string comment = 3;</code>
+     * <code>required string token_str = 1;</code>
      */
-    public java.lang.String getComment() {
-      java.lang.Object ref = comment_;
+    public java.lang.String getTokenStr() {
+      java.lang.Object ref = tokenStr_;
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
@@ -10483,22 +10357,22 @@ public final class HbaseMetastoreProto {
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
-          comment_ = s;
+          tokenStr_ = s;
         }
         return s;
       }
     }
     /**
-     * <code>optional string comment = 3;</code>
+     * <code>required string token_str = 1;</code>
      */
     public com.google.protobuf.ByteString
-        getCommentBytes() {
-      java.lang.Object ref = comment_;
+        getTokenStrBytes() {
+      java.lang.Object ref = tokenStr_;
       if (ref instanceof java.lang.String) {
         com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
-        comment_ = b;
+        tokenStr_ = b;
         return b;
       } else {
         return (com.google.protobuf.ByteString) ref;
@@ -10506,20 +10380,14 @@ public final class HbaseMetastoreProto {
     }
 
     private void initFields() {
-      name_ = "";
-      type_ = "";
-      comment_ = "";
+      tokenStr_ = "";
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (!hasName()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasType()) {
+      if (!hasTokenStr()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -10531,13 +10399,7 @@ public final class HbaseMetastoreProto {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, getNameBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, getTypeBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBytes(3, getCommentBytes());
+        output.writeBytes(1, getTokenStrBytes());
       }
       getUnknownFields().writeTo(output);
     }
@@ -10550,15 +10412,7 @@ public final class HbaseMetastoreProto {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, getNameBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, getTypeBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(3, getCommentBytes());
+          .computeBytesSize(1, getTokenStrBytes());
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -10572,53 +10426,53 @@ public final class HbaseMetastoreProto {
       return super.writeReplace();
     }
 
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(byte[] data)
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseDelimitedFrom(
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -10627,7 +10481,7 @@ public final class HbaseMetastoreProto {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -10639,24 +10493,24 @@ public final class HbaseMetastoreProto {
       return builder;
     }
     /**
-     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.FieldSchema}
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.DelegationToken}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchemaOrBuilder {
+       implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationTokenOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_descriptor;
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_DelegationToken_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_fieldAccessorTable
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_DelegationToken_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.Builder.class);
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.newBuilder()
+      // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -10676,12 +10530,8 @@ public final class HbaseMetastoreProto {
 
       public Builder clear() {
         super.clear();
-        name_ = "";
+        tokenStr_ = "";
         bitField0_ = (bitField0_ & ~0x00000001);
-        type_ = "";
-        bitField0_ = (bitField0_ & ~0x00000002);
-        comment_ = "";
-        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -10691,66 +10541,48 @@ public final class HbaseMetastoreProto {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_descriptor;
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_DelegationToken_descriptor;
       }
 
-      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema getDefaultInstanceForType() {
-        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.getDefaultInstance();
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken getDefaultInstanceForType() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema build() {
-        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema result = buildPartial();
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken build() {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema buildPartial() {
-        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema(this);
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken buildPartial() {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.name_ = name_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.type_ = type_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.comment_ = comment_;
+        result.tokenStr_ = tokenStr_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema) {
-          return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema)other);
+        if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken) {
+          return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema other) {
-        if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.getDefaultInstance()) return this;
-        if (other.hasName()) {
+      public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken other) {
+        if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken.getDefaultInstance()) return this;
+        if (other.hasTokenStr()) {
           bitField0_ |= 0x00000001;
-          name_ = other.name_;
-          onChanged();
-        }
-        if (other.hasType()) {
-          bitField0_ |= 0x00000002;
-          type_ = other.type_;
-          onChanged();
-        }
-        if (other.hasComment()) {
-          bitField0_ |= 0x00000004;
-          comment_ = other.comment_;
+          tokenStr_ = other.tokenStr_;
           onChanged();
         }
         this.mergeUnknownFields(other.getUnknownFields());
@@ -10758,11 +10590,7 @@ public final class HbaseMetastoreProto {
       }
 
       public final boolean isInitialized() {
-        if (!hasName()) {
-          
-          return false;
-        }
-        if (!hasType()) {
+        if (!hasTokenStr()) {
           
           return false;
         }
@@ -10773,11 +10601,11 @@ public final class HbaseMetastoreProto {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parsedMessage = null;
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.DelegationToken) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -10788,346 +10616,158 @@ public final class HbaseMetastoreProto {
       }
       private int bitField0_;
 
-      // required string name = 1;
-      private java.lang.Object name_ = "";
+      // required string token_str = 1;
+      private java.lang.Object tokenStr_ = "";
       /**
-       * <code>required string name = 1;</code>
+       * <code>required string token_str = 1;</code>
        */
-      public boolean hasName() {
+      public boolean hasTokenStr() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required string name = 1;</code>
+       * <code>required string token_str = 1;</code>
        */
-      public java.lang.String getName() {
-        java.lang.Object ref = name_;
+      public java.lang.String getTokenStr() {
+        java.lang.Object ref = tokenStr_;
         if (!(ref instanceof java.lang.String)) {
           java.lang.String s = ((com.google.protobuf.ByteString) ref)
               .toStringUtf8();
-          name_ = s;
+          tokenStr_ = s;
           return s;
         } else {
           return (java.lang.String) ref;
         }
       }
       /**
-       * <code>required string name = 1;</code>
+       * <code>required string token_str = 1;</code>
        */
       public com.google.protobuf.ByteString
-          getNameBytes() {
-        java.lang.Object ref = name_;
+          getTokenStrBytes() {
+        java.lang.Object ref = tokenStr_;
         if (ref instanceof String) {
           com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
-          name_ = b;
+          tokenStr_ = b;
           return b;
         } else {
           return (com.google.protobuf.ByteString) ref;
         }
       }
       /**
-       * <code>required string name = 1;</code>
+       * <code>required string token_str = 1;</code>
        */
-      public Builder setName(
+      public Builder setTokenStr(
           java.lang.String value) {
         if (value == null) {
     throw new NullPointerException();
   }
   bitField0_ |= 0x00000001;
-        name_ = value;
+        tokenStr_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required string name = 1;</code>
+       * <code>required string token_str = 1;</code>
        */
-      public Builder clearName() {
+      public Builder clearTokenStr() {
         bitField0_ = (bitField0_ & ~0x00000001);
-        name_ = getDefaultInstance().getName();
+        tokenStr_ = getDefaultInstance().getTokenStr();
         onChanged();
         return this;
       }
       /**
-       * <code>required string name = 1;</code>
+       * <code>required string token_str = 1;</code>
        */
-      public Builder setNameBytes(
+      public Builder setTokenStrBytes(
           com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
   bitField0_ |= 0x00000001;
-        name_ = value;
-        onChanged();
-        return this;
-      }
-
-      // required string type = 2;
-      private java.lang.Object type_ = "";
-      /**
-       * <code>required string type = 2;</code>
-       */
-      public boolean hasType() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>required string type = 2;</code>
-       */
-      public java.lang.String getType() {
-        java.lang.Object ref = type_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          type_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>required string type = 2;</code>
-       */
-      public com.google.protobuf.ByteString
-          getTypeBytes() {
-        java.lang.Object ref = type_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          type_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>required string type = 2;</code>
-       */
-      public Builder setType(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        type_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string type = 2;</code>
-       */
-      public Builder clearType() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        type_ = getDefaultInstance().getType();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string type = 2;</code>
-       */
-      public Builder setTypeBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        type_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional string comment = 3;
-      private java.lang.Object comment_ = "";
-      /**
-       * <code>optional string comment = 3;</code>
-       */
-      public boolean hasComment() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional string comment = 3;</code>
-       */
-      public java.lang.String getComment() {
-        java.lang.Object ref = comment_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          comment_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string comment = 3;</code>
-       */
-      public com.google.protobuf.ByteString
-          getCommentBytes() {
-        java.lang.Object ref = comment_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          comment_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string comment = 3;</code>
-       */
-      public Builder setComment(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000004;
-        comment_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string comment = 3;</code>
-       */
-      public Builder clearComment() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        comment_ = getDefaultInstance().getComment();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string comment = 3;</code>
-       */
-      public Builder setCommentBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000004;
-        comment_ = value;
+        tokenStr_ = value;
         onChanged();
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.FieldSchema)
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.DelegationToken)
     }
 
     static {
-      defaultInstance = new FieldSchema(true);
+      defaultInstance = new DelegationToken(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.FieldSchema)
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.DelegationToken)
   }
 
-  public interface FunctionOrBuilder
+  public interface FieldSchemaOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional string class_name = 1;
+    // required string name = 1;
     /**
-     * <code>optional string class_name = 1;</code>
+     * <code>required string name = 1;</code>
      */
-    boolean hasClassName();
+    boolean hasName();
     /**
-     * <code>optional string class_name = 1;</code>
+     * <code>required string name = 1;</code>
      */
-    java.lang.String getClassName();
+    java.lang.String getName();
     /**
-     * <code>optional string class_name = 1;</code>
+     * <code>required string name = 1;</code>
      */
     com.google.protobuf.ByteString
-        getClassNameBytes();
+        getNameBytes();
 
-    // optional string owner_name = 2;
+    // required string type = 2;
     /**
-     * <code>optional string owner_name = 2;</code>
+     * <code>required string type = 2;</code>
      */
-    boolean hasOwnerName();
+    boolean hasType();
     /**
-     * <code>optional string owner_name = 2;</code>
+     * <code>required string type = 2;</code>
      */
-    java.lang.String getOwnerName();
+    java.lang.String getType();
     /**
-     * <code>optional string owner_name = 2;</code>
+     * <code>required string type = 2;</code>
      */
     com.google.protobuf.ByteString
-        getOwnerNameBytes();
-
-    // optional .org.apache.hadoop.hive.metastore.hbase.PrincipalType owner_type = 3;
-    /**
-     * <code>optional .org.apache.hadoop.hive.metastore.hbase.PrincipalType owner_type = 3;</code>
-     */
-    boolean hasOwnerType();
-    /**
-     * <code>optional .org.apache.hadoop.hive.metastore.hbase.PrincipalType owner_type = 3;</code>
-     */
-    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrincipalType getOwnerType();
-
-    // optional sint64 create_time = 4;
-    /**
-     * <code>optional sint64 create_time = 4;</code>
-     */
-    boolean hasCreateTime();
-    /**
-     * <code>optional sint64 create_time = 4;</code>
-     */
-    long getCreateTime();
-
-    // optional .org.apache.hadoop.hive.metastore.hbase.Function.FunctionType function_type = 5;
-    /**
-     * <code>optional .org.apache.hadoop.hive.metastore.hbase.Function.FunctionType function_type = 5;</code>
-     */
-    boolean hasFunctionType();
-    /**
-     * <code>optional .org.apache.hadoop.hive.metastore.hbase.Function.FunctionType function_type = 5;</code>
-     */
-    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.FunctionType getFunctionType();
+        getTypeBytes();
 
-    // repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;
-    /**
-     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
-     */
-    java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri> 
-        getResourceUrisList();
-    /**
-     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
-     */
-    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri getResourceUris(int index);
+    // optional string comment = 3;
     /**
-     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
+     * <code>optional string comment = 3;</code>
      */
-    int getResourceUrisCount();
+    boolean hasComment();
     /**
-     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
+     * <code>optional string comment = 3;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUriOrBuilder> 
-        getResourceUrisOrBuilderList();
+    java.lang.String getComment();
     /**
-     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
+     * <code>optional string comment = 3;</code>
      */
-    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUriOrBuilder getResourceUrisOrBuilder(
-        int index);
+    com.google.protobuf.ByteString
+        getCommentBytes();
   }
   /**
-   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.Function}
+   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.FieldSchema}
    */
-  public static final class Function extends
+  public static final class FieldSchema extends
       com.google.protobuf.GeneratedMessage
-      implements FunctionOrBuilder {
-    // Use Function.newBuilder() to construct.
-    private Function(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements FieldSchemaOrBuilder {
+    // Use FieldSchema.newBuilder() to construct.
+    private FieldSchema(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private Function(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private FieldSchema(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final Function defaultInstance;
-    public static Function getDefaultInstance() {
+    private static final FieldSchema defaultInstance;
+    public static FieldSchema getDefaultInstance() {
       return defaultInstance;
     }
 
-    public Function getDefaultInstanceForType() {
+    public FieldSchema getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -11137,7 +10777,7 @@ public final class HbaseMetastoreProto {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private Function(
+    private FieldSchema(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -11162,47 +10802,17 @@ public final class HbaseMetastoreProto {
             }
             case 10: {
               bitField0_ |= 0x00000001;
-              className_ = input.readBytes();
+              name_ = input.readBytes();
               break;
             }
             case 18: {
               bitField0_ |= 0x00000002;
-              ownerName_ = input.readBytes();
-              break;
-            }
-            case 24: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrincipalType value = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrincipalType.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(3, rawValue);
-              } else {
-                bitField0_ |= 0x00000004;
-                ownerType_ = value;
-              }
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              createTime_ = input.readSInt64();
-              break;
-            }
-            case 40: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.FunctionType value = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.FunctionType.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(5, rawValue);
-              } else {
-                bitField0_ |= 0x00000010;
-                functionType_ = value;
-              }
+              type_ = input.readBytes();
               break;
             }
-            case 50: {
-              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-                resourceUris_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri>();
-                mutable_bitField0_ |= 0x00000020;
-              }
-              resourceUris_.add(input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.PARSER, extensionRegistry));
+            case 26: {
+              bitField0_ |= 0x00000004;
+              comment_ = input.readBytes();
               break;
             }
           }
@@ -11213,528 +10823,1397 @@ public final class HbaseMetastoreProto {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-          resourceUris_ = java.util.Collections.unmodifiableList(resourceUris_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_Function_descriptor;
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_Function_fieldAccessorTable
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.Builder.class);
+              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<Function> PARSER =
-        new com.google.protobuf.AbstractParser<Function>() {
-      public Function parsePartialFrom(
+    public static com.google.protobuf.Parser<FieldSchema> PARSER =
+        new com.google.protobuf.AbstractParser<FieldSchema>() {
+      public FieldSchema parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new Function(input, extensionRegistry);
+        return new FieldSchema(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<Function> getParserForType() {
+    public com.google.protobuf.Parser<FieldSchema> getParserForType() {
       return PARSER;
     }
 
+    private int bitField0_;
+    // required string name = 1;
+    public static final int NAME_FIELD_NUMBER = 1;
+    private java.lang.Object name_;
     /**
-     * Protobuf enum {@code org.apache.hadoop.hive.metastore.hbase.Function.FunctionType}
+     * <code>required string name = 1;</code>
      */
-    public enum FunctionType
-        implements com.google.protobuf.ProtocolMessageEnum {
-      /**
-       * <code>JAVA = 1;</code>
-       */
-      JAVA(0, 1),
-      ;
-
-      /**
-       * <code>JAVA = 1;</code>
-       */
-      public static final int JAVA_VALUE = 1;
-
-
-      public final int getNumber() { return value; }
-
-      public static FunctionType valueOf(int value) {
-        switch (value) {
-          case 1: return JAVA;
-          default: return null;
+    public boolean hasName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string name = 1;</code>
+     */
+    public java.lang.String getName() {
+      java.lang.Object ref = name_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          name_ = s;
         }
+        return s;
       }
-
-      public static com.google.protobuf.Internal.EnumLiteMap<FunctionType>
-          internalGetValueMap() {
-        return internalValueMap;
-      }
-      private static com.google.protobuf.Internal.EnumLiteMap<FunctionType>
-          internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap<FunctionType>() {
-              public FunctionType findValueByNumber(int number) {
-                return FunctionType.valueOf(number);
-              }
-            };
-
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
-          getValueDescriptor() {
-        return getDescriptor().getValues().get(index);
-      }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptorForType() {
-        return getDescriptor();
-      }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptor() {
-        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.getDescriptor().getEnumTypes().get(0);
+    }
+    /**
+     * <code>required string name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getNameBytes() {
+      java.lang.Object ref = name_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        name_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
       }
+    }
 
-      private static final FunctionType[] VALUES = values();
-
-      public static FunctionType valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-        if (desc.getType() != getDescriptor()) {
-          throw new java.lang.IllegalArgumentException(
-            "EnumValueDescriptor is not for this type.");
+    // required string type = 2;
+    public static final int TYPE_FIELD_NUMBER = 2;
+    private java.lang.Object type_;
+    /**
+     * <code>required string type = 2;</code>
+     */
+    public boolean hasType() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required string type = 2;</code>
+     */
+    public java.lang.String getType() {
+      java.lang.Object ref = type_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          type_ = s;
         }
-        return VALUES[desc.getIndex()];
+        return s;
       }
-
-      private final int index;
-      private final int value;
-
-      private FunctionType(int index, int value) {
-        this.index = index;
-        this.value = value;
+    }
+    /**
+     * <code>required string type = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getTypeBytes() {
+      java.lang.Object ref = type_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        type_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
       }
-
-      // @@protoc_insertion_point(enum_scope:org.apache.hadoop.hive.metastore.hbase.Function.FunctionType)
     }
 
-    public interface ResourceUriOrBuilder
-        extends com.google.protobuf.MessageOrBuilder {
-
-      // required .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri.ResourceType resource_type = 1;
-      /**
-       * <code>required .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri.ResourceType resource_type = 1;</code>
-       */
-      boolean hasResourceType();
-      /**
-       * <code>required .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri.ResourceType resource_type = 1;</code>
-       */
-      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.ResourceType getResourceType();
-
-      // required string uri = 2;
-      /**
-       * <code>required string uri = 2;</code>
-       */
-      boolean hasUri();
-      /**
-       * <code>required string uri = 2;</code>
-       */
-      java.lang.String getUri();
-      /**
-       * <code>required string uri = 2;</code>
-       */
-      com.google.protobuf.ByteString
-          getUriBytes();
+    // optional string comment = 3;
+    public static final int COMMENT_FIELD_NUMBER = 3;
+    private java.lang.Object comment_;
+    /**
+     * <code>optional string comment = 3;</code>
+     */
+    public boolean hasComment() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri}
+     * <code>optional string comment = 3;</code>
      */
-    public static final class ResourceUri extends
-        com.google.protobuf.GeneratedMessage
-        implements ResourceUriOrBuilder {
-      // Use ResourceUri.newBuilder() to construct.
-      private ResourceUri(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-        super(builder);
-        this.unknownFields = builder.getUnknownFields();
+    public java.lang.String getComment() {
+      java.lang.Object ref = comment_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          comment_ = s;
+        }
+        return s;
       }
-      private ResourceUri(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    }
+    /**
+     * <code>optional string comment = 3;</code>
+     */
+    public com.google.protobuf.ByteString
+        getCommentBytes() {
+      java.lang.Object ref = comment_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        comment_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
 
-      private static final ResourceUri defaultInstance;
-      public static ResourceUri getDefaultInstance() {
-        return defaultInstance;
+    private void initFields() {
+      name_ = "";
+      type_ = "";
+      comment_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasName()) {
+        memoizedIsInitialized = 0;
+        return false;
       }
+      if (!hasType()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
 
-      public ResourceUri getDefaultInstanceForType() {
-        return defaultInstance;
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getTypeBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, getCommentBytes());
       }
+      getUnknownFields().writeTo(output);
+    }
 
-      private final com.google.protobuf.UnknownFieldSet unknownFields;
-      @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
-          getUnknownFields() {
-        return this.unknownFields;
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getNameBytes());
       }
-      private ResourceUri(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        initFields();
-        int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
-        try {
-          boolean done = false;
-          while (!done) {
-            int tag = input.readTag();
-            switch (tag) {
-              case 0:
-                done = true;
-                break;
-              default: {
-                if (!parseUnknownField(input, unknownFields,
-                                       extensionRegistry, tag)) {
-                  done = true;
-                }
-                break;
-              }
-              case 8: {
-                int rawValue = input.readEnum();
-                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.ResourceType value = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.ResourceType.valueOf(rawValue);
-                if (value == null) {
-                  unknownFields.mergeVarintField(1, rawValue);
-                } else {
-                  bitField0_ |= 0x00000001;
-                  resourceType_ = value;
-                }
-                break;
-              }
-              case 18: {
-                bitField0_ |= 0x00000002;
-                uri_ = input.readBytes();
-                break;
-              }
-            }
-          }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          throw e.setUnfinishedMessage(this);
-        } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
-              e.getMessage()).setUnfinishedMessage(this);
-        } finally {
-          this.unknownFields = unknownFields.build();
-          makeExtensionsImmutable();
-        }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getTypeBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getCommentBytes());
       }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.FieldSchema}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchemaOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_Function_ResourceUri_descriptor;
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_Function_ResourceUri_fieldAccessorTable
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.Builder.class);
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.Builder.class);
       }
 
-      public static com.google.protobuf.Parser<ResourceUri> PARSER =
-          new com.google.protobuf.AbstractParser<ResourceUri>() {
-        public ResourceUri parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
-          return new ResourceUri(input, extensionRegistry);
-        }
-      };
-
-      @java.lang.Override
-      public com.google.protobuf.Parser<ResourceUri> getParserForType() {
-        return PARSER;
+      // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
       }
 
-      /**
-       * Protobuf enum {@code org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri.ResourceType}
-       */
-      public enum ResourceType
-          implements com.google.protobuf.ProtocolMessageEnum {
-        /**
-         * <code>JAR = 1;</code>
-         */
-        JAR(0, 1),
-        /**
-         * <code>FILE = 2;</code>
-         */
-        FILE(1, 2),
-        /**
-         * <code>ARCHIVE = 3;</code>
-         */
-        ARCHIVE(2, 3),
-        ;
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
 
-        /**
-         * <code>JAR = 1;</code>
-         */
-        public static final int JAR_VALUE = 1;
-        /**
-         * <code>FILE = 2;</code>
-         */
-        public static final int FILE_VALUE = 2;
-        /**
-         * <code>ARCHIVE = 3;</code>
-         */
-        public static final int ARCHIVE_VALUE = 3;
+      public Builder clear() {
+        super.clear();
+        name_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        type_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        comment_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
 
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
 
-        public final int getNumber() { return value; }
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_FieldSchema_descriptor;
+      }
 
-        public static ResourceType valueOf(int value) {
-          switch (value) {
-            case 1: return JAR;
-            case 2: return FILE;
-            case 3: return ARCHIVE;
-            default: return null;
-          }
-        }
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema getDefaultInstanceForType() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.getDefaultInstance();
+      }
 
-        public static com.google.protobuf.Internal.EnumLiteMap<ResourceType>
-            internalGetValueMap() {
-          return internalValueMap;
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema build() {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
         }
-        private static com.google.protobuf.Internal.EnumLiteMap<ResourceType>
-            internalValueMap =
-              new com.google.protobuf.Internal.EnumLiteMap<ResourceType>() {
-                public ResourceType findValueByNumber(int number) {
-                  return ResourceType.valueOf(number);
-                }
-              };
+        return result;
+      }
 
-        public final com.google.protobuf.Descriptors.EnumValueDescriptor
-            getValueDescriptor() {
-          return getDescriptor().getValues().get(index);
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema buildPartial() {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
         }
-        public final com.google.protobuf.Descriptors.EnumDescriptor
-            getDescriptorForType() {
-          return getDescriptor();
+        result.name_ = name_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
         }
-        public static final com.google.protobuf.Descriptors.EnumDescriptor
-            getDescriptor() {
-          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.getDescriptor().getEnumTypes().get(0);
+        result.type_ = type_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
         }
+        result.comment_ = comment_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
 
-        private static final ResourceType[] VALUES = values();
-
-        public static ResourceType valueOf(
-            com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-          if (desc.getType() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "EnumValueDescriptor is not for this type.");
-          }
-          return VALUES[desc.getIndex()];
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema) {
+          return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
         }
+      }
 
-        private final int index;
-        private final int value;
-
-        private ResourceType(int index, int value) {
-          this.index = index;
-          this.value = value;
+      public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema other) {
+        if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema.getDefaultInstance()) return this;
+        if (other.hasName()) {
+          bitField0_ |= 0x00000001;
+          name_ = other.name_;
+          onChanged();
         }
-
-        // @@protoc_insertion_point(enum_scope:org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri.ResourceType)
+        if (other.hasType()) {
+          bitField0_ |= 0x00000002;
+          type_ = other.type_;
+          onChanged();
+        }
+        if (other.hasComment()) {
+          bitField0_ |= 0x00000004;
+          comment_ = other.comment_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
       }
 
-      private int bitField0_;
-      // required .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri.ResourceType resource_type = 1;
-      public static final int RESOURCE_TYPE_FIELD_NUMBER = 1;
-      private org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.ResourceType resourceType_;
-      /**
-       * <code>required .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri.ResourceType resource_type = 1;</code>
-       */
-      public boolean hasResourceType() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
+      public final boolean isInitialized() {
+        if (!hasName()) {
+          
+          return false;
+        }
+        if (!hasType()) {
+          
+          return false;
+        }
+        return true;
       }
-      /**
-       * <code>required .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri.ResourceType resource_type = 1;</code>
-       */
-      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.ResourceType getResourceType() {
-        return resourceType_;
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.FieldSchema) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
       }
+      private int bitField0_;
 
-      // required string uri = 2;
-      public static final int URI_FIELD_NUMBER = 2;
-      private java.lang.Object uri_;
+      // required string name = 1;
+      private java.lang.Object name_ = "";
       /**
-       * <code>required string uri = 2;</code>
+       * <code>required string name = 1;</code>
        */
-      public boolean hasUri() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
+      public boolean hasName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required string uri = 2;</code>
+       * <code>required string name = 1;</code>
        */
-      public java.lang.String getUri() {
-        java.lang.Object ref = uri_;
-        if (ref instanceof java.lang.String) {
-          return (java.lang.String) ref;
-        } else {
-          com.google.protobuf.ByteString bs = 
-              (com.google.protobuf.ByteString) ref;
-          java.lang.String s = bs.toStringUtf8();
-          if (bs.isValidUtf8()) {
-            uri_ = s;
-          }
+      public java.lang.String getName() {
+        java.lang.Object ref = name_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          name_ = s;
           return s;
+        } else {
+          return (java.lang.String) ref;
         }
       }
       /**
-       * <code>required string uri = 2;</code>
+       * <code>required string name = 1;</code>
        */
       public com.google.protobuf.ByteString
-          getUriBytes() {
-        java.lang.Object ref = uri_;
-        if (ref instanceof java.lang.String) {
+          getNameBytes() {
+        java.lang.Object ref = name_;
+        if (ref instanceof String) {
           com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
-          uri_ = b;
+          name_ = b;
           return b;
         } else {
           return (com.google.protobuf.ByteString) ref;
         }
       }
-
-      private void initFields() {
-        resourceType_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri.ResourceType.JAR;
-        uri_ = "";
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder setName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+        return this;
       }
-      private byte memoizedIsInitialized = -1;
-      public final boolean isInitialized() {
-        byte isInitialized = memoizedIsInitialized;
-        if (isInitialized != -1) return isInitialized == 1;
-
-        if (!hasResourceType()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-        if (!hasUri()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-        memoizedIsInitialized = 1;
-        return true;
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder clearName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        name_ = getDefaultInstance().getName();
+        onChanged();
+        return this;
       }
-
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
-                          throws java.io.IOException {
-        getSerializedSize();
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          output.writeEnum(1, resourceType_.getNumber());
-        }
-        if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          output.writeBytes(2, getUriBytes());
-        }
-        getUnknownFields().writeTo(output);
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder setNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+        return this;
       }
 
-      private int memoizedSerializedSize = -1;
-      public int getSerializedSize() {
-        int size = memoizedSerializedSize;
-        if (size != -1) return size;
-
-        size = 0;
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
-            .computeEnumSize(1, resourceType_.getNumber());
-        }
-        if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          size += com.google.protobuf.CodedOutputStream
-            .computeBytesSize(2, getUriBytes());
-        }
-        size += getUnknownFields().getSerializedSize();
-        memoizedSerializedSize = size;
-        return size;
+      // required string type = 2;
+      private java.lang.Object type_ = "";
+      /**
+       * <code>required string type = 2;</code>
+       */
+      public boolean hasType() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
-
-      private static final long serialVersionUID = 0L;
-      @java.lang.Override
-      protected java.lang.Object writeReplace()
-          throws java.io.ObjectStreamException {
-        return super.writeReplace();
+      /**
+       * <code>required string type = 2;</code>
+       */
+      public java.lang.String getType() {
+        java.lang.Object ref = type_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          type_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
       }
-
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data);
+      /**
+       * <code>required string type = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getTypeBytes() {
+        java.lang.Object ref = type_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          type_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data, extensionRegistry);
+      /**
+       * <code>required string type = 2;</code>
+       */
+      public Builder setType(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        type_ = value;
+        onChanged();
+        return this;
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data);
+      /**
+       * <code>required string type = 2;</code>
+       */
+      public Builder clearType() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        type_ = getDefaultInstance().getType();
+        onChanged();
+        return this;
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseFrom(
-          byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data, extensionRegistry);
+      /**
+       * <code>required string type = 2;</code>
+       */
+      public Builder setTypeBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        type_ = value;
+        onChanged();
+        return this;
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseFrom(java.io.InputStream input)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input);
+
+      // optional string comment = 3;
+      private java.lang.Object comment_ = "";
+      /**
+       * <code>optional string comment = 3;</code>
+       */
+      public boolean hasComment() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseFrom(
-          java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input, extensionRegistry);
+      /**
+       * <code>optional string comment = 3;</code>
+       */
+      public java.lang.String getComment() {
+        java.lang.Object ref = comment_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          comment_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseDelimitedFrom(java.io.InputStream input)
-          throws java.io.IOException {
-        return PARSER.parseDelimitedFrom(input);
+      /**
+       * <code>optional string comment = 3;</code>
+       */
+      public com.google.protobuf.ByteString
+          getCommentBytes() {
+        java.lang.Object ref = comment_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          comment_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseDelimitedFrom(
-          java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      /**
+       * <code>optional string comment = 3;</code>
+       */
+      public Builder setComment(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        comment_ = value;
+        onChanged();
+        return this;
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseFrom(
-          com.google.protobuf.CodedInputStream input)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input);
+      /**
+       * <code>optional string comment = 3;</code>
+       */
+      public Builder clearComment() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        comment_ = getDefaultInstance().getComment();
+        onChanged();
+        return this;
       }
-      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input, extensionRegistry);
+      /**
+       * <code>optional string comment = 3;</code>
+       */
+      public Builder setCommentBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        comment_ = value;
+        onChanged();
+        return this;
       }
 
-      public static Builder newBuilder() { return Builder.create(); }
-      public Builder newBuilderForType() { return newBuilder(); }
-      public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri prototype) {
-        return newBuilder().mergeFrom(prototype);
-      }
-      public Builder toBuilder() { return newBuilder(this); }
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.FieldSchema)
+    }
 
-      @java.lang.Override
-      protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        Builder builder = new Builder(parent);
-        return builder;
-      }
-      /**
-       * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri}
-       */
-      public static final class Builder extends
-          com.google.protobuf.GeneratedMessage.Builder<Builder>
+    static {
+      defaultInstance = new FieldSchema(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.FieldSchema)
+  }
+
+  public interface FunctionOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional string class_name = 1;
+    /**
+     * <code>optional string class_name = 1;</code>
+     */
+    boolean hasClassName();
+    /**
+     * <code>optional string class_name = 1;</code>
+     */
+    java.lang.String getClassName();
+    /**
+     * <code>optional string class_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getClassNameBytes();
+
+    // optional string owner_name = 2;
+    /**
+     * <code>optional string owner_name = 2;</code>
+     */
+    boolean hasOwnerName();
+    /**
+     * <code>optional string owner_name = 2;</code>
+     */
+    java.lang.String getOwnerName();
+    /**
+     * <code>optional string owner_name = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getOwnerNameBytes();
+
+    // optional .org.apache.hadoop.hive.metastore.hbase.PrincipalType owner_type = 3;
+    /**
+     * <code>optional .org.apache.hadoop.hive.metastore.hbase.PrincipalType owner_type = 3;</code>
+     */
+    boolean hasOwnerType();
+    /**
+     * <code>optional .org.apache.hadoop.hive.metastore.hbase.PrincipalType owner_type = 3;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrincipalType getOwnerType();
+
+    // optional sint64 create_time = 4;
+    /**
+     * <code>optional sint64 create_time = 4;</code>
+     */
+    boolean hasCreateTime();
+    /**
+     * <code>optional sint64 create_time = 4;</code>
+     */
+    long getCreateTime();
+
+    // optional .org.apache.hadoop.hive.metastore.hbase.Function.FunctionType function_type = 5;
+    /**
+     * <code>optional .org.apache.hadoop.hive.metastore.hbase.Function.FunctionType function_type = 5;</code>
+     */
+    boolean hasFunctionType();
+    /**
+     * <code>optional .org.apache.hadoop.hive.metastore.hbase.Function.FunctionType function_type = 5;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.FunctionType getFunctionType();
+
+    // repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri> 
+        getResourceUrisList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUri getResourceUris(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
+     */
+    int getResourceUrisCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUriOrBuilder> 
+        getResourceUrisOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.Function.ResourceUri resource_uris = 6;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.Function.ResourceUriOrBuilder getResourceUrisOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.Function}
+   */
+  public static final class Function extends
+      com.google.protobuf.GeneratedMessage
+      implements FunctionOrBuilder {
+    // Use Function.newBuilder() to construct.
+    private Function(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private Function(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final Function defaultInstance;
+    public static Function getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public Function getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    privat

<TRUNCATED>

[33/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 36110e6..e072866 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -321,6 +321,24 @@ class FireEventRequest;
 
 class FireEventResponse;
 
+class MetadataPpdResult;
+
+class GetFileMetadataByExprResult;
+
+class GetFileMetadataByExprRequest;
+
+class GetFileMetadataResult;
+
+class GetFileMetadataRequest;
+
+class PutFileMetadataResult;
+
+class PutFileMetadataRequest;
+
+class ClearFileMetadataResult;
+
+class ClearFileMetadataRequest;
+
 class GetAllFunctionsResponse;
 
 class MetaException;
@@ -5401,6 +5419,359 @@ class FireEventResponse {
 
 void swap(FireEventResponse &a, FireEventResponse &b);
 
+
+class MetadataPpdResult {
+ public:
+
+  static const char* ascii_fingerprint; // = "07A9615F837F7D0A952B595DD3020972";
+  static const uint8_t binary_fingerprint[16]; // = {0x07,0xA9,0x61,0x5F,0x83,0x7F,0x7D,0x0A,0x95,0x2B,0x59,0x5D,0xD3,0x02,0x09,0x72};
+
+  MetadataPpdResult(const MetadataPpdResult&);
+  MetadataPpdResult& operator=(const MetadataPpdResult&);
+  MetadataPpdResult() : metadata(), includeBitset() {
+  }
+
+  virtual ~MetadataPpdResult() throw();
+  std::string metadata;
+  std::string includeBitset;
+
+  void __set_metadata(const std::string& val);
+
+  void __set_includeBitset(const std::string& val);
+
+  bool operator == (const MetadataPpdResult & rhs) const
+  {
+    if (!(metadata == rhs.metadata))
+      return false;
+    if (!(includeBitset == rhs.includeBitset))
+      return false;
+    return true;
+  }
+  bool operator != (const MetadataPpdResult &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const MetadataPpdResult & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const MetadataPpdResult& obj);
+};
+
+void swap(MetadataPpdResult &a, MetadataPpdResult &b);
+
+
+class GetFileMetadataByExprResult {
+ public:
+
+  static const char* ascii_fingerprint; // = "2B0C1B8D7599529A5797481BE308375D";
+  static const uint8_t binary_fingerprint[16]; // = {0x2B,0x0C,0x1B,0x8D,0x75,0x99,0x52,0x9A,0x57,0x97,0x48,0x1B,0xE3,0x08,0x37,0x5D};
+
+  GetFileMetadataByExprResult(const GetFileMetadataByExprResult&);
+  GetFileMetadataByExprResult& operator=(const GetFileMetadataByExprResult&);
+  GetFileMetadataByExprResult() : isSupported(0) {
+  }
+
+  virtual ~GetFileMetadataByExprResult() throw();
+  std::map<int64_t, MetadataPpdResult>  metadata;
+  bool isSupported;
+  std::vector<int64_t>  unknownFileIds;
+
+  void __set_metadata(const std::map<int64_t, MetadataPpdResult> & val);
+
+  void __set_isSupported(const bool val);
+
+  void __set_unknownFileIds(const std::vector<int64_t> & val);
+
+  bool operator == (const GetFileMetadataByExprResult & rhs) const
+  {
+    if (!(metadata == rhs.metadata))
+      return false;
+    if (!(isSupported == rhs.isSupported))
+      return false;
+    if (!(unknownFileIds == rhs.unknownFileIds))
+      return false;
+    return true;
+  }
+  bool operator != (const GetFileMetadataByExprResult &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetFileMetadataByExprResult & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprResult& obj);
+};
+
+void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b);
+
+
+class GetFileMetadataByExprRequest {
+ public:
+
+  static const char* ascii_fingerprint; // = "925353917FC0AF87976A2338011F5A31";
+  static const uint8_t binary_fingerprint[16]; // = {0x92,0x53,0x53,0x91,0x7F,0xC0,0xAF,0x87,0x97,0x6A,0x23,0x38,0x01,0x1F,0x5A,0x31};
+
+  GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest&);
+  GetFileMetadataByExprRequest& operator=(const GetFileMetadataByExprRequest&);
+  GetFileMetadataByExprRequest() : expr() {
+  }
+
+  virtual ~GetFileMetadataByExprRequest() throw();
+  std::vector<int64_t>  fileIds;
+  std::string expr;
+
+  void __set_fileIds(const std::vector<int64_t> & val);
+
+  void __set_expr(const std::string& val);
+
+  bool operator == (const GetFileMetadataByExprRequest & rhs) const
+  {
+    if (!(fileIds == rhs.fileIds))
+      return false;
+    if (!(expr == rhs.expr))
+      return false;
+    return true;
+  }
+  bool operator != (const GetFileMetadataByExprRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetFileMetadataByExprRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprRequest& obj);
+};
+
+void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b);
+
+
+class GetFileMetadataResult {
+ public:
+
+  static const char* ascii_fingerprint; // = "D18BCBD4BA945E7F6500F5CD95205706";
+  static const uint8_t binary_fingerprint[16]; // = {0xD1,0x8B,0xCB,0xD4,0xBA,0x94,0x5E,0x7F,0x65,0x00,0xF5,0xCD,0x95,0x20,0x57,0x06};
+
+  GetFileMetadataResult(const GetFileMetadataResult&);
+  GetFileMetadataResult& operator=(const GetFileMetadataResult&);
+  GetFileMetadataResult() : isSupported(0) {
+  }
+
+  virtual ~GetFileMetadataResult() throw();
+  std::map<int64_t, std::string>  metadata;
+  bool isSupported;
+
+  void __set_metadata(const std::map<int64_t, std::string> & val);
+
+  void __set_isSupported(const bool val);
+
+  bool operator == (const GetFileMetadataResult & rhs) const
+  {
+    if (!(metadata == rhs.metadata))
+      return false;
+    if (!(isSupported == rhs.isSupported))
+      return false;
+    return true;
+  }
+  bool operator != (const GetFileMetadataResult &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetFileMetadataResult & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const GetFileMetadataResult& obj);
+};
+
+void swap(GetFileMetadataResult &a, GetFileMetadataResult &b);
+
+
+class GetFileMetadataRequest {
+ public:
+
+  static const char* ascii_fingerprint; // = "E49D7D1A9013CC81CD0F69D631EF82E4";
+  static const uint8_t binary_fingerprint[16]; // = {0xE4,0x9D,0x7D,0x1A,0x90,0x13,0xCC,0x81,0xCD,0x0F,0x69,0xD6,0x31,0xEF,0x82,0xE4};
+
+  GetFileMetadataRequest(const GetFileMetadataRequest&);
+  GetFileMetadataRequest& operator=(const GetFileMetadataRequest&);
+  GetFileMetadataRequest() {
+  }
+
+  virtual ~GetFileMetadataRequest() throw();
+  std::vector<int64_t>  fileIds;
+
+  void __set_fileIds(const std::vector<int64_t> & val);
+
+  bool operator == (const GetFileMetadataRequest & rhs) const
+  {
+    if (!(fileIds == rhs.fileIds))
+      return false;
+    return true;
+  }
+  bool operator != (const GetFileMetadataRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetFileMetadataRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const GetFileMetadataRequest& obj);
+};
+
+void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b);
+
+
+class PutFileMetadataResult {
+ public:
+
+  static const char* ascii_fingerprint; // = "99914B932BD37A50B983C5E7C90AE93B";
+  static const uint8_t binary_fingerprint[16]; // = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B};
+
+  PutFileMetadataResult(const PutFileMetadataResult&);
+  PutFileMetadataResult& operator=(const PutFileMetadataResult&);
+  PutFileMetadataResult() {
+  }
+
+  virtual ~PutFileMetadataResult() throw();
+
+  bool operator == (const PutFileMetadataResult & /* rhs */) const
+  {
+    return true;
+  }
+  bool operator != (const PutFileMetadataResult &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const PutFileMetadataResult & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const PutFileMetadataResult& obj);
+};
+
+void swap(PutFileMetadataResult &a, PutFileMetadataResult &b);
+
+
+class PutFileMetadataRequest {
+ public:
+
+  static const char* ascii_fingerprint; // = "D64A208A8BCFCE146F4E2CB2176A807C";
+  static const uint8_t binary_fingerprint[16]; // = {0xD6,0x4A,0x20,0x8A,0x8B,0xCF,0xCE,0x14,0x6F,0x4E,0x2C,0xB2,0x17,0x6A,0x80,0x7C};
+
+  PutFileMetadataRequest(const PutFileMetadataRequest&);
+  PutFileMetadataRequest& operator=(const PutFileMetadataRequest&);
+  PutFileMetadataRequest() {
+  }
+
+  virtual ~PutFileMetadataRequest() throw();
+  std::vector<int64_t>  fileIds;
+  std::vector<std::string>  metadata;
+
+  void __set_fileIds(const std::vector<int64_t> & val);
+
+  void __set_metadata(const std::vector<std::string> & val);
+
+  bool operator == (const PutFileMetadataRequest & rhs) const
+  {
+    if (!(fileIds == rhs.fileIds))
+      return false;
+    if (!(metadata == rhs.metadata))
+      return false;
+    return true;
+  }
+  bool operator != (const PutFileMetadataRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const PutFileMetadataRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const PutFileMetadataRequest& obj);
+};
+
+void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b);
+
+
+class ClearFileMetadataResult {
+ public:
+
+  static const char* ascii_fingerprint; // = "99914B932BD37A50B983C5E7C90AE93B";
+  static const uint8_t binary_fingerprint[16]; // = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B};
+
+  ClearFileMetadataResult(const ClearFileMetadataResult&);
+  ClearFileMetadataResult& operator=(const ClearFileMetadataResult&);
+  ClearFileMetadataResult() {
+  }
+
+  virtual ~ClearFileMetadataResult() throw();
+
+  bool operator == (const ClearFileMetadataResult & /* rhs */) const
+  {
+    return true;
+  }
+  bool operator != (const ClearFileMetadataResult &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ClearFileMetadataResult & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ClearFileMetadataResult& obj);
+};
+
+void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b);
+
+
+class ClearFileMetadataRequest {
+ public:
+
+  static const char* ascii_fingerprint; // = "E49D7D1A9013CC81CD0F69D631EF82E4";
+  static const uint8_t binary_fingerprint[16]; // = {0xE4,0x9D,0x7D,0x1A,0x90,0x13,0xCC,0x81,0xCD,0x0F,0x69,0xD6,0x31,0xEF,0x82,0xE4};
+
+  ClearFileMetadataRequest(const ClearFileMetadataRequest&);
+  ClearFileMetadataRequest& operator=(const ClearFileMetadataRequest&);
+  ClearFileMetadataRequest() {
+  }
+
+  virtual ~ClearFileMetadataRequest() throw();
+  std::vector<int64_t>  fileIds;
+
+  void __set_fileIds(const std::vector<int64_t> & val);
+
+  bool operator == (const ClearFileMetadataRequest & rhs) const
+  {
+    if (!(fileIds == rhs.fileIds))
+      return false;
+    return true;
+  }
+  bool operator != (const ClearFileMetadataRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ClearFileMetadataRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ClearFileMetadataRequest& obj);
+};
+
+void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b);
+
 typedef struct _GetAllFunctionsResponse__isset {
   _GetAllFunctionsResponse__isset() : functions(false) {}
   bool functions :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java
index ae12142..73e0ffd 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class AbortTxnRequest implements org.apache.thrift.TBase<AbortTxnRequest, AbortTxnRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AbortTxnRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AbortTxnRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index f60521f..8652d47 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicPartitions, AddDynamicPartitions._Fields>, java.io.Serializable, Cloneable, Comparable<AddDynamicPartitions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddDynamicPartitions");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
index 00a7236..dde146d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitionsRequest, AddPartitionsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AddPartitionsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddPartitionsRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
index 7150e68..922aa42 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartitionsResult, AddPartitionsResult._Fields>, java.io.Serializable, Cloneable, Comparable<AddPartitionsResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddPartitionsResult");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
index bf14ac0..9dbc5c5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class AggrStats implements org.apache.thrift.TBase<AggrStats, AggrStats._Fields>, java.io.Serializable, Cloneable, Comparable<AggrStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AggrStats");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java
index 16f2cb7..2290762 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlreadyExistsException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class AlreadyExistsException extends TException implements org.apache.thrift.TBase<AlreadyExistsException, AlreadyExistsException._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExistsException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExistsException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java
index 7ddb91a..32b8916 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BinaryColumnStatsData.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class BinaryColumnStatsData implements org.apache.thrift.TBase<BinaryColumnStatsData, BinaryColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<BinaryColumnStatsData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BinaryColumnStatsData");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java
index f98e56b..c019753 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BooleanColumnStatsData.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class BooleanColumnStatsData implements org.apache.thrift.TBase<BooleanColumnStatsData, BooleanColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<BooleanColumnStatsData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BooleanColumnStatsData");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java
index 667d12e..1efa060 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class CheckLockRequest implements org.apache.thrift.TBase<CheckLockRequest, CheckLockRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CheckLockRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CheckLockRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
new file mode 100644
index 0000000..04408a6
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -0,0 +1,438 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFileMetadataRequest, ClearFileMetadataRequest._Fields>, java.io.Serializable, Cloneable, Comparable<ClearFileMetadataRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClearFileMetadataRequest");
+
+  private static final org.apache.thrift.protocol.TField FILE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("fileIds", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ClearFileMetadataRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ClearFileMetadataRequestTupleSchemeFactory());
+  }
+
+  private List<Long> fileIds; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    FILE_IDS((short)1, "fileIds");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FILE_IDS
+          return FILE_IDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FILE_IDS, new org.apache.thrift.meta_data.FieldMetaData("fileIds", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClearFileMetadataRequest.class, metaDataMap);
+  }
+
+  public ClearFileMetadataRequest() {
+  }
+
+  public ClearFileMetadataRequest(
+    List<Long> fileIds)
+  {
+    this();
+    this.fileIds = fileIds;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ClearFileMetadataRequest(ClearFileMetadataRequest other) {
+    if (other.isSetFileIds()) {
+      List<Long> __this__fileIds = new ArrayList<Long>(other.fileIds);
+      this.fileIds = __this__fileIds;
+    }
+  }
+
+  public ClearFileMetadataRequest deepCopy() {
+    return new ClearFileMetadataRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.fileIds = null;
+  }
+
+  public int getFileIdsSize() {
+    return (this.fileIds == null) ? 0 : this.fileIds.size();
+  }
+
+  public java.util.Iterator<Long> getFileIdsIterator() {
+    return (this.fileIds == null) ? null : this.fileIds.iterator();
+  }
+
+  public void addToFileIds(long elem) {
+    if (this.fileIds == null) {
+      this.fileIds = new ArrayList<Long>();
+    }
+    this.fileIds.add(elem);
+  }
+
+  public List<Long> getFileIds() {
+    return this.fileIds;
+  }
+
+  public void setFileIds(List<Long> fileIds) {
+    this.fileIds = fileIds;
+  }
+
+  public void unsetFileIds() {
+    this.fileIds = null;
+  }
+
+  /** Returns true if field fileIds is set (has been assigned a value) and false otherwise */
+  public boolean isSetFileIds() {
+    return this.fileIds != null;
+  }
+
+  public void setFileIdsIsSet(boolean value) {
+    if (!value) {
+      this.fileIds = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FILE_IDS:
+      if (value == null) {
+        unsetFileIds();
+      } else {
+        setFileIds((List<Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FILE_IDS:
+      return getFileIds();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case FILE_IDS:
+      return isSetFileIds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ClearFileMetadataRequest)
+      return this.equals((ClearFileMetadataRequest)that);
+    return false;
+  }
+
+  public boolean equals(ClearFileMetadataRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_fileIds = true && this.isSetFileIds();
+    boolean that_present_fileIds = true && that.isSetFileIds();
+    if (this_present_fileIds || that_present_fileIds) {
+      if (!(this_present_fileIds && that_present_fileIds))
+        return false;
+      if (!this.fileIds.equals(that.fileIds))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_fileIds = true && (isSetFileIds());
+    list.add(present_fileIds);
+    if (present_fileIds)
+      list.add(fileIds);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ClearFileMetadataRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetFileIds()).compareTo(other.isSetFileIds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFileIds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileIds, other.fileIds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ClearFileMetadataRequest(");
+    boolean first = true;
+
+    sb.append("fileIds:");
+    if (this.fileIds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fileIds);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetFileIds()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'fileIds' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ClearFileMetadataRequestStandardSchemeFactory implements SchemeFactory {
+    public ClearFileMetadataRequestStandardScheme getScheme() {
+      return new ClearFileMetadataRequestStandardScheme();
+    }
+  }
+
+  private static class ClearFileMetadataRequestStandardScheme extends StandardScheme<ClearFileMetadataRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // FILE_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list584 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list584.size);
+                long _elem585;
+                for (int _i586 = 0; _i586 < _list584.size; ++_i586)
+                {
+                  _elem585 = iprot.readI64();
+                  struct.fileIds.add(_elem585);
+                }
+                iprot.readListEnd();
+              }
+              struct.setFileIdsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.fileIds != null) {
+        oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
+          for (long _iter587 : struct.fileIds)
+          {
+            oprot.writeI64(_iter587);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ClearFileMetadataRequestTupleSchemeFactory implements SchemeFactory {
+    public ClearFileMetadataRequestTupleScheme getScheme() {
+      return new ClearFileMetadataRequestTupleScheme();
+    }
+  }
+
+  private static class ClearFileMetadataRequestTupleScheme extends TupleScheme<ClearFileMetadataRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.fileIds.size());
+        for (long _iter588 : struct.fileIds)
+        {
+          oprot.writeI64(_iter588);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list589 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list589.size);
+        long _elem590;
+        for (int _i591 = 0; _i591 < _list589.size; ++_i591)
+        {
+          _elem590 = iprot.readI64();
+          struct.fileIds.add(_elem590);
+        }
+      }
+      struct.setFileIdsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataResult.java
new file mode 100644
index 0000000..4d9dfb8
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataResult.java
@@ -0,0 +1,283 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class ClearFileMetadataResult implements org.apache.thrift.TBase<ClearFileMetadataResult, ClearFileMetadataResult._Fields>, java.io.Serializable, Cloneable, Comparable<ClearFileMetadataResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClearFileMetadataResult");
+
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ClearFileMetadataResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ClearFileMetadataResultTupleSchemeFactory());
+  }
+
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClearFileMetadataResult.class, metaDataMap);
+  }
+
+  public ClearFileMetadataResult() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ClearFileMetadataResult(ClearFileMetadataResult other) {
+  }
+
+  public ClearFileMetadataResult deepCopy() {
+    return new ClearFileMetadataResult(this);
+  }
+
+  @Override
+  public void clear() {
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ClearFileMetadataResult)
+      return this.equals((ClearFileMetadataResult)that);
+    return false;
+  }
+
+  public boolean equals(ClearFileMetadataResult that) {
+    if (that == null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ClearFileMetadataResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ClearFileMetadataResult(");
+    boolean first = true;
+
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ClearFileMetadataResultStandardSchemeFactory implements SchemeFactory {
+    public ClearFileMetadataResultStandardScheme getScheme() {
+      return new ClearFileMetadataResultStandardScheme();
+    }
+  }
+
+  private static class ClearFileMetadataResultStandardScheme extends StandardScheme<ClearFileMetadataResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ClearFileMetadataResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ClearFileMetadataResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ClearFileMetadataResultTupleSchemeFactory implements SchemeFactory {
+    public ClearFileMetadataResultTupleScheme getScheme() {
+      return new ClearFileMetadataResultTupleScheme();
+    }
+  }
+
+  private static class ClearFileMetadataResultTupleScheme extends TupleScheme<ClearFileMetadataResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
index dd9aeb7..55cfab0 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ColumnStatistics implements org.apache.thrift.TBase<ColumnStatistics, ColumnStatistics._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnStatistics> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnStatistics");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
index 09f925c..ad72c3d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ColumnStatisticsDesc implements org.apache.thrift.TBase<ColumnStatisticsDesc, ColumnStatisticsDesc._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnStatisticsDesc> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnStatisticsDesc");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java
index 2be715f..4fbe506 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsObj.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ColumnStatisticsObj implements org.apache.thrift.TBase<ColumnStatisticsObj, ColumnStatisticsObj._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnStatisticsObj> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnStatisticsObj");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
index 91483e2..93ff732 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class CommitTxnRequest implements org.apache.thrift.TBase<CommitTxnRequest, CommitTxnRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CommitTxnRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CommitTxnRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index c6ad69e..688706e 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequest, CompactionRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CompactionRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CompactionRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java
index 99e7a83..e92b6d6 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ConfigValSecurityException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class ConfigValSecurityException extends TException implements org.apache.thrift.TBase<ConfigValSecurityException, ConfigValSecurityException._Fields>, java.io.Serializable, Cloneable, Comparable<ConfigValSecurityException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ConfigValSecurityException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java
index 89abc78..a3acf64 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CurrentNotificationEventId.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class CurrentNotificationEventId implements org.apache.thrift.TBase<CurrentNotificationEventId, CurrentNotificationEventId._Fields>, java.io.Serializable, Cloneable, Comparable<CurrentNotificationEventId> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CurrentNotificationEventId");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
index 759eec9..35c63b6 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Database implements org.apache.thrift.TBase<Database, Database._Fields>, java.io.Serializable, Cloneable, Comparable<Database> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Database");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java
index b4a44a4..b762895 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Date.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Date implements org.apache.thrift.TBase<Date, Date._Fields>, java.io.Serializable, Cloneable, Comparable<Date> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Date");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java
index 7050334..e669ee8 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DateColumnStatsData.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class DateColumnStatsData implements org.apache.thrift.TBase<DateColumnStatsData, DateColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<DateColumnStatsData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DateColumnStatsData");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java
index 1f82543..e54c906 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Decimal.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Decimal implements org.apache.thrift.TBase<Decimal, Decimal._Fields>, java.io.Serializable, Cloneable, Comparable<Decimal> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Decimal");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java
index 02092dc..74bbe33 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DecimalColumnStatsData.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class DecimalColumnStatsData implements org.apache.thrift.TBase<DecimalColumnStatsData, DecimalColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<DecimalColumnStatsData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DecimalColumnStatsData");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java
index 52288e5..48a742f 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DoubleColumnStatsData.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class DoubleColumnStatsData implements org.apache.thrift.TBase<DoubleColumnStatsData, DoubleColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<DoubleColumnStatsData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DoubleColumnStatsData");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java
index 0d1e50d..2552cbd 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsExpr.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class DropPartitionsExpr implements org.apache.thrift.TBase<DropPartitionsExpr, DropPartitionsExpr._Fields>, java.io.Serializable, Cloneable, Comparable<DropPartitionsExpr> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DropPartitionsExpr");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
index 46cc9a7..f6c873a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class DropPartitionsRequest implements org.apache.thrift.TBase<DropPartitionsRequest, DropPartitionsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<DropPartitionsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DropPartitionsRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
index 09da136..697e1b8 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartitionsResult, DropPartitionsResult._Fields>, java.io.Serializable, Cloneable, Comparable<DropPartitionsResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DropPartitionsResult");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
index 3eabc86..9c80329 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class EnvironmentContext implements org.apache.thrift.TBase<EnvironmentContext, EnvironmentContext._Fields>, java.io.Serializable, Cloneable, Comparable<EnvironmentContext> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("EnvironmentContext");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
index e73edd4..de53201 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class FieldSchema implements org.apache.thrift.TBase<FieldSchema, FieldSchema._Fields>, java.io.Serializable, Cloneable, Comparable<FieldSchema> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FieldSchema");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 25f9d54..04b6f0a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class FireEventRequest implements org.apache.thrift.TBase<FireEventRequest, FireEventRequest._Fields>, java.io.Serializable, Cloneable, Comparable<FireEventRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FireEventRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java
index 6f277aa..c3234f2 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class FireEventResponse implements org.apache.thrift.TBase<FireEventResponse, FireEventResponse._Fields>, java.io.Serializable, Cloneable, Comparable<FireEventResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FireEventResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
index 33c617e..e6b847d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Function implements org.apache.thrift.TBase<Function, Function._Fields>, java.io.Serializable, Cloneable, Comparable<Function> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Function");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index 170d8e7..a98db18 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFunctionsResponse, GetAllFunctionsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetAllFunctionsResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetAllFunctionsResponse");
 
@@ -346,14 +346,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list524.size);
-                Function _elem525;
-                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
+                org.apache.thrift.protocol.TList _list592 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list592.size);
+                Function _elem593;
+                for (int _i594 = 0; _i594 < _list592.size; ++_i594)
                 {
-                  _elem525 = new Function();
-                  _elem525.read(iprot);
-                  struct.functions.add(_elem525);
+                  _elem593 = new Function();
+                  _elem593.read(iprot);
+                  struct.functions.add(_elem593);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter527 : struct.functions)
+            for (Function _iter595 : struct.functions)
             {
-              _iter527.write(oprot);
+              _iter595.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter528 : struct.functions)
+          for (Function _iter596 : struct.functions)
           {
-            _iter528.write(oprot);
+            _iter596.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list529.size);
-          Function _elem530;
-          for (int _i531 = 0; _i531 < _list529.size; ++_i531)
+          org.apache.thrift.protocol.TList _list597 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list597.size);
+          Function _elem598;
+          for (int _i599 = 0; _i599 < _list597.size; ++_i599)
           {
-            _elem530 = new Function();
-            _elem530.read(iprot);
-            struct.functions.add(_elem530);
+            _elem598 = new Function();
+            _elem598.read(iprot);
+            struct.functions.add(_elem598);
           }
         }
         struct.setFunctionsIsSet(true);


[34/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index baa28e3..49d31e6 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -141,6 +141,10 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_current_notificationEventId(CurrentNotificationEventId& _return) = 0;
   virtual void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) = 0;
   virtual void flushCache() = 0;
+  virtual void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) = 0;
+  virtual void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) = 0;
+  virtual void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) = 0;
+  virtual void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) = 0;
 };
 
 class ThriftHiveMetastoreIfFactory : virtual public  ::facebook::fb303::FacebookServiceIfFactory {
@@ -568,6 +572,18 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void flushCache() {
     return;
   }
+  void get_file_metadata_by_expr(GetFileMetadataByExprResult& /* _return */, const GetFileMetadataByExprRequest& /* req */) {
+    return;
+  }
+  void get_file_metadata(GetFileMetadataResult& /* _return */, const GetFileMetadataRequest& /* req */) {
+    return;
+  }
+  void put_file_metadata(PutFileMetadataResult& /* _return */, const PutFileMetadataRequest& /* req */) {
+    return;
+  }
+  void clear_file_metadata(ClearFileMetadataResult& /* _return */, const ClearFileMetadataRequest& /* req */) {
+    return;
+  }
 };
 
 typedef struct _ThriftHiveMetastore_getMetaConf_args__isset {
@@ -18287,6 +18303,486 @@ class ThriftHiveMetastore_flushCache_presult {
   friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_flushCache_presult& obj);
 };
 
+typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset {
+  _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset;
+
+class ThriftHiveMetastore_get_file_metadata_by_expr_args {
+ public:
+
+  static const char* ascii_fingerprint; // = "35F3A2DA650F5293300EA6DB58284F86";
+  static const uint8_t binary_fingerprint[16]; // = {0x35,0xF3,0xA2,0xDA,0x65,0x0F,0x52,0x93,0x30,0x0E,0xA6,0xDB,0x58,0x28,0x4F,0x86};
+
+  ThriftHiveMetastore_get_file_metadata_by_expr_args(const ThriftHiveMetastore_get_file_metadata_by_expr_args&);
+  ThriftHiveMetastore_get_file_metadata_by_expr_args& operator=(const ThriftHiveMetastore_get_file_metadata_by_expr_args&);
+  ThriftHiveMetastore_get_file_metadata_by_expr_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_args() throw();
+  GetFileMetadataByExprRequest req;
+
+  _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset __isset;
+
+  void __set_req(const GetFileMetadataByExprRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_file_metadata_by_expr_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_file_metadata_by_expr_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_file_metadata_by_expr_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_get_file_metadata_by_expr_args& obj);
+};
+
+
+class ThriftHiveMetastore_get_file_metadata_by_expr_pargs {
+ public:
+
+  static const char* ascii_fingerprint; // = "35F3A2DA650F5293300EA6DB58284F86";
+  static const uint8_t binary_fingerprint[16]; // = {0x35,0xF3,0xA2,0xDA,0x65,0x0F,0x52,0x93,0x30,0x0E,0xA6,0xDB,0x58,0x28,0x4F,0x86};
+
+
+  virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_pargs() throw();
+  const GetFileMetadataByExprRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_get_file_metadata_by_expr_pargs& obj);
+};
+
+typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset {
+  _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset;
+
+class ThriftHiveMetastore_get_file_metadata_by_expr_result {
+ public:
+
+  static const char* ascii_fingerprint; // = "E2053E1FBA55841322D49B2FBE16E310";
+  static const uint8_t binary_fingerprint[16]; // = {0xE2,0x05,0x3E,0x1F,0xBA,0x55,0x84,0x13,0x22,0xD4,0x9B,0x2F,0xBE,0x16,0xE3,0x10};
+
+  ThriftHiveMetastore_get_file_metadata_by_expr_result(const ThriftHiveMetastore_get_file_metadata_by_expr_result&);
+  ThriftHiveMetastore_get_file_metadata_by_expr_result& operator=(const ThriftHiveMetastore_get_file_metadata_by_expr_result&);
+  ThriftHiveMetastore_get_file_metadata_by_expr_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_result() throw();
+  GetFileMetadataByExprResult success;
+
+  _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset __isset;
+
+  void __set_success(const GetFileMetadataByExprResult& val);
+
+  bool operator == (const ThriftHiveMetastore_get_file_metadata_by_expr_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_file_metadata_by_expr_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_file_metadata_by_expr_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_get_file_metadata_by_expr_result& obj);
+};
+
+typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset {
+  _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset;
+
+class ThriftHiveMetastore_get_file_metadata_by_expr_presult {
+ public:
+
+  static const char* ascii_fingerprint; // = "E2053E1FBA55841322D49B2FBE16E310";
+  static const uint8_t binary_fingerprint[16]; // = {0xE2,0x05,0x3E,0x1F,0xBA,0x55,0x84,0x13,0x22,0xD4,0x9B,0x2F,0xBE,0x16,0xE3,0x10};
+
+
+  virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_presult() throw();
+  GetFileMetadataByExprResult* success;
+
+  _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_get_file_metadata_by_expr_presult& obj);
+};
+
+typedef struct _ThriftHiveMetastore_get_file_metadata_args__isset {
+  _ThriftHiveMetastore_get_file_metadata_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_get_file_metadata_args__isset;
+
+class ThriftHiveMetastore_get_file_metadata_args {
+ public:
+
+  static const char* ascii_fingerprint; // = "1BC0267F37F033AE9EEA2EB33C70733E";
+  static const uint8_t binary_fingerprint[16]; // = {0x1B,0xC0,0x26,0x7F,0x37,0xF0,0x33,0xAE,0x9E,0xEA,0x2E,0xB3,0x3C,0x70,0x73,0x3E};
+
+  ThriftHiveMetastore_get_file_metadata_args(const ThriftHiveMetastore_get_file_metadata_args&);
+  ThriftHiveMetastore_get_file_metadata_args& operator=(const ThriftHiveMetastore_get_file_metadata_args&);
+  ThriftHiveMetastore_get_file_metadata_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_file_metadata_args() throw();
+  GetFileMetadataRequest req;
+
+  _ThriftHiveMetastore_get_file_metadata_args__isset __isset;
+
+  void __set_req(const GetFileMetadataRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_file_metadata_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_file_metadata_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_file_metadata_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_get_file_metadata_args& obj);
+};
+
+
+class ThriftHiveMetastore_get_file_metadata_pargs {
+ public:
+
+  static const char* ascii_fingerprint; // = "1BC0267F37F033AE9EEA2EB33C70733E";
+  static const uint8_t binary_fingerprint[16]; // = {0x1B,0xC0,0x26,0x7F,0x37,0xF0,0x33,0xAE,0x9E,0xEA,0x2E,0xB3,0x3C,0x70,0x73,0x3E};
+
+
+  virtual ~ThriftHiveMetastore_get_file_metadata_pargs() throw();
+  const GetFileMetadataRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_get_file_metadata_pargs& obj);
+};
+
+typedef struct _ThriftHiveMetastore_get_file_metadata_result__isset {
+  _ThriftHiveMetastore_get_file_metadata_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_get_file_metadata_result__isset;
+
+class ThriftHiveMetastore_get_file_metadata_result {
+ public:
+
+  static const char* ascii_fingerprint; // = "2E68003888122322D29FD7969DAE8C2A";
+  static const uint8_t binary_fingerprint[16]; // = {0x2E,0x68,0x00,0x38,0x88,0x12,0x23,0x22,0xD2,0x9F,0xD7,0x96,0x9D,0xAE,0x8C,0x2A};
+
+  ThriftHiveMetastore_get_file_metadata_result(const ThriftHiveMetastore_get_file_metadata_result&);
+  ThriftHiveMetastore_get_file_metadata_result& operator=(const ThriftHiveMetastore_get_file_metadata_result&);
+  ThriftHiveMetastore_get_file_metadata_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_file_metadata_result() throw();
+  GetFileMetadataResult success;
+
+  _ThriftHiveMetastore_get_file_metadata_result__isset __isset;
+
+  void __set_success(const GetFileMetadataResult& val);
+
+  bool operator == (const ThriftHiveMetastore_get_file_metadata_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_file_metadata_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_file_metadata_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_get_file_metadata_result& obj);
+};
+
+typedef struct _ThriftHiveMetastore_get_file_metadata_presult__isset {
+  _ThriftHiveMetastore_get_file_metadata_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_get_file_metadata_presult__isset;
+
+class ThriftHiveMetastore_get_file_metadata_presult {
+ public:
+
+  static const char* ascii_fingerprint; // = "2E68003888122322D29FD7969DAE8C2A";
+  static const uint8_t binary_fingerprint[16]; // = {0x2E,0x68,0x00,0x38,0x88,0x12,0x23,0x22,0xD2,0x9F,0xD7,0x96,0x9D,0xAE,0x8C,0x2A};
+
+
+  virtual ~ThriftHiveMetastore_get_file_metadata_presult() throw();
+  GetFileMetadataResult* success;
+
+  _ThriftHiveMetastore_get_file_metadata_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_get_file_metadata_presult& obj);
+};
+
+typedef struct _ThriftHiveMetastore_put_file_metadata_args__isset {
+  _ThriftHiveMetastore_put_file_metadata_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_put_file_metadata_args__isset;
+
+class ThriftHiveMetastore_put_file_metadata_args {
+ public:
+
+  static const char* ascii_fingerprint; // = "CC8035DE851F68540899C9E7BDA51238";
+  static const uint8_t binary_fingerprint[16]; // = {0xCC,0x80,0x35,0xDE,0x85,0x1F,0x68,0x54,0x08,0x99,0xC9,0xE7,0xBD,0xA5,0x12,0x38};
+
+  ThriftHiveMetastore_put_file_metadata_args(const ThriftHiveMetastore_put_file_metadata_args&);
+  ThriftHiveMetastore_put_file_metadata_args& operator=(const ThriftHiveMetastore_put_file_metadata_args&);
+  ThriftHiveMetastore_put_file_metadata_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_put_file_metadata_args() throw();
+  PutFileMetadataRequest req;
+
+  _ThriftHiveMetastore_put_file_metadata_args__isset __isset;
+
+  void __set_req(const PutFileMetadataRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_put_file_metadata_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_put_file_metadata_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_put_file_metadata_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_put_file_metadata_args& obj);
+};
+
+
+class ThriftHiveMetastore_put_file_metadata_pargs {
+ public:
+
+  static const char* ascii_fingerprint; // = "CC8035DE851F68540899C9E7BDA51238";
+  static const uint8_t binary_fingerprint[16]; // = {0xCC,0x80,0x35,0xDE,0x85,0x1F,0x68,0x54,0x08,0x99,0xC9,0xE7,0xBD,0xA5,0x12,0x38};
+
+
+  virtual ~ThriftHiveMetastore_put_file_metadata_pargs() throw();
+  const PutFileMetadataRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_put_file_metadata_pargs& obj);
+};
+
+typedef struct _ThriftHiveMetastore_put_file_metadata_result__isset {
+  _ThriftHiveMetastore_put_file_metadata_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_put_file_metadata_result__isset;
+
+class ThriftHiveMetastore_put_file_metadata_result {
+ public:
+
+  static const char* ascii_fingerprint; // = "3A26B8DD823AB72F2FE94404F3D36070";
+  static const uint8_t binary_fingerprint[16]; // = {0x3A,0x26,0xB8,0xDD,0x82,0x3A,0xB7,0x2F,0x2F,0xE9,0x44,0x04,0xF3,0xD3,0x60,0x70};
+
+  ThriftHiveMetastore_put_file_metadata_result(const ThriftHiveMetastore_put_file_metadata_result&);
+  ThriftHiveMetastore_put_file_metadata_result& operator=(const ThriftHiveMetastore_put_file_metadata_result&);
+  ThriftHiveMetastore_put_file_metadata_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_put_file_metadata_result() throw();
+  PutFileMetadataResult success;
+
+  _ThriftHiveMetastore_put_file_metadata_result__isset __isset;
+
+  void __set_success(const PutFileMetadataResult& val);
+
+  bool operator == (const ThriftHiveMetastore_put_file_metadata_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_put_file_metadata_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_put_file_metadata_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_put_file_metadata_result& obj);
+};
+
+typedef struct _ThriftHiveMetastore_put_file_metadata_presult__isset {
+  _ThriftHiveMetastore_put_file_metadata_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_put_file_metadata_presult__isset;
+
+class ThriftHiveMetastore_put_file_metadata_presult {
+ public:
+
+  static const char* ascii_fingerprint; // = "3A26B8DD823AB72F2FE94404F3D36070";
+  static const uint8_t binary_fingerprint[16]; // = {0x3A,0x26,0xB8,0xDD,0x82,0x3A,0xB7,0x2F,0x2F,0xE9,0x44,0x04,0xF3,0xD3,0x60,0x70};
+
+
+  virtual ~ThriftHiveMetastore_put_file_metadata_presult() throw();
+  PutFileMetadataResult* success;
+
+  _ThriftHiveMetastore_put_file_metadata_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_put_file_metadata_presult& obj);
+};
+
+typedef struct _ThriftHiveMetastore_clear_file_metadata_args__isset {
+  _ThriftHiveMetastore_clear_file_metadata_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_clear_file_metadata_args__isset;
+
+class ThriftHiveMetastore_clear_file_metadata_args {
+ public:
+
+  static const char* ascii_fingerprint; // = "1BC0267F37F033AE9EEA2EB33C70733E";
+  static const uint8_t binary_fingerprint[16]; // = {0x1B,0xC0,0x26,0x7F,0x37,0xF0,0x33,0xAE,0x9E,0xEA,0x2E,0xB3,0x3C,0x70,0x73,0x3E};
+
+  ThriftHiveMetastore_clear_file_metadata_args(const ThriftHiveMetastore_clear_file_metadata_args&);
+  ThriftHiveMetastore_clear_file_metadata_args& operator=(const ThriftHiveMetastore_clear_file_metadata_args&);
+  ThriftHiveMetastore_clear_file_metadata_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_clear_file_metadata_args() throw();
+  ClearFileMetadataRequest req;
+
+  _ThriftHiveMetastore_clear_file_metadata_args__isset __isset;
+
+  void __set_req(const ClearFileMetadataRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_clear_file_metadata_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_clear_file_metadata_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_clear_file_metadata_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_clear_file_metadata_args& obj);
+};
+
+
+class ThriftHiveMetastore_clear_file_metadata_pargs {
+ public:
+
+  static const char* ascii_fingerprint; // = "1BC0267F37F033AE9EEA2EB33C70733E";
+  static const uint8_t binary_fingerprint[16]; // = {0x1B,0xC0,0x26,0x7F,0x37,0xF0,0x33,0xAE,0x9E,0xEA,0x2E,0xB3,0x3C,0x70,0x73,0x3E};
+
+
+  virtual ~ThriftHiveMetastore_clear_file_metadata_pargs() throw();
+  const ClearFileMetadataRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_clear_file_metadata_pargs& obj);
+};
+
+typedef struct _ThriftHiveMetastore_clear_file_metadata_result__isset {
+  _ThriftHiveMetastore_clear_file_metadata_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_clear_file_metadata_result__isset;
+
+class ThriftHiveMetastore_clear_file_metadata_result {
+ public:
+
+  static const char* ascii_fingerprint; // = "3A26B8DD823AB72F2FE94404F3D36070";
+  static const uint8_t binary_fingerprint[16]; // = {0x3A,0x26,0xB8,0xDD,0x82,0x3A,0xB7,0x2F,0x2F,0xE9,0x44,0x04,0xF3,0xD3,0x60,0x70};
+
+  ThriftHiveMetastore_clear_file_metadata_result(const ThriftHiveMetastore_clear_file_metadata_result&);
+  ThriftHiveMetastore_clear_file_metadata_result& operator=(const ThriftHiveMetastore_clear_file_metadata_result&);
+  ThriftHiveMetastore_clear_file_metadata_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_clear_file_metadata_result() throw();
+  ClearFileMetadataResult success;
+
+  _ThriftHiveMetastore_clear_file_metadata_result__isset __isset;
+
+  void __set_success(const ClearFileMetadataResult& val);
+
+  bool operator == (const ThriftHiveMetastore_clear_file_metadata_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_clear_file_metadata_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_clear_file_metadata_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_clear_file_metadata_result& obj);
+};
+
+typedef struct _ThriftHiveMetastore_clear_file_metadata_presult__isset {
+  _ThriftHiveMetastore_clear_file_metadata_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_clear_file_metadata_presult__isset;
+
+class ThriftHiveMetastore_clear_file_metadata_presult {
+ public:
+
+  static const char* ascii_fingerprint; // = "3A26B8DD823AB72F2FE94404F3D36070";
+  static const uint8_t binary_fingerprint[16]; // = {0x3A,0x26,0xB8,0xDD,0x82,0x3A,0xB7,0x2F,0x2F,0xE9,0x44,0x04,0xF3,0xD3,0x60,0x70};
+
+
+  virtual ~ThriftHiveMetastore_clear_file_metadata_presult() throw();
+  ClearFileMetadataResult* success;
+
+  _ThriftHiveMetastore_clear_file_metadata_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+  friend std::ostream& operator<<(std::ostream& out, const ThriftHiveMetastore_clear_file_metadata_presult& obj);
+};
+
 class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public  ::facebook::fb303::FacebookServiceClient {
  public:
   ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :
@@ -18673,6 +19169,18 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void flushCache();
   void send_flushCache();
   void recv_flushCache();
+  void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req);
+  void send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req);
+  void recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return);
+  void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req);
+  void send_get_file_metadata(const GetFileMetadataRequest& req);
+  void recv_get_file_metadata(GetFileMetadataResult& _return);
+  void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req);
+  void send_put_file_metadata(const PutFileMetadataRequest& req);
+  void recv_put_file_metadata(PutFileMetadataResult& _return);
+  void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req);
+  void send_clear_file_metadata(const ClearFileMetadataRequest& req);
+  void recv_clear_file_metadata(ClearFileMetadataResult& _return);
 };
 
 class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceProcessor {
@@ -18808,6 +19316,10 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_current_notificationEventId(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_fire_listener_event(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_flushCache(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_file_metadata_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_put_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_clear_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
  public:
   ThriftHiveMetastoreProcessor(boost::shared_ptr<ThriftHiveMetastoreIf> iface) :
      ::facebook::fb303::FacebookServiceProcessor(iface),
@@ -18937,6 +19449,10 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_current_notificationEventId"] = &ThriftHiveMetastoreProcessor::process_get_current_notificationEventId;
     processMap_["fire_listener_event"] = &ThriftHiveMetastoreProcessor::process_fire_listener_event;
     processMap_["flushCache"] = &ThriftHiveMetastoreProcessor::process_flushCache;
+    processMap_["get_file_metadata_by_expr"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata_by_expr;
+    processMap_["get_file_metadata"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata;
+    processMap_["put_file_metadata"] = &ThriftHiveMetastoreProcessor::process_put_file_metadata;
+    processMap_["clear_file_metadata"] = &ThriftHiveMetastoreProcessor::process_clear_file_metadata;
   }
 
   virtual ~ThriftHiveMetastoreProcessor() {}
@@ -20169,6 +20685,46 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->flushCache();
   }
 
+  void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_file_metadata_by_expr(_return, req);
+    }
+    ifaces_[i]->get_file_metadata_by_expr(_return, req);
+    return;
+  }
+
+  void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_file_metadata(_return, req);
+    }
+    ifaces_[i]->get_file_metadata(_return, req);
+    return;
+  }
+
+  void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->put_file_metadata(_return, req);
+    }
+    ifaces_[i]->put_file_metadata(_return, req);
+    return;
+  }
+
+  void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->clear_file_metadata(_return, req);
+    }
+    ifaces_[i]->clear_file_metadata(_return, req);
+    return;
+  }
+
 };
 
 }}} // namespace

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 339bc7d..9eca65c 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -647,6 +647,26 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("flushCache\n");
   }
 
+  void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) {
+    // Your implementation goes here
+    printf("get_file_metadata_by_expr\n");
+  }
+
+  void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) {
+    // Your implementation goes here
+    printf("get_file_metadata\n");
+  }
+
+  void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) {
+    // Your implementation goes here
+    printf("put_file_metadata\n");
+  }
+
+  void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) {
+    // Your implementation goes here
+    printf("clear_file_metadata\n");
+  }
+
 };
 
 int main(int argc, char **argv) {

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 57c3477..e61ba1b 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -14373,6 +14373,1076 @@ std::ostream& operator<<(std::ostream& out, const FireEventResponse& obj) {
 }
 
 
+MetadataPpdResult::~MetadataPpdResult() throw() {
+}
+
+
+void MetadataPpdResult::__set_metadata(const std::string& val) {
+  this->metadata = val;
+}
+
+void MetadataPpdResult::__set_includeBitset(const std::string& val) {
+  this->includeBitset = val;
+}
+
+const char* MetadataPpdResult::ascii_fingerprint = "07A9615F837F7D0A952B595DD3020972";
+const uint8_t MetadataPpdResult::binary_fingerprint[16] = {0x07,0xA9,0x61,0x5F,0x83,0x7F,0x7D,0x0A,0x95,0x2B,0x59,0x5D,0xD3,0x02,0x09,0x72};
+
+uint32_t MetadataPpdResult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_metadata = false;
+  bool isset_includeBitset = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->metadata);
+          isset_metadata = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->includeBitset);
+          isset_includeBitset = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_metadata)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_includeBitset)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t MetadataPpdResult::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("MetadataPpdResult");
+
+  xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeBinary(this->metadata);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("includeBitset", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->includeBitset);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(MetadataPpdResult &a, MetadataPpdResult &b) {
+  using ::std::swap;
+  swap(a.metadata, b.metadata);
+  swap(a.includeBitset, b.includeBitset);
+}
+
+MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other609) {
+  metadata = other609.metadata;
+  includeBitset = other609.includeBitset;
+}
+MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other610) {
+  metadata = other610.metadata;
+  includeBitset = other610.includeBitset;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const MetadataPpdResult& obj) {
+  using apache::thrift::to_string;
+  out << "MetadataPpdResult(";
+  out << "metadata=" << to_string(obj.metadata);
+  out << ", " << "includeBitset=" << to_string(obj.includeBitset);
+  out << ")";
+  return out;
+}
+
+
+GetFileMetadataByExprResult::~GetFileMetadataByExprResult() throw() {
+}
+
+
+void GetFileMetadataByExprResult::__set_metadata(const std::map<int64_t, MetadataPpdResult> & val) {
+  this->metadata = val;
+}
+
+void GetFileMetadataByExprResult::__set_isSupported(const bool val) {
+  this->isSupported = val;
+}
+
+void GetFileMetadataByExprResult::__set_unknownFileIds(const std::vector<int64_t> & val) {
+  this->unknownFileIds = val;
+}
+
+const char* GetFileMetadataByExprResult::ascii_fingerprint = "2B0C1B8D7599529A5797481BE308375D";
+const uint8_t GetFileMetadataByExprResult::binary_fingerprint[16] = {0x2B,0x0C,0x1B,0x8D,0x75,0x99,0x52,0x9A,0x57,0x97,0x48,0x1B,0xE3,0x08,0x37,0x5D};
+
+uint32_t GetFileMetadataByExprResult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_metadata = false;
+  bool isset_isSupported = false;
+  bool isset_unknownFileIds = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_MAP) {
+          {
+            this->metadata.clear();
+            uint32_t _size611;
+            ::apache::thrift::protocol::TType _ktype612;
+            ::apache::thrift::protocol::TType _vtype613;
+            xfer += iprot->readMapBegin(_ktype612, _vtype613, _size611);
+            uint32_t _i615;
+            for (_i615 = 0; _i615 < _size611; ++_i615)
+            {
+              int64_t _key616;
+              xfer += iprot->readI64(_key616);
+              MetadataPpdResult& _val617 = this->metadata[_key616];
+              xfer += _val617.read(iprot);
+            }
+            xfer += iprot->readMapEnd();
+          }
+          isset_metadata = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->isSupported);
+          isset_isSupported = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->unknownFileIds.clear();
+            uint32_t _size618;
+            ::apache::thrift::protocol::TType _etype621;
+            xfer += iprot->readListBegin(_etype621, _size618);
+            this->unknownFileIds.resize(_size618);
+            uint32_t _i622;
+            for (_i622 = 0; _i622 < _size618; ++_i622)
+            {
+              xfer += iprot->readI64(this->unknownFileIds[_i622]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_unknownFileIds = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_metadata)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_isSupported)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_unknownFileIds)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t GetFileMetadataByExprResult::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("GetFileMetadataByExprResult");
+
+  xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1);
+  {
+    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->metadata.size()));
+    std::map<int64_t, MetadataPpdResult> ::const_iterator _iter623;
+    for (_iter623 = this->metadata.begin(); _iter623 != this->metadata.end(); ++_iter623)
+    {
+      xfer += oprot->writeI64(_iter623->first);
+      xfer += _iter623->second.write(oprot);
+    }
+    xfer += oprot->writeMapEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 2);
+  xfer += oprot->writeBool(this->isSupported);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("unknownFileIds", ::apache::thrift::protocol::T_LIST, 3);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->unknownFileIds.size()));
+    std::vector<int64_t> ::const_iterator _iter624;
+    for (_iter624 = this->unknownFileIds.begin(); _iter624 != this->unknownFileIds.end(); ++_iter624)
+    {
+      xfer += oprot->writeI64((*_iter624));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b) {
+  using ::std::swap;
+  swap(a.metadata, b.metadata);
+  swap(a.isSupported, b.isSupported);
+  swap(a.unknownFileIds, b.unknownFileIds);
+}
+
+GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other625) {
+  metadata = other625.metadata;
+  isSupported = other625.isSupported;
+  unknownFileIds = other625.unknownFileIds;
+}
+GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other626) {
+  metadata = other626.metadata;
+  isSupported = other626.isSupported;
+  unknownFileIds = other626.unknownFileIds;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprResult& obj) {
+  using apache::thrift::to_string;
+  out << "GetFileMetadataByExprResult(";
+  out << "metadata=" << to_string(obj.metadata);
+  out << ", " << "isSupported=" << to_string(obj.isSupported);
+  out << ", " << "unknownFileIds=" << to_string(obj.unknownFileIds);
+  out << ")";
+  return out;
+}
+
+
+GetFileMetadataByExprRequest::~GetFileMetadataByExprRequest() throw() {
+}
+
+
+void GetFileMetadataByExprRequest::__set_fileIds(const std::vector<int64_t> & val) {
+  this->fileIds = val;
+}
+
+void GetFileMetadataByExprRequest::__set_expr(const std::string& val) {
+  this->expr = val;
+}
+
+const char* GetFileMetadataByExprRequest::ascii_fingerprint = "925353917FC0AF87976A2338011F5A31";
+const uint8_t GetFileMetadataByExprRequest::binary_fingerprint[16] = {0x92,0x53,0x53,0x91,0x7F,0xC0,0xAF,0x87,0x97,0x6A,0x23,0x38,0x01,0x1F,0x5A,0x31};
+
+uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_fileIds = false;
+  bool isset_expr = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->fileIds.clear();
+            uint32_t _size627;
+            ::apache::thrift::protocol::TType _etype630;
+            xfer += iprot->readListBegin(_etype630, _size627);
+            this->fileIds.resize(_size627);
+            uint32_t _i631;
+            for (_i631 = 0; _i631 < _size627; ++_i631)
+            {
+              xfer += iprot->readI64(this->fileIds[_i631]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_fileIds = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->expr);
+          isset_expr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_fileIds)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_expr)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t GetFileMetadataByExprRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("GetFileMetadataByExprRequest");
+
+  xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
+    std::vector<int64_t> ::const_iterator _iter632;
+    for (_iter632 = this->fileIds.begin(); _iter632 != this->fileIds.end(); ++_iter632)
+    {
+      xfer += oprot->writeI64((*_iter632));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("expr", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->expr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b) {
+  using ::std::swap;
+  swap(a.fileIds, b.fileIds);
+  swap(a.expr, b.expr);
+}
+
+GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other633) {
+  fileIds = other633.fileIds;
+  expr = other633.expr;
+}
+GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other634) {
+  fileIds = other634.fileIds;
+  expr = other634.expr;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprRequest& obj) {
+  using apache::thrift::to_string;
+  out << "GetFileMetadataByExprRequest(";
+  out << "fileIds=" << to_string(obj.fileIds);
+  out << ", " << "expr=" << to_string(obj.expr);
+  out << ")";
+  return out;
+}
+
+
+GetFileMetadataResult::~GetFileMetadataResult() throw() {
+}
+
+
+void GetFileMetadataResult::__set_metadata(const std::map<int64_t, std::string> & val) {
+  this->metadata = val;
+}
+
+void GetFileMetadataResult::__set_isSupported(const bool val) {
+  this->isSupported = val;
+}
+
+const char* GetFileMetadataResult::ascii_fingerprint = "D18BCBD4BA945E7F6500F5CD95205706";
+const uint8_t GetFileMetadataResult::binary_fingerprint[16] = {0xD1,0x8B,0xCB,0xD4,0xBA,0x94,0x5E,0x7F,0x65,0x00,0xF5,0xCD,0x95,0x20,0x57,0x06};
+
+uint32_t GetFileMetadataResult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_metadata = false;
+  bool isset_isSupported = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_MAP) {
+          {
+            this->metadata.clear();
+            uint32_t _size635;
+            ::apache::thrift::protocol::TType _ktype636;
+            ::apache::thrift::protocol::TType _vtype637;
+            xfer += iprot->readMapBegin(_ktype636, _vtype637, _size635);
+            uint32_t _i639;
+            for (_i639 = 0; _i639 < _size635; ++_i639)
+            {
+              int64_t _key640;
+              xfer += iprot->readI64(_key640);
+              std::string& _val641 = this->metadata[_key640];
+              xfer += iprot->readBinary(_val641);
+            }
+            xfer += iprot->readMapEnd();
+          }
+          isset_metadata = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->isSupported);
+          isset_isSupported = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_metadata)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_isSupported)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t GetFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("GetFileMetadataResult");
+
+  xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1);
+  {
+    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->metadata.size()));
+    std::map<int64_t, std::string> ::const_iterator _iter642;
+    for (_iter642 = this->metadata.begin(); _iter642 != this->metadata.end(); ++_iter642)
+    {
+      xfer += oprot->writeI64(_iter642->first);
+      xfer += oprot->writeBinary(_iter642->second);
+    }
+    xfer += oprot->writeMapEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 2);
+  xfer += oprot->writeBool(this->isSupported);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(GetFileMetadataResult &a, GetFileMetadataResult &b) {
+  using ::std::swap;
+  swap(a.metadata, b.metadata);
+  swap(a.isSupported, b.isSupported);
+}
+
+GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other643) {
+  metadata = other643.metadata;
+  isSupported = other643.isSupported;
+}
+GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other644) {
+  metadata = other644.metadata;
+  isSupported = other644.isSupported;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const GetFileMetadataResult& obj) {
+  using apache::thrift::to_string;
+  out << "GetFileMetadataResult(";
+  out << "metadata=" << to_string(obj.metadata);
+  out << ", " << "isSupported=" << to_string(obj.isSupported);
+  out << ")";
+  return out;
+}
+
+
+GetFileMetadataRequest::~GetFileMetadataRequest() throw() {
+}
+
+
+void GetFileMetadataRequest::__set_fileIds(const std::vector<int64_t> & val) {
+  this->fileIds = val;
+}
+
+const char* GetFileMetadataRequest::ascii_fingerprint = "E49D7D1A9013CC81CD0F69D631EF82E4";
+const uint8_t GetFileMetadataRequest::binary_fingerprint[16] = {0xE4,0x9D,0x7D,0x1A,0x90,0x13,0xCC,0x81,0xCD,0x0F,0x69,0xD6,0x31,0xEF,0x82,0xE4};
+
+uint32_t GetFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_fileIds = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->fileIds.clear();
+            uint32_t _size645;
+            ::apache::thrift::protocol::TType _etype648;
+            xfer += iprot->readListBegin(_etype648, _size645);
+            this->fileIds.resize(_size645);
+            uint32_t _i649;
+            for (_i649 = 0; _i649 < _size645; ++_i649)
+            {
+              xfer += iprot->readI64(this->fileIds[_i649]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_fileIds = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_fileIds)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t GetFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("GetFileMetadataRequest");
+
+  xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
+    std::vector<int64_t> ::const_iterator _iter650;
+    for (_iter650 = this->fileIds.begin(); _iter650 != this->fileIds.end(); ++_iter650)
+    {
+      xfer += oprot->writeI64((*_iter650));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b) {
+  using ::std::swap;
+  swap(a.fileIds, b.fileIds);
+}
+
+GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other651) {
+  fileIds = other651.fileIds;
+}
+GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other652) {
+  fileIds = other652.fileIds;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const GetFileMetadataRequest& obj) {
+  using apache::thrift::to_string;
+  out << "GetFileMetadataRequest(";
+  out << "fileIds=" << to_string(obj.fileIds);
+  out << ")";
+  return out;
+}
+
+
+PutFileMetadataResult::~PutFileMetadataResult() throw() {
+}
+
+
+const char* PutFileMetadataResult::ascii_fingerprint = "99914B932BD37A50B983C5E7C90AE93B";
+const uint8_t PutFileMetadataResult::binary_fingerprint[16] = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B};
+
+uint32_t PutFileMetadataResult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    xfer += iprot->skip(ftype);
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t PutFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("PutFileMetadataResult");
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(PutFileMetadataResult &a, PutFileMetadataResult &b) {
+  using ::std::swap;
+  (void) a;
+  (void) b;
+}
+
+PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other653) {
+  (void) other653;
+}
+PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other654) {
+  (void) other654;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const PutFileMetadataResult& obj) {
+  using apache::thrift::to_string;
+  (void) obj;
+  out << "PutFileMetadataResult(";
+  out << ")";
+  return out;
+}
+
+
+PutFileMetadataRequest::~PutFileMetadataRequest() throw() {
+}
+
+
+void PutFileMetadataRequest::__set_fileIds(const std::vector<int64_t> & val) {
+  this->fileIds = val;
+}
+
+void PutFileMetadataRequest::__set_metadata(const std::vector<std::string> & val) {
+  this->metadata = val;
+}
+
+const char* PutFileMetadataRequest::ascii_fingerprint = "D64A208A8BCFCE146F4E2CB2176A807C";
+const uint8_t PutFileMetadataRequest::binary_fingerprint[16] = {0xD6,0x4A,0x20,0x8A,0x8B,0xCF,0xCE,0x14,0x6F,0x4E,0x2C,0xB2,0x17,0x6A,0x80,0x7C};
+
+uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_fileIds = false;
+  bool isset_metadata = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->fileIds.clear();
+            uint32_t _size655;
+            ::apache::thrift::protocol::TType _etype658;
+            xfer += iprot->readListBegin(_etype658, _size655);
+            this->fileIds.resize(_size655);
+            uint32_t _i659;
+            for (_i659 = 0; _i659 < _size655; ++_i659)
+            {
+              xfer += iprot->readI64(this->fileIds[_i659]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_fileIds = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->metadata.clear();
+            uint32_t _size660;
+            ::apache::thrift::protocol::TType _etype663;
+            xfer += iprot->readListBegin(_etype663, _size660);
+            this->metadata.resize(_size660);
+            uint32_t _i664;
+            for (_i664 = 0; _i664 < _size660; ++_i664)
+            {
+              xfer += iprot->readBinary(this->metadata[_i664]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_metadata = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_fileIds)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_metadata)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("PutFileMetadataRequest");
+
+  xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
+    std::vector<int64_t> ::const_iterator _iter665;
+    for (_iter665 = this->fileIds.begin(); _iter665 != this->fileIds.end(); ++_iter665)
+    {
+      xfer += oprot->writeI64((*_iter665));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_LIST, 2);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->metadata.size()));
+    std::vector<std::string> ::const_iterator _iter666;
+    for (_iter666 = this->metadata.begin(); _iter666 != this->metadata.end(); ++_iter666)
+    {
+      xfer += oprot->writeBinary((*_iter666));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b) {
+  using ::std::swap;
+  swap(a.fileIds, b.fileIds);
+  swap(a.metadata, b.metadata);
+}
+
+PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other667) {
+  fileIds = other667.fileIds;
+  metadata = other667.metadata;
+}
+PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other668) {
+  fileIds = other668.fileIds;
+  metadata = other668.metadata;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const PutFileMetadataRequest& obj) {
+  using apache::thrift::to_string;
+  out << "PutFileMetadataRequest(";
+  out << "fileIds=" << to_string(obj.fileIds);
+  out << ", " << "metadata=" << to_string(obj.metadata);
+  out << ")";
+  return out;
+}
+
+
+ClearFileMetadataResult::~ClearFileMetadataResult() throw() {
+}
+
+
+const char* ClearFileMetadataResult::ascii_fingerprint = "99914B932BD37A50B983C5E7C90AE93B";
+const uint8_t ClearFileMetadataResult::binary_fingerprint[16] = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B};
+
+uint32_t ClearFileMetadataResult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    xfer += iprot->skip(ftype);
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ClearFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("ClearFileMetadataResult");
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b) {
+  using ::std::swap;
+  (void) a;
+  (void) b;
+}
+
+ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other669) {
+  (void) other669;
+}
+ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other670) {
+  (void) other670;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const ClearFileMetadataResult& obj) {
+  using apache::thrift::to_string;
+  (void) obj;
+  out << "ClearFileMetadataResult(";
+  out << ")";
+  return out;
+}
+
+
+ClearFileMetadataRequest::~ClearFileMetadataRequest() throw() {
+}
+
+
+void ClearFileMetadataRequest::__set_fileIds(const std::vector<int64_t> & val) {
+  this->fileIds = val;
+}
+
+const char* ClearFileMetadataRequest::ascii_fingerprint = "E49D7D1A9013CC81CD0F69D631EF82E4";
+const uint8_t ClearFileMetadataRequest::binary_fingerprint[16] = {0xE4,0x9D,0x7D,0x1A,0x90,0x13,0xCC,0x81,0xCD,0x0F,0x69,0xD6,0x31,0xEF,0x82,0xE4};
+
+uint32_t ClearFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_fileIds = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->fileIds.clear();
+            uint32_t _size671;
+            ::apache::thrift::protocol::TType _etype674;
+            xfer += iprot->readListBegin(_etype674, _size671);
+            this->fileIds.resize(_size671);
+            uint32_t _i675;
+            for (_i675 = 0; _i675 < _size671; ++_i675)
+            {
+              xfer += iprot->readI64(this->fileIds[_i675]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_fileIds = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_fileIds)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t ClearFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  oprot->incrementRecursionDepth();
+  xfer += oprot->writeStructBegin("ClearFileMetadataRequest");
+
+  xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
+    std::vector<int64_t> ::const_iterator _iter676;
+    for (_iter676 = this->fileIds.begin(); _iter676 != this->fileIds.end(); ++_iter676)
+    {
+      xfer += oprot->writeI64((*_iter676));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  oprot->decrementRecursionDepth();
+  return xfer;
+}
+
+void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b) {
+  using ::std::swap;
+  swap(a.fileIds, b.fileIds);
+}
+
+ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other677) {
+  fileIds = other677.fileIds;
+}
+ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other678) {
+  fileIds = other678.fileIds;
+  return *this;
+}
+std::ostream& operator<<(std::ostream& out, const ClearFileMetadataRequest& obj) {
+  using apache::thrift::to_string;
+  out << "ClearFileMetadataRequest(";
+  out << "fileIds=" << to_string(obj.fileIds);
+  out << ")";
+  return out;
+}
+
+
 GetAllFunctionsResponse::~GetAllFunctionsResponse() throw() {
 }
 
@@ -14409,14 +15479,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size609;
-            ::apache::thrift::protocol::TType _etype612;
-            xfer += iprot->readListBegin(_etype612, _size609);
-            this->functions.resize(_size609);
-            uint32_t _i613;
-            for (_i613 = 0; _i613 < _size609; ++_i613)
+            uint32_t _size679;
+            ::apache::thrift::protocol::TType _etype682;
+            xfer += iprot->readListBegin(_etype682, _size679);
+            this->functions.resize(_size679);
+            uint32_t _i683;
+            for (_i683 = 0; _i683 < _size679; ++_i683)
             {
-              xfer += this->functions[_i613].read(iprot);
+              xfer += this->functions[_i683].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14446,10 +15516,10 @@ uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* o
     xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->functions.size()));
-      std::vector<Function> ::const_iterator _iter614;
-      for (_iter614 = this->functions.begin(); _iter614 != this->functions.end(); ++_iter614)
+      std::vector<Function> ::const_iterator _iter684;
+      for (_iter684 = this->functions.begin(); _iter684 != this->functions.end(); ++_iter684)
       {
-        xfer += (*_iter614).write(oprot);
+        xfer += (*_iter684).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14467,13 +15537,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other615) {
-  functions = other615.functions;
-  __isset = other615.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other685) {
+  functions = other685.functions;
+  __isset = other685.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other616) {
-  functions = other616.functions;
-  __isset = other616.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other686) {
+  functions = other686.functions;
+  __isset = other686.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const GetAllFunctionsResponse& obj) {
@@ -14557,13 +15627,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other617) : TException() {
-  message = other617.message;
-  __isset = other617.__isset;
+MetaException::MetaException(const MetaException& other687) : TException() {
+  message = other687.message;
+  __isset = other687.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other618) {
-  message = other618.message;
-  __isset = other618.__isset;
+MetaException& MetaException::operator=(const MetaException& other688) {
+  message = other688.message;
+  __isset = other688.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const MetaException& obj) {
@@ -14647,13 +15717,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other619) : TException() {
-  message = other619.message;
-  __isset = other619.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other689) : TException() {
+  message = other689.message;
+  __isset = other689.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other620) {
-  message = other620.message;
-  __isset = other620.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other690) {
+  message = other690.message;
+  __isset = other690.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const UnknownTableException& obj) {
@@ -14737,13 +15807,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other621) : TException() {
-  message = other621.message;
-  __isset = other621.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other691) : TException() {
+  message = other691.message;
+  __isset = other691.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other622) {
-  message = other622.message;
-  __isset = other622.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other692) {
+  message = other692.message;
+  __isset = other692.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const UnknownDBException& obj) {
@@ -14827,13 +15897,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other623) : TException() {
-  message = other623.message;
-  __isset = other623.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other693) : TException() {
+  message = other693.message;
+  __isset = other693.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other624) {
-  message = other624.message;
-  __isset = other624.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other694) {
+  message = other694.message;
+  __isset = other694.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const AlreadyExistsException& obj) {
@@ -14917,13 +15987,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other625) : TException() {
-  message = other625.message;
-  __isset = other625.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other695) : TException() {
+  message = other695.message;
+  __isset = other695.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other626) {
-  message = other626.message;
-  __isset = other626.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other696) {
+  message = other696.message;
+  __isset = other696.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const InvalidPartitionException& obj) {
@@ -15007,13 +16077,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other627) : TException() {
-  message = other627.message;
-  __isset = other627.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other697) : TException() {
+  message = other697.message;
+  __isset = other697.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other628) {
-  message = other628.message;
-  __isset = other628.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other698) {
+  message = other698.message;
+  __isset = other698.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const UnknownPartitionException& obj) {
@@ -15097,13 +16167,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other629) : TException() {
-  message = other629.message;
-  __isset = other629.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other699) : TException() {
+  message = other699.message;
+  __isset = other699.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other630) {
-  message = other630.message;
-  __isset = other630.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other700) {
+  message = other700.message;
+  __isset = other700.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const InvalidObjectException& obj) {
@@ -15187,13 +16257,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other631) : TException() {
-  message = other631.message;
-  __isset = other631.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other701) : TException() {
+  message = other701.message;
+  __isset = other701.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other632) {
-  message = other632.message;
-  __isset = other632.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other702) {
+  message = other702.message;
+  __isset = other702.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const NoSuchObjectException& obj) {
@@ -15277,13 +16347,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other633) : TException() {
-  message = other633.message;
-  __isset = other633.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other703) : TException() {
+  message = other703.message;
+  __isset = other703.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other634) {
-  message = other634.message;
-  __isset = other634.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other704) {
+  message = other704.message;
+  __isset = other704.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const IndexAlreadyExistsException& obj) {
@@ -15367,13 +16437,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other635) : TException() {
-  message = other635.message;
-  __isset = other635.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other705) : TException() {
+  message = other705.message;
+  __isset = other705.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other636) {
-  message = other636.message;
-  __isset = other636.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other706) {
+  message = other706.message;
+  __isset = other706.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const InvalidOperationException& obj) {
@@ -15457,13 +16527,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other637) : TException() {
-  message = other637.message;
-  __isset = other637.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other707) : TException() {
+  message = other707.message;
+  __isset = other707.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other638) {
-  message = other638.message;
-  __isset = other638.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other708) {
+  message = other708.message;
+  __isset = other708.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const ConfigValSecurityException& obj) {
@@ -15547,13 +16617,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other639) : TException() {
-  message = other639.message;
-  __isset = other639.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other709) : TException() {
+  message = other709.message;
+  __isset = other709.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other640) {
-  message = other640.message;
-  __isset = other640.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other710) {
+  message = other710.message;
+  __isset = other710.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const InvalidInputException& obj) {
@@ -15637,13 +16707,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other641) : TException() {
-  message = other641.message;
-  __isset = other641.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other711) : TException() {
+  message = other711.message;
+  __isset = other711.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other642) {
-  message = other642.message;
-  __isset = other642.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other712) {
+  message = other712.message;
+  __isset = other712.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const NoSuchTxnException& obj) {
@@ -15727,13 +16797,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other643) : TException() {
-  message = other643.message;
-  __isset = other643.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other713) : TException() {
+  message = other713.message;
+  __isset = other713.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other644) {
-  message = other644.message;
-  __isset = other644.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other714) {
+  message = other714.message;
+  __isset = other714.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const TxnAbortedException& obj) {
@@ -15817,13 +16887,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other645) : TException() {
-  message = other645.message;
-  __isset = other645.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other715) : TException() {
+  message = other715.message;
+  __isset = other715.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other646) {
-  message = other646.message;
-  __isset = other646.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other716) {
+  message = other716.message;
+  __isset = other716.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const TxnOpenException& obj) {
@@ -15907,13 +16977,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other647) : TException() {
-  message = other647.message;
-  __isset = other647.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other717) : TException() {
+  message = other717.message;
+  __isset = other717.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other648) {
-  message = other648.message;
-  __isset = other648.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other718) {
+  message = other718.message;
+  __isset = other718.__isset;
   return *this;
 }
 std::ostream& operator<<(std::ostream& out, const NoSuchLockException& obj) {


[27/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 9c73767..8770e85 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1010,6 +1010,26 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
   /**
    */
   public function flushCache();
+  /**
+   * @param \metastore\GetFileMetadataByExprRequest $req
+   * @return \metastore\GetFileMetadataByExprResult
+   */
+  public function get_file_metadata_by_expr(\metastore\GetFileMetadataByExprRequest $req);
+  /**
+   * @param \metastore\GetFileMetadataRequest $req
+   * @return \metastore\GetFileMetadataResult
+   */
+  public function get_file_metadata(\metastore\GetFileMetadataRequest $req);
+  /**
+   * @param \metastore\PutFileMetadataRequest $req
+   * @return \metastore\PutFileMetadataResult
+   */
+  public function put_file_metadata(\metastore\PutFileMetadataRequest $req);
+  /**
+   * @param \metastore\ClearFileMetadataRequest $req
+   * @return \metastore\ClearFileMetadataResult
+   */
+  public function clear_file_metadata(\metastore\ClearFileMetadataRequest $req);
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@ -8191,196 +8211,221 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
-}
-
-// HELPER FUNCTIONS AND STRUCTURES
+  public function get_file_metadata_by_expr(\metastore\GetFileMetadataByExprRequest $req)
+  {
+    $this->send_get_file_metadata_by_expr($req);
+    return $this->recv_get_file_metadata_by_expr();
+  }
 
-class ThriftHiveMetastore_getMetaConf_args {
-  static $_TSPEC;
+  public function send_get_file_metadata_by_expr(\metastore\GetFileMetadataByExprRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_file_metadata_by_expr_args();
+    $args->req = $req;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_file_metadata_by_expr', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_file_metadata_by_expr', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
 
-  /**
-   * @var string
-   */
-  public $key = null;
+  public function recv_get_file_metadata_by_expr()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_file_metadata_by_expr_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
 
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'key',
-          'type' => TType::STRING,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['key'])) {
-        $this->key = $vals['key'];
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
       }
+      $result = new \metastore\ThriftHiveMetastore_get_file_metadata_by_expr_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
     }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("get_file_metadata_by_expr failed: unknown result");
   }
 
-  public function getName() {
-    return 'ThriftHiveMetastore_getMetaConf_args';
+  public function get_file_metadata(\metastore\GetFileMetadataRequest $req)
+  {
+    $this->send_get_file_metadata($req);
+    return $this->recv_get_file_metadata();
   }
 
-  public function read($input)
+  public function send_get_file_metadata(\metastore\GetFileMetadataRequest $req)
   {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
+    $args = new \metastore\ThriftHiveMetastore_get_file_metadata_args();
+    $args->req = $req;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
     {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->key);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
+      thrift_protocol_write_binary($this->output_, 'get_file_metadata', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_file_metadata', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
     }
-    $xfer += $input->readStructEnd();
-    return $xfer;
   }
 
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_getMetaConf_args');
-    if ($this->key !== null) {
-      $xfer += $output->writeFieldBegin('key', TType::STRING, 1);
-      $xfer += $output->writeString($this->key);
-      $xfer += $output->writeFieldEnd();
+  public function recv_get_file_metadata()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_file_metadata_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_file_metadata_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
     }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("get_file_metadata failed: unknown result");
   }
 
-}
+  public function put_file_metadata(\metastore\PutFileMetadataRequest $req)
+  {
+    $this->send_put_file_metadata($req);
+    return $this->recv_put_file_metadata();
+  }
 
-class ThriftHiveMetastore_getMetaConf_result {
-  static $_TSPEC;
+  public function send_put_file_metadata(\metastore\PutFileMetadataRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_put_file_metadata_args();
+    $args->req = $req;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'put_file_metadata', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('put_file_metadata', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
 
-  /**
-   * @var string
-   */
-  public $success = null;
-  /**
-   * @var \metastore\MetaException
-   */
-  public $o1 = null;
+  public function recv_put_file_metadata()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_put_file_metadata_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
 
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::STRING,
-          ),
-        1 => array(
-          'var' => 'o1',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\MetaException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['o1'])) {
-        $this->o1 = $vals['o1'];
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
       }
+      $result = new \metastore\ThriftHiveMetastore_put_file_metadata_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
     }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("put_file_metadata failed: unknown result");
   }
 
-  public function getName() {
-    return 'ThriftHiveMetastore_getMetaConf_result';
+  public function clear_file_metadata(\metastore\ClearFileMetadataRequest $req)
+  {
+    $this->send_clear_file_metadata($req);
+    return $this->recv_clear_file_metadata();
   }
 
-  public function read($input)
+  public function send_clear_file_metadata(\metastore\ClearFileMetadataRequest $req)
   {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
+    $args = new \metastore\ThriftHiveMetastore_clear_file_metadata_args();
+    $args->req = $req;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
     {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->success);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->o1 = new \metastore\MetaException();
-            $xfer += $this->o1->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
+      thrift_protocol_write_binary($this->output_, 'clear_file_metadata', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('clear_file_metadata', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
     }
-    $xfer += $input->readStructEnd();
-    return $xfer;
   }
 
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_getMetaConf_result');
-    if ($this->success !== null) {
-      $xfer += $output->writeFieldBegin('success', TType::STRING, 0);
-      $xfer += $output->writeString($this->success);
-      $xfer += $output->writeFieldEnd();
+  public function recv_clear_file_metadata()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_clear_file_metadata_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_clear_file_metadata_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
     }
-    if ($this->o1 !== null) {
-      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
-      $xfer += $this->o1->write($output);
-      $xfer += $output->writeFieldEnd();
+    if ($result->success !== null) {
+      return $result->success;
     }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
+    throw new \Exception("clear_file_metadata failed: unknown result");
   }
 
 }
 
-class ThriftHiveMetastore_setMetaConf_args {
+// HELPER FUNCTIONS AND STRUCTURES
+
+class ThriftHiveMetastore_getMetaConf_args {
   static $_TSPEC;
 
   /**
    * @var string
    */
   public $key = null;
-  /**
-   * @var string
-   */
-  public $value = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -8389,24 +8434,203 @@ class ThriftHiveMetastore_setMetaConf_args {
           'var' => 'key',
           'type' => TType::STRING,
           ),
-        2 => array(
-          'var' => 'value',
-          'type' => TType::STRING,
-          ),
         );
     }
     if (is_array($vals)) {
       if (isset($vals['key'])) {
         $this->key = $vals['key'];
       }
-      if (isset($vals['value'])) {
-        $this->value = $vals['value'];
-      }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_setMetaConf_args';
+    return 'ThriftHiveMetastore_getMetaConf_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->key);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_getMetaConf_args');
+    if ($this->key !== null) {
+      $xfer += $output->writeFieldBegin('key', TType::STRING, 1);
+      $xfer += $output->writeString($this->key);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_getMetaConf_result {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRING,
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_getMetaConf_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->success);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_getMetaConf_result');
+    if ($this->success !== null) {
+      $xfer += $output->writeFieldBegin('success', TType::STRING, 0);
+      $xfer += $output->writeString($this->success);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_setMetaConf_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $key = null;
+  /**
+   * @var string
+   */
+  public $value = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'key',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'value',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['key'])) {
+        $this->key = $vals['key'];
+      }
+      if (isset($vals['value'])) {
+        $this->value = $vals['value'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_setMetaConf_args';
   }
 
   public function read($input)
@@ -9342,14 +9566,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size472 = 0;
-            $_etype475 = 0;
-            $xfer += $input->readListBegin($_etype475, $_size472);
-            for ($_i476 = 0; $_i476 < $_size472; ++$_i476)
+            $_size532 = 0;
+            $_etype535 = 0;
+            $xfer += $input->readListBegin($_etype535, $_size532);
+            for ($_i536 = 0; $_i536 < $_size532; ++$_i536)
             {
-              $elem477 = null;
-              $xfer += $input->readString($elem477);
-              $this->success []= $elem477;
+              $elem537 = null;
+              $xfer += $input->readString($elem537);
+              $this->success []= $elem537;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9385,9 +9609,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter478)
+          foreach ($this->success as $iter538)
           {
-            $xfer += $output->writeString($iter478);
+            $xfer += $output->writeString($iter538);
           }
         }
         $output->writeListEnd();
@@ -9518,14 +9742,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size479 = 0;
-            $_etype482 = 0;
-            $xfer += $input->readListBegin($_etype482, $_size479);
-            for ($_i483 = 0; $_i483 < $_size479; ++$_i483)
+            $_size539 = 0;
+            $_etype542 = 0;
+            $xfer += $input->readListBegin($_etype542, $_size539);
+            for ($_i543 = 0; $_i543 < $_size539; ++$_i543)
             {
-              $elem484 = null;
-              $xfer += $input->readString($elem484);
-              $this->success []= $elem484;
+              $elem544 = null;
+              $xfer += $input->readString($elem544);
+              $this->success []= $elem544;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9561,9 +9785,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter485)
+          foreach ($this->success as $iter545)
           {
-            $xfer += $output->writeString($iter485);
+            $xfer += $output->writeString($iter545);
           }
         }
         $output->writeListEnd();
@@ -10564,18 +10788,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size486 = 0;
-            $_ktype487 = 0;
-            $_vtype488 = 0;
-            $xfer += $input->readMapBegin($_ktype487, $_vtype488, $_size486);
-            for ($_i490 = 0; $_i490 < $_size486; ++$_i490)
+            $_size546 = 0;
+            $_ktype547 = 0;
+            $_vtype548 = 0;
+            $xfer += $input->readMapBegin($_ktype547, $_vtype548, $_size546);
+            for ($_i550 = 0; $_i550 < $_size546; ++$_i550)
             {
-              $key491 = '';
-              $val492 = new \metastore\Type();
-              $xfer += $input->readString($key491);
-              $val492 = new \metastore\Type();
-              $xfer += $val492->read($input);
-              $this->success[$key491] = $val492;
+              $key551 = '';
+              $val552 = new \metastore\Type();
+              $xfer += $input->readString($key551);
+              $val552 = new \metastore\Type();
+              $xfer += $val552->read($input);
+              $this->success[$key551] = $val552;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -10611,10 +10835,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter493 => $viter494)
+          foreach ($this->success as $kiter553 => $viter554)
           {
-            $xfer += $output->writeString($kiter493);
-            $xfer += $viter494->write($output);
+            $xfer += $output->writeString($kiter553);
+            $xfer += $viter554->write($output);
           }
         }
         $output->writeMapEnd();
@@ -10818,15 +11042,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size495 = 0;
-            $_etype498 = 0;
-            $xfer += $input->readListBegin($_etype498, $_size495);
-            for ($_i499 = 0; $_i499 < $_size495; ++$_i499)
+            $_size555 = 0;
+            $_etype558 = 0;
+            $xfer += $input->readListBegin($_etype558, $_size555);
+            for ($_i559 = 0; $_i559 < $_size555; ++$_i559)
             {
-              $elem500 = null;
-              $elem500 = new \metastore\FieldSchema();
-              $xfer += $elem500->read($input);
-              $this->success []= $elem500;
+              $elem560 = null;
+              $elem560 = new \metastore\FieldSchema();
+              $xfer += $elem560->read($input);
+              $this->success []= $elem560;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10878,9 +11102,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter501)
+          foreach ($this->success as $iter561)
           {
-            $xfer += $iter501->write($output);
+            $xfer += $iter561->write($output);
           }
         }
         $output->writeListEnd();
@@ -11122,15 +11346,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size502 = 0;
-            $_etype505 = 0;
-            $xfer += $input->readListBegin($_etype505, $_size502);
-            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
+            $_size562 = 0;
+            $_etype565 = 0;
+            $xfer += $input->readListBegin($_etype565, $_size562);
+            for ($_i566 = 0; $_i566 < $_size562; ++$_i566)
             {
-              $elem507 = null;
-              $elem507 = new \metastore\FieldSchema();
-              $xfer += $elem507->read($input);
-              $this->success []= $elem507;
+              $elem567 = null;
+              $elem567 = new \metastore\FieldSchema();
+              $xfer += $elem567->read($input);
+              $this->success []= $elem567;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11182,9 +11406,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter508)
+          foreach ($this->success as $iter568)
           {
-            $xfer += $iter508->write($output);
+            $xfer += $iter568->write($output);
           }
         }
         $output->writeListEnd();
@@ -11398,15 +11622,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size509 = 0;
-            $_etype512 = 0;
-            $xfer += $input->readListBegin($_etype512, $_size509);
-            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
+            $_size569 = 0;
+            $_etype572 = 0;
+            $xfer += $input->readListBegin($_etype572, $_size569);
+            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
             {
-              $elem514 = null;
-              $elem514 = new \metastore\FieldSchema();
-              $xfer += $elem514->read($input);
-              $this->success []= $elem514;
+              $elem574 = null;
+              $elem574 = new \metastore\FieldSchema();
+              $xfer += $elem574->read($input);
+              $this->success []= $elem574;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11458,9 +11682,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter515)
+          foreach ($this->success as $iter575)
           {
-            $xfer += $iter515->write($output);
+            $xfer += $iter575->write($output);
           }
         }
         $output->writeListEnd();
@@ -11702,15 +11926,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size516 = 0;
-            $_etype519 = 0;
-            $xfer += $input->readListBegin($_etype519, $_size516);
-            for ($_i520 = 0; $_i520 < $_size516; ++$_i520)
+            $_size576 = 0;
+            $_etype579 = 0;
+            $xfer += $input->readListBegin($_etype579, $_size576);
+            for ($_i580 = 0; $_i580 < $_size576; ++$_i580)
             {
-              $elem521 = null;
-              $elem521 = new \metastore\FieldSchema();
-              $xfer += $elem521->read($input);
-              $this->success []= $elem521;
+              $elem581 = null;
+              $elem581 = new \metastore\FieldSchema();
+              $xfer += $elem581->read($input);
+              $this->success []= $elem581;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11762,9 +11986,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter522)
+          foreach ($this->success as $iter582)
           {
-            $xfer += $iter522->write($output);
+            $xfer += $iter582->write($output);
           }
         }
         $output->writeListEnd();
@@ -12919,14 +13143,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size523 = 0;
-            $_etype526 = 0;
-            $xfer += $input->readListBegin($_etype526, $_size523);
-            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
+            $_size583 = 0;
+            $_etype586 = 0;
+            $xfer += $input->readListBegin($_etype586, $_size583);
+            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
             {
-              $elem528 = null;
-              $xfer += $input->readString($elem528);
-              $this->success []= $elem528;
+              $elem588 = null;
+              $xfer += $input->readString($elem588);
+              $this->success []= $elem588;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12962,9 +13186,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter529)
+          foreach ($this->success as $iter589)
           {
-            $xfer += $output->writeString($iter529);
+            $xfer += $output->writeString($iter589);
           }
         }
         $output->writeListEnd();
@@ -13120,14 +13344,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size530 = 0;
-            $_etype533 = 0;
-            $xfer += $input->readListBegin($_etype533, $_size530);
-            for ($_i534 = 0; $_i534 < $_size530; ++$_i534)
+            $_size590 = 0;
+            $_etype593 = 0;
+            $xfer += $input->readListBegin($_etype593, $_size590);
+            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
             {
-              $elem535 = null;
-              $xfer += $input->readString($elem535);
-              $this->success []= $elem535;
+              $elem595 = null;
+              $xfer += $input->readString($elem595);
+              $this->success []= $elem595;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13163,9 +13387,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter536)
+          foreach ($this->success as $iter596)
           {
-            $xfer += $output->writeString($iter536);
+            $xfer += $output->writeString($iter596);
           }
         }
         $output->writeListEnd();
@@ -13480,14 +13704,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size537 = 0;
-            $_etype540 = 0;
-            $xfer += $input->readListBegin($_etype540, $_size537);
-            for ($_i541 = 0; $_i541 < $_size537; ++$_i541)
+            $_size597 = 0;
+            $_etype600 = 0;
+            $xfer += $input->readListBegin($_etype600, $_size597);
+            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
             {
-              $elem542 = null;
-              $xfer += $input->readString($elem542);
-              $this->tbl_names []= $elem542;
+              $elem602 = null;
+              $xfer += $input->readString($elem602);
+              $this->tbl_names []= $elem602;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13520,9 +13744,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter543)
+          foreach ($this->tbl_names as $iter603)
           {
-            $xfer += $output->writeString($iter543);
+            $xfer += $output->writeString($iter603);
           }
         }
         $output->writeListEnd();
@@ -13623,15 +13847,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size544 = 0;
-            $_etype547 = 0;
-            $xfer += $input->readListBegin($_etype547, $_size544);
-            for ($_i548 = 0; $_i548 < $_size544; ++$_i548)
+            $_size604 = 0;
+            $_etype607 = 0;
+            $xfer += $input->readListBegin($_etype607, $_size604);
+            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
             {
-              $elem549 = null;
-              $elem549 = new \metastore\Table();
-              $xfer += $elem549->read($input);
-              $this->success []= $elem549;
+              $elem609 = null;
+              $elem609 = new \metastore\Table();
+              $xfer += $elem609->read($input);
+              $this->success []= $elem609;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13683,9 +13907,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter550)
+          foreach ($this->success as $iter610)
           {
-            $xfer += $iter550->write($output);
+            $xfer += $iter610->write($output);
           }
         }
         $output->writeListEnd();
@@ -13921,14 +14145,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size551 = 0;
-            $_etype554 = 0;
-            $xfer += $input->readListBegin($_etype554, $_size551);
-            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
+            $_size611 = 0;
+            $_etype614 = 0;
+            $xfer += $input->readListBegin($_etype614, $_size611);
+            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
             {
-              $elem556 = null;
-              $xfer += $input->readString($elem556);
-              $this->success []= $elem556;
+              $elem616 = null;
+              $xfer += $input->readString($elem616);
+              $this->success []= $elem616;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13980,9 +14204,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter557)
+          foreach ($this->success as $iter617)
           {
-            $xfer += $output->writeString($iter557);
+            $xfer += $output->writeString($iter617);
           }
         }
         $output->writeListEnd();
@@ -15295,15 +15519,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size558 = 0;
-            $_etype561 = 0;
-            $xfer += $input->readListBegin($_etype561, $_size558);
-            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
+            $_size618 = 0;
+            $_etype621 = 0;
+            $xfer += $input->readListBegin($_etype621, $_size618);
+            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
             {
-              $elem563 = null;
-              $elem563 = new \metastore\Partition();
-              $xfer += $elem563->read($input);
-              $this->new_parts []= $elem563;
+              $elem623 = null;
+              $elem623 = new \metastore\Partition();
+              $xfer += $elem623->read($input);
+              $this->new_parts []= $elem623;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15331,9 +15555,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter564)
+          foreach ($this->new_parts as $iter624)
           {
-            $xfer += $iter564->write($output);
+            $xfer += $iter624->write($output);
           }
         }
         $output->writeListEnd();
@@ -15548,15 +15772,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size565 = 0;
-            $_etype568 = 0;
-            $xfer += $input->readListBegin($_etype568, $_size565);
-            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
+            $_size625 = 0;
+            $_etype628 = 0;
+            $xfer += $input->readListBegin($_etype628, $_size625);
+            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
             {
-              $elem570 = null;
-              $elem570 = new \metastore\PartitionSpec();
-              $xfer += $elem570->read($input);
-              $this->new_parts []= $elem570;
+              $elem630 = null;
+              $elem630 = new \metastore\PartitionSpec();
+              $xfer += $elem630->read($input);
+              $this->new_parts []= $elem630;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15584,9 +15808,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter571)
+          foreach ($this->new_parts as $iter631)
           {
-            $xfer += $iter571->write($output);
+            $xfer += $iter631->write($output);
           }
         }
         $output->writeListEnd();
@@ -15836,14 +16060,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size572 = 0;
-            $_etype575 = 0;
-            $xfer += $input->readListBegin($_etype575, $_size572);
-            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
+            $_size632 = 0;
+            $_etype635 = 0;
+            $xfer += $input->readListBegin($_etype635, $_size632);
+            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
             {
-              $elem577 = null;
-              $xfer += $input->readString($elem577);
-              $this->part_vals []= $elem577;
+              $elem637 = null;
+              $xfer += $input->readString($elem637);
+              $this->part_vals []= $elem637;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15881,9 +16105,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter578)
+          foreach ($this->part_vals as $iter638)
           {
-            $xfer += $output->writeString($iter578);
+            $xfer += $output->writeString($iter638);
           }
         }
         $output->writeListEnd();
@@ -16385,14 +16609,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size579 = 0;
-            $_etype582 = 0;
-            $xfer += $input->readListBegin($_etype582, $_size579);
-            for ($_i583 = 0; $_i583 < $_size579; ++$_i583)
+            $_size639 = 0;
+            $_etype642 = 0;
+            $xfer += $input->readListBegin($_etype642, $_size639);
+            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
             {
-              $elem584 = null;
-              $xfer += $input->readString($elem584);
-              $this->part_vals []= $elem584;
+              $elem644 = null;
+              $xfer += $input->readString($elem644);
+              $this->part_vals []= $elem644;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16438,9 +16662,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter585)
+          foreach ($this->part_vals as $iter645)
           {
-            $xfer += $output->writeString($iter585);
+            $xfer += $output->writeString($iter645);
           }
         }
         $output->writeListEnd();
@@ -17294,14 +17518,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size586 = 0;
-            $_etype589 = 0;
-            $xfer += $input->readListBegin($_etype589, $_size586);
-            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
+            $_size646 = 0;
+            $_etype649 = 0;
+            $xfer += $input->readListBegin($_etype649, $_size646);
+            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
             {
-              $elem591 = null;
-              $xfer += $input->readString($elem591);
-              $this->part_vals []= $elem591;
+              $elem651 = null;
+              $xfer += $input->readString($elem651);
+              $this->part_vals []= $elem651;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17346,9 +17570,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter592)
+          foreach ($this->part_vals as $iter652)
           {
-            $xfer += $output->writeString($iter592);
+            $xfer += $output->writeString($iter652);
           }
         }
         $output->writeListEnd();
@@ -17601,14 +17825,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size593 = 0;
-            $_etype596 = 0;
-            $xfer += $input->readListBegin($_etype596, $_size593);
-            for ($_i597 = 0; $_i597 < $_size593; ++$_i597)
+            $_size653 = 0;
+            $_etype656 = 0;
+            $xfer += $input->readListBegin($_etype656, $_size653);
+            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
             {
-              $elem598 = null;
-              $xfer += $input->readString($elem598);
-              $this->part_vals []= $elem598;
+              $elem658 = null;
+              $xfer += $input->readString($elem658);
+              $this->part_vals []= $elem658;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17661,9 +17885,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter599)
+          foreach ($this->part_vals as $iter659)
           {
-            $xfer += $output->writeString($iter599);
+            $xfer += $output->writeString($iter659);
           }
         }
         $output->writeListEnd();
@@ -18677,14 +18901,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size600 = 0;
-            $_etype603 = 0;
-            $xfer += $input->readListBegin($_etype603, $_size600);
-            for ($_i604 = 0; $_i604 < $_size600; ++$_i604)
+            $_size660 = 0;
+            $_etype663 = 0;
+            $xfer += $input->readListBegin($_etype663, $_size660);
+            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
             {
-              $elem605 = null;
-              $xfer += $input->readString($elem605);
-              $this->part_vals []= $elem605;
+              $elem665 = null;
+              $xfer += $input->readString($elem665);
+              $this->part_vals []= $elem665;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18722,9 +18946,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter606)
+          foreach ($this->part_vals as $iter666)
           {
-            $xfer += $output->writeString($iter606);
+            $xfer += $output->writeString($iter666);
           }
         }
         $output->writeListEnd();
@@ -18966,17 +19190,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size607 = 0;
-            $_ktype608 = 0;
-            $_vtype609 = 0;
-            $xfer += $input->readMapBegin($_ktype608, $_vtype609, $_size607);
-            for ($_i611 = 0; $_i611 < $_size607; ++$_i611)
+            $_size667 = 0;
+            $_ktype668 = 0;
+            $_vtype669 = 0;
+            $xfer += $input->readMapBegin($_ktype668, $_vtype669, $_size667);
+            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
             {
-              $key612 = '';
-              $val613 = '';
-              $xfer += $input->readString($key612);
-              $xfer += $input->readString($val613);
-              $this->partitionSpecs[$key612] = $val613;
+              $key672 = '';
+              $val673 = '';
+              $xfer += $input->readString($key672);
+              $xfer += $input->readString($val673);
+              $this->partitionSpecs[$key672] = $val673;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19032,10 +19256,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter614 => $viter615)
+          foreach ($this->partitionSpecs as $kiter674 => $viter675)
           {
-            $xfer += $output->writeString($kiter614);
-            $xfer += $output->writeString($viter615);
+            $xfer += $output->writeString($kiter674);
+            $xfer += $output->writeString($viter675);
           }
         }
         $output->writeMapEnd();
@@ -19361,14 +19585,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size616 = 0;
-            $_etype619 = 0;
-            $xfer += $input->readListBegin($_etype619, $_size616);
-            for ($_i620 = 0; $_i620 < $_size616; ++$_i620)
+            $_size676 = 0;
+            $_etype679 = 0;
+            $xfer += $input->readListBegin($_etype679, $_size676);
+            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
             {
-              $elem621 = null;
-              $xfer += $input->readString($elem621);
-              $this->part_vals []= $elem621;
+              $elem681 = null;
+              $xfer += $input->readString($elem681);
+              $this->part_vals []= $elem681;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19385,14 +19609,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size622 = 0;
-            $_etype625 = 0;
-            $xfer += $input->readListBegin($_etype625, $_size622);
-            for ($_i626 = 0; $_i626 < $_size622; ++$_i626)
+            $_size682 = 0;
+            $_etype685 = 0;
+            $xfer += $input->readListBegin($_etype685, $_size682);
+            for ($_i686 = 0; $_i686 < $_size682; ++$_i686)
             {
-              $elem627 = null;
-              $xfer += $input->readString($elem627);
-              $this->group_names []= $elem627;
+              $elem687 = null;
+              $xfer += $input->readString($elem687);
+              $this->group_names []= $elem687;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19430,9 +19654,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter628)
+          foreach ($this->part_vals as $iter688)
           {
-            $xfer += $output->writeString($iter628);
+            $xfer += $output->writeString($iter688);
           }
         }
         $output->writeListEnd();
@@ -19452,9 +19676,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter629)
+          foreach ($this->group_names as $iter689)
           {
-            $xfer += $output->writeString($iter629);
+            $xfer += $output->writeString($iter689);
           }
         }
         $output->writeListEnd();
@@ -20045,15 +20269,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size630 = 0;
-            $_etype633 = 0;
-            $xfer += $input->readListBegin($_etype633, $_size630);
-            for ($_i634 = 0; $_i634 < $_size630; ++$_i634)
+            $_size690 = 0;
+            $_etype693 = 0;
+            $xfer += $input->readListBegin($_etype693, $_size690);
+            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
             {
-              $elem635 = null;
-              $elem635 = new \metastore\Partition();
-              $xfer += $elem635->read($input);
-              $this->success []= $elem635;
+              $elem695 = null;
+              $elem695 = new \metastore\Partition();
+              $xfer += $elem695->read($input);
+              $this->success []= $elem695;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20097,9 +20321,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter636)
+          foreach ($this->success as $iter696)
           {
-            $xfer += $iter636->write($output);
+            $xfer += $iter696->write($output);
           }
         }
         $output->writeListEnd();
@@ -20245,14 +20469,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size637 = 0;
-            $_etype640 = 0;
-            $xfer += $input->readListBegin($_etype640, $_size637);
-            for ($_i641 = 0; $_i641 < $_size637; ++$_i641)
+            $_size697 = 0;
+            $_etype700 = 0;
+            $xfer += $input->readListBegin($_etype700, $_size697);
+            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
             {
-              $elem642 = null;
-              $xfer += $input->readString($elem642);
-              $this->group_names []= $elem642;
+              $elem702 = null;
+              $xfer += $input->readString($elem702);
+              $this->group_names []= $elem702;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20300,9 +20524,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter643)
+          foreach ($this->group_names as $iter703)
           {
-            $xfer += $output->writeString($iter643);
+            $xfer += $output->writeString($iter703);
           }
         }
         $output->writeListEnd();
@@ -20391,15 +20615,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size644 = 0;
-            $_etype647 = 0;
-            $xfer += $input->readListBegin($_etype647, $_size644);
-            for ($_i648 = 0; $_i648 < $_size644; ++$_i648)
+            $_size704 = 0;
+            $_etype707 = 0;
+            $xfer += $input->readListBegin($_etype707, $_size704);
+            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
             {
-              $elem649 = null;
-              $elem649 = new \metastore\Partition();
-              $xfer += $elem649->read($input);
-              $this->success []= $elem649;
+              $elem709 = null;
+              $elem709 = new \metastore\Partition();
+              $xfer += $elem709->read($input);
+              $this->success []= $elem709;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20443,9 +20667,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter650)
+          foreach ($this->success as $iter710)
           {
-            $xfer += $iter650->write($output);
+            $xfer += $iter710->write($output);
           }
         }
         $output->writeListEnd();
@@ -20665,15 +20889,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size651 = 0;
-            $_etype654 = 0;
-            $xfer += $input->readListBegin($_etype654, $_size651);
-            for ($_i655 = 0; $_i655 < $_size651; ++$_i655)
+            $_size711 = 0;
+            $_etype714 = 0;
+            $xfer += $input->readListBegin($_etype714, $_size711);
+            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
             {
-              $elem656 = null;
-              $elem656 = new \metastore\PartitionSpec();
-              $xfer += $elem656->read($input);
-              $this->success []= $elem656;
+              $elem716 = null;
+              $elem716 = new \metastore\PartitionSpec();
+              $xfer += $elem716->read($input);
+              $this->success []= $elem716;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20717,9 +20941,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter657)
+          foreach ($this->success as $iter717)
           {
-            $xfer += $iter657->write($output);
+            $xfer += $iter717->write($output);
           }
         }
         $output->writeListEnd();
@@ -20926,14 +21150,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size658 = 0;
-            $_etype661 = 0;
-            $xfer += $input->readListBegin($_etype661, $_size658);
-            for ($_i662 = 0; $_i662 < $_size658; ++$_i662)
+            $_size718 = 0;
+            $_etype721 = 0;
+            $xfer += $input->readListBegin($_etype721, $_size718);
+            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
             {
-              $elem663 = null;
-              $xfer += $input->readString($elem663);
-              $this->success []= $elem663;
+              $elem723 = null;
+              $xfer += $input->readString($elem723);
+              $this->success []= $elem723;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20969,9 +21193,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter664)
+          foreach ($this->success as $iter724)
           {
-            $xfer += $output->writeString($iter664);
+            $xfer += $output->writeString($iter724);
           }
         }
         $output->writeListEnd();
@@ -21087,14 +21311,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size665 = 0;
-            $_etype668 = 0;
-            $xfer += $input->readListBegin($_etype668, $_size665);
-            for ($_i669 = 0; $_i669 < $_size665; ++$_i669)
+            $_size725 = 0;
+            $_etype728 = 0;
+            $xfer += $input->readListBegin($_etype728, $_size725);
+            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
             {
-              $elem670 = null;
-              $xfer += $input->readString($elem670);
-              $this->part_vals []= $elem670;
+              $elem730 = null;
+              $xfer += $input->readString($elem730);
+              $this->part_vals []= $elem730;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21139,9 +21363,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter671)
+          foreach ($this->part_vals as $iter731)
           {
-            $xfer += $output->writeString($iter671);
+            $xfer += $output->writeString($iter731);
           }
         }
         $output->writeListEnd();
@@ -21235,15 +21459,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size672 = 0;
-            $_etype675 = 0;
-            $xfer += $input->readListBegin($_etype675, $_size672);
-            for ($_i676 = 0; $_i676 < $_size672; ++$_i676)
+            $_size732 = 0;
+            $_etype735 = 0;
+            $xfer += $input->readListBegin($_etype735, $_size732);
+            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
             {
-              $elem677 = null;
-              $elem677 = new \metastore\Partition();
-              $xfer += $elem677->read($input);
-              $this->success []= $elem677;
+              $elem737 = null;
+              $elem737 = new \metastore\Partition();
+              $xfer += $elem737->read($input);
+              $this->success []= $elem737;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21287,9 +21511,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter678)
+          foreach ($this->success as $iter738)
           {
-            $xfer += $iter678->write($output);
+            $xfer += $iter738->write($output);
           }
         }
         $output->writeListEnd();
@@ -21436,14 +21660,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size679 = 0;
-            $_etype682 = 0;
-            $xfer += $input->readListBegin($_etype682, $_size679);
-            for ($_i683 = 0; $_i683 < $_size679; ++$_i683)
+            $_size739 = 0;
+            $_etype742 = 0;
+            $xfer += $input->readListBegin($_etype742, $_size739);
+            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
             {
-              $elem684 = null;
-              $xfer += $input->readString($elem684);
-              $this->part_vals []= $elem684;
+              $elem744 = null;
+              $xfer += $input->readString($elem744);
+              $this->part_vals []= $elem744;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21467,14 +21691,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size685 = 0;
-            $_etype688 = 0;
-            $xfer += $input->readListBegin($_etype688, $_size685);
-            for ($_i689 = 0; $_i689 < $_size685; ++$_i689)
+            $_size745 = 0;
+            $_etype748 = 0;
+            $xfer += $input->readListBegin($_etype748, $_size745);
+            for ($_i749 = 0; $_i749 < $_size745; ++$_i749)
             {
-              $elem690 = null;
-              $xfer += $input->readString($elem690);
-              $this->group_names []= $elem690;
+              $elem750 = null;
+              $xfer += $input->readString($elem750);
+              $this->group_names []= $elem750;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21512,9 +21736,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter691)
+          foreach ($this->part_vals as $iter751)
           {
-            $xfer += $output->writeString($iter691);
+            $xfer += $output->writeString($iter751);
           }
         }
         $output->writeListEnd();
@@ -21539,9 +21763,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter692)
+          foreach ($this->group_names as $iter752)
           {
-            $xfer += $output->writeString($iter692);
+            $xfer += $output->writeString($iter752);
           }
         }
         $output->writeListEnd();
@@ -21630,15 +21854,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size693 = 0;
-            $_etype696 = 0;
-            $xfer += $input->readListBegin($_etype696, $_size693);
-            for ($_i697 = 0; $_i697 < $_size693; ++$_i697)
+            $_size753 = 0;
+            $_etype756 = 0;
+            $xfer += $input->readListBegin($_etype756, $_size753);
+            for ($_i757 = 0; $_i757 < $_size753; ++$_i757)
             {
-              $elem698 = null;
-              $elem698 = new \metastore\Partition();
-              $xfer += $elem698->read($input);
-              $this->success []= $elem698;
+              $elem758 = null;
+              $elem758 = new \metastore\Partition();
+              $xfer += $elem758->read($input);
+              $this->success []= $elem758;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21682,9 +21906,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter699)
+          foreach ($this->success as $iter759)
           {
-            $xfer += $iter699->write($output);
+            $xfer += $iter759->write($output);
           }
         }
         $output->writeListEnd();
@@ -21805,14 +22029,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size700 = 0;
-            $_etype703 = 0;
-            $xfer += $input->readListBegin($_etype703, $_size700);
-            for ($_i704 = 0; $_i704 < $_size700; ++$_i704)
+            $_size760 = 0;
+            $_etype763 = 0;
+            $xfer += $input->readListBegin($_etype763, $_size760);
+            for ($_i764 = 0; $_i764 < $_size760; ++$_i764)
             {
-              $elem705 = null;
-              $xfer += $input->readString($elem705);
-              $this->part_vals []= $elem705;
+              $elem765 = null;
+              $xfer += $input->readString($elem765);
+              $this->part_vals []= $elem765;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21857,9 +22081,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter706)
+          foreach ($this->part_vals as $iter766)
           {
-            $xfer += $output->writeString($iter706);
+            $xfer += $output->writeString($iter766);
           }
         }
         $output->writeListEnd();
@@ -21952,14 +22176,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size707 = 0;
-            $_etype710 = 0;
-            $xfer += $input->readListBegin($_etype710, $_size707);
-            for ($_i711 = 0; $_i711 < $_size707; ++$_i711)
+            $_size767 = 0;
+            $_etype770 = 0;
+            $xfer += $input->readListBegin($_etype770, $_size767);
+            for ($_i771 = 0; $_i771 < $_size767; ++$_i771)
             {
-              $elem712 = null;
-              $xfer += $input->readString($elem712);
-              $this->success []= $elem712;
+              $elem772 = null;
+              $xfer += $input->readString($elem772);
+              $this->success []= $elem772;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22003,9 +22227,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter713)
+          foreach ($this->success as $iter773)
           {
-            $xfer += $output->writeString($iter713);
+            $xfer += $output->writeString($iter773);
           }
         }
         $output->writeListEnd();
@@ -22248,15 +22472,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size714 = 0;
-            $_etype717 = 0;
-            $xfer += $input->readListBegin($_etype717, $_size714);
-            for ($_i718 = 0; $_i718 < $_size714; ++$_i718)
+            $_size774 = 0;
+            $_etype777 = 0;
+            $xfer += $input->readListBegin($_etype777, $_size774);
+            for ($_i778 = 0; $_i778 < $_size774; ++$_i778)
             {
-              $elem719 = null;
-              $elem719 = new \metastore\Partition();
-              $xfer += $elem719->read($input);
-              $this->success []= $elem719;
+              $elem779 = null;
+              $elem779 = new \metastore\Partition();
+              $xfer += $elem779->read($input);
+              $this->success []= $elem779;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22300,9 +22524,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter720)
+          foreach ($this->success as $iter780)
           {
-            $xfer += $iter720->write($output);
+            $xfer += $iter780->write($output);
           }
         }
         $output->writeListEnd();
@@ -22545,15 +22769,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size721 = 0;
-            $_etype724 = 0;
-            $xfer += $input->readListBegin($_etype724, $_size721);
-            for ($_i725 = 0; $_i725 < $_size721; ++$_i725)
+            $_size781 = 0;
+            $_etype784 = 0;
+            $xfer += $input->readListBegin($_etype784, $_size781);
+            for ($_i785 = 0; $_i785 < $_size781; ++$_i785)
             {
-              $elem726 = null;
-              $elem726 = new \metastore\PartitionSpec();
-              $xfer += $elem726->read($input);
-              $this->success []= $elem726;
+              $elem786 = null;
+              $elem786 = new \metastore\PartitionSpec();
+              $xfer += $elem786->read($input);
+              $this->success []= $elem786;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22597,9 +22821,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter727)
+          foreach ($this->success as $iter787)
           {
-            $xfer += $iter727->write($output);
+            $xfer += $iter787->write($output);
           }
         }
         $output->writeListEnd();
@@ -22919,14 +23143,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size728 = 0;
-            $_etype731 = 0;
-            $xfer += $input->readListBegin($_etype731, $_size728);
-            for ($_i732 = 0; $_i732 < $_size728; ++$_i732)
+            $_size788 = 0;
+            $_etype791 = 0;
+            $xfer += $input->readListBegin($_etype791, $_size788);
+            for ($_i792 = 0; $_i792 < $_size788; ++$_i792)
             {
-              $elem733 = null;
-              $xfer += $input->readString($elem733);
-              $this->names []= $elem733;
+              $elem793 = null;
+              $xfer += $input->readString($elem793);
+              $this->names []= $elem793;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22964,9 +23188,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter734)
+          foreach ($this->names as $iter794)
           {
-            $xfer += $output->writeString($iter734);
+            $xfer += $output->writeString($iter794);
           }
         }
         $output->writeListEnd();
@@ -23055,15 +23279,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size735 = 0;
-            $_etype738 = 0;
-            $xfer += $input->readListBegin($_etype738, $_size735);
-            for ($_i739 = 0; $_i739 < $_size735; ++$_i739)
+            $_size795 = 0;
+            $_etype798 = 0;
+            $xfer += $input->readListBegin($_etype798, $_size795);
+            for ($_i799 = 0; $_i799 < $_size795; ++$_i799)
             {
-              $elem740 = null;
-              $elem740 = new \metastore\Partition();
-              $xfer += $elem740->read($input);
-              $this->success []= $elem740;
+              $elem800 = null;
+              $elem800 = new \metastore\Partition();
+              $xfer += $elem800->read($input);
+              $this->success []= $elem800;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23107,9 +23331,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter741)
+          foreach ($this->success as $iter801)
           {
-            $xfer += $iter741->write($output);
+            $xfer += $iter801->write($output);
           }
         }
         $output->writeListEnd();
@@ -23448,15 +23672,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size742 = 0;
-            $_etype745 = 0;
-            $xfer += $input->readListBegin($_etype745, $_size742);
-            for ($_i746 = 0; $_i746 < $_size742; ++$_i746)
+            $_size802 = 0;
+            $_etype805 = 0;
+            $xfer += $input->readListBegin($_etype805, $_size802);
+            for ($_i806 = 0; $_i806 < $_size802; ++$_i806)
             {
-              $elem747 = null;
-              $elem747 = new \metastore\Partition();
-              $xfer += $elem747->read($input);
-              $this->new_parts []= $elem747;
+              $elem807 = null;
+              $elem807 = new \metastore\Partition();
+              $xfer += $elem807->read($input);
+              $this->new_parts []= $elem807;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23494,9 +23718,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter748)
+          foreach ($this->new_parts as $iter808)
           {
-            $xfer += $iter748->write($output);
+            $xfer += $iter808->write($output);
           }
         }
         $output->writeListEnd();
@@ -23966,14 +24190,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size749 = 0;
-            $_etype752 = 0;
-            $xfer += $input->readListBegin($_etype752, $_size749);
-            for ($_i753 = 0; $_i753 < $_size749; ++$_i753)
+            $_size809 = 0;
+            $_etype812 = 0;
+            $xfer += $input->readListBegin($_etype812, $_size809);
+            for ($_i813 = 0; $_i813 < $_size809; ++$_i813)
             {
-              $elem754 = null;
-              $xfer += $input->readString($elem754);
-              $this->part_vals []= $elem754;
+              $elem814 = null;
+              $xfer += $input->readString($elem814);
+              $this->part_vals []= $elem814;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24019,9 +24243,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter755)
+          foreach ($this->part_vals as $iter815)
           {
-            $xfer += $output->writeString($iter755);
+            $xfer += $output->writeString($iter815);
           }
         }
         $output->writeListEnd();
@@ -24206,14 +24430,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size756 = 0;
-            $_etype759 = 0;
-            $xfer += $input->readListBegin($_etype759, $_size756);
-            for ($_i760 = 0; $_i760 < $_size756; ++$_i760)
+            $_size816 = 0;
+            $_etype819 = 0;
+            $xfer += $input->readListBegin($_etype819, $_size816);
+            for ($_i820 = 0; $_i820 < $_size816; ++$_i820)
             {
-              $elem761 = null;
-              $xfer += $input->readString($elem761);
-              $this->part_vals []= $elem761;
+              $elem821 = null;
+              $xfer += $input->readString($elem821);
+              $this->part_vals []= $elem821;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24248,9 +24472,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter762)
+          foreach ($this->part_vals as $iter822)
           {
-            $xfer += $output->writeString($iter762);
+            $xfer += $output->writeString($iter822);
           }
         }
         $output->writeListEnd();
@@ -24704,14 +24928,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size763 = 0;
-            $_etype766 = 0;
-            $xfer += $input->readListBegin($_etype766, $_size763);
-            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
+            $_size823 = 0;
+            $_etype826 = 0;
+            $xfer += $input->readListBegin($_etype826, $_size823);
+            for ($_i827 = 0; $_i827 < $_size823; ++$_i827)
             {
-              $elem768 = null;
-              $xfer += $input->readString($elem768);
-              $this->success []= $elem768;
+              $elem828 = null;
+              $xfer += $input->readString($elem828);
+              $this->success []= $elem828;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24747,9 +24971,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter769)
+          foreach ($this->success as $iter829)
           {
-            $xfer += $output->writeString($iter769);
+            $xfer += $output->writeString($iter829);
           }
         }
         $output->writeListEnd();
@@ -24909,17 +25133,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size770 = 0;
-            $_ktype771 = 0;
-            $_vtype772 = 0;
-            $xfer += $input->readMapBegin($_ktype771, $_vtype772, $_size770);
-            for ($_i774 = 0; $_i774 < $_size770; ++$_i774)
+            $_size830 = 0;
+            $_ktype831 = 0;
+            $_vtype832 = 0;
+            $xfer += $input->readMapBegin($_ktype831, $_vtype832, $_size830);
+            for ($_i834 = 0; $_i834 < $_size830; ++$_i834)
             {
-              $key775 = '';
-              $val776 = '';
-              $xfer += $input->readString($key775);
-              $xfer += $input->readString($val776);
-              $this->success[$key775] = $val776;
+              $key835 = '';
+              $val836 = '';
+              $xfer += $input->readString($key835);
+              $xfer += $input->readString($val836);
+              $this->success[$key835] = $val836;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -24955,10 +25179,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter777 => $viter778)
+          foreach ($this->success as $kiter837 => $viter838)
           {
-            $xfer += $output->writeString($kiter777);
-            $xfer += $output->writeString($viter778);
+            $xfer += $output->writeString($kiter837);
+            $xfer += $output->writeString($viter838);
           }
         }
         $output->writeMapEnd();
@@ -25078,17 +25302,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size779 = 0;
-            $_ktype780 = 0;
-            $_vtype781 = 0;
-            $xfer += $input->readMapBegin($_ktype780, $_vtype781, $_size779);
-            for ($_i783 = 0; $_i783 < $_size779; ++$_i783)
+            $_size839 = 0;
+            $_ktype840 = 0;
+            $_vtype841 = 0;
+            $xfer += $input->readMapBegin($_ktype840, $_vtype841, $_size839);
+            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
             {
-              $key784 = '';
-              $val785 = '';
-              $xfer += $input->readString($key784);
-              $xfer += $input->readString($val785);
-              $this->part_vals[$key784] = $val785;
+              $key844 = '';
+              $val845 = '';
+              $xfer += $input->readString($key844);
+              $xfer += $input->readString($val845);
+              $this->part_vals[$key844] = $val845;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25133,10 +25357,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter786 => $viter787)
+          foreach ($this->part_vals as $kiter846 => $viter847)
           {
-            $xfer += $output->writeString($kiter786);
-            $xfer += $output->writeString($viter787);
+            $xfer += $output->writeString($kiter846);
+            $xfer += $output->writeString($viter847);
           }
         }
         $output->writeMapEnd();
@@ -25458,17 +25682,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size788 = 0;
-            $_ktype789 = 0;
-            $_vtype790 = 0;
-            $xfer += $input->readMapBegin($_ktype789, $_vtype790, $_size788);
-            for ($_i792 = 0; $_i792 < $_size788; ++$_i792)
+            $_size848 = 0;
+            $_ktype849 = 0;
+            $_vtype850 = 0;
+            $xfer += $input->readMapBegin($_ktype849, $_vtype850, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
             {
-              $key793 = '';
-              $val794 = '';
-              $xfer += $input->readString($key793);
-              $xfer += $input->readString($val794);
-              $this->part_vals[$key793] = $val794;
+              $key853 = '';
+              $val854 = '';
+              $xfer += $input->readString($key853);
+              $xfer += $input->readString($val854);
+              $this->part_vals[$key853] = $val854;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25513,10 +25737,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter795 => $viter796)
+          foreach ($this->part_vals as $kiter855 => $viter856)
           {
-            $xfer += $output->writeString($kiter795);
-            $xfer += $output->writeString($viter796);
+            $xfer += $output->writeString($kiter855);
+            $xfer += $output->writeString($viter856);
           }
         }
         $output->writeMapEnd();
@@ -26990,15 +27214,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size797 = 0;
-            $_etype800 = 0;
-            $xfer += $input->readListBegin($_etype800, $_size797);
-            for ($_i801 = 0; $_i801 < $_size797; ++$_i801)
+            $_size857 = 0;
+            $_etype860 = 0;
+            $xfer += $input->readListBegin($_etype860, $_size857);
+            for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
             {
-              $elem802 = null;
-              $elem802 = new \metastore\Index();
-              $xfer += $elem802->read($input);
-              $this->success []= $elem802;
+              $elem862 = null;
+              $elem862 = new \metastore\Index();
+              $xfer += $elem862->read($input);
+              $this->success []= $elem862;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27042,9 +27266,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter803)
+          foreach ($this->success as $iter863)
           {
-            $xfer += $iter803->write($output);
+            $xfer += $iter863->write($output);
           }
         }
         $output->writeListEnd();
@@ -27251,14 +27475,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size804 = 0;
-            $_etype807 = 0;
-            $xfer += $input->readListBegin($_etype807, $_size804);
-            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
+            $_size864 = 0;
+            $_etype867 = 0;
+            $xfer += $input->readListBegin($_etype867, $_size864);
+            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
             {
-              $elem809 = null;
-              $xfer += $input->readString($elem809);
-              $this->success []= $elem809;
+              $elem869 = null;
+              $xfer += $input->readString($elem869);
+              $this->success []= $elem869;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27294,9 +27518,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter810)
+          foreach ($this->success as $iter870)
           {
-            $xfer += $output->writeString($iter810);
+            $xfer += $output->writeString($iter870);
           }
         }
         $output->writeListEnd();
@@ -30770,14 +30994,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size811 = 0;
-            $_etype814 = 0;
-            $xfer += $input->readListBegin($_etype814, $_size811);
-            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
+            $_size871 = 0;
+            $_etype874 = 0;
+            $xfer += $input->readListBegin($_etype874, $_size871);
+            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
             {
-              $elem816 = null;
-              $xfer += $input->readString($elem816);
-              $this->success []= $elem816;
+              $elem876 = null;
+              $xfer += $input->readString($elem876);
+              $this->success []= $elem876;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30813,9 +31037,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter817)
+          foreach ($this->success as $iter877)
           {
-            $xfer += $output->writeString($iter817);
+            $xfer += $output->writeString($iter877);
           }
         }
         $output->writeListEnd();
@@ -31684,14 +31908,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size818 = 0;
-            $_etype821 = 0;
-            $xfer += $input->readListBegin($_etype821, $_size818);
-            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
+            $_size878 = 0;
+            $_etype881 = 0;
+            $xfer += $input->readListBegin($_etype881, $_size878);
+            for ($_i882 = 0; $_i882 < $_size878; ++$_i882)
             {
-              $elem823 = null;
-              $xfer += $input->readString($elem823);
-              $this->success []= $elem823;
+              $elem883 = null;
+              $xfer += $input->readString($elem883);
+              $this->success []= $elem883;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31727,9 +31951,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter824)
+          foreach ($this->success as $iter884)
           {
-            $xfer += $output->writeString($iter824);
+            $xfer += $output->writeString($iter884);
           }
         }
         $output->writeListEnd();
@@ -32420,15 +32644,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size825 = 0;
-            $_etype828 = 0;
-            $xfer += $input->readListBegin($_etype828, $_size825);
-            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
+            $_size885 

<TRUNCATED>

[47/52] [abbrv] hive git commit: HIVE-11711: Merge hbase-metastore branch to trunk

Posted by se...@apache.org.
HIVE-11711: Merge hbase-metastore branch to trunk


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

Branch: refs/heads/llap
Commit: 4c17ecfda525f2f65a29dab40563c50267e46eba
Parents: 76828e0
Author: Daniel Dai <da...@hortonworks.com>
Authored: Mon Sep 21 21:54:52 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Mon Sep 21 21:54:52 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java     | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4c17ecfd/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 1d98766..0d07173 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -177,7 +177,23 @@ public class HiveConf extends Configuration {
       HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT,
       HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT,
       HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_FULL,
-      HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_CLEAN_UNTIL
+      HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_CLEAN_UNTIL,
+      HiveConf.ConfVars.METASTORE_FASTPATH,
+      HiveConf.ConfVars.METASTORE_HBASE_CATALOG_CACHE_SIZE,
+      HiveConf.ConfVars.METASTORE_HBASE_AGGREGATE_STATS_CACHE_SIZE,
+      HiveConf.ConfVars.METASTORE_HBASE_AGGREGATE_STATS_CACHE_MAX_PARTITIONS,
+      HiveConf.ConfVars.METASTORE_HBASE_AGGREGATE_STATS_CACHE_FALSE_POSITIVE_PROBABILITY,
+      HiveConf.ConfVars.METASTORE_HBASE_AGGREGATE_STATS_CACHE_MAX_VARIANCE,
+      HiveConf.ConfVars.METASTORE_HBASE_CACHE_TIME_TO_LIVE,
+      HiveConf.ConfVars.METASTORE_HBASE_CACHE_MAX_WRITER_WAIT,
+      HiveConf.ConfVars.METASTORE_HBASE_CACHE_MAX_READER_WAIT,
+      HiveConf.ConfVars.METASTORE_HBASE_CACHE_MAX_FULL,
+      HiveConf.ConfVars.METASTORE_HBASE_CACHE_CLEAN_UNTIL,
+      HiveConf.ConfVars.METASTORE_HBASE_CONNECTION_CLASS,
+      HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_CACHE_ENTRIES,
+      HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_MEMORY_TTL,
+      HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_INVALIDATOR_FREQUENCY,
+      HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_HBASE_TTL
       };
 
   /**


[28/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java
index fe60838..ecff000 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TxnAbortedException extends TException implements org.apache.thrift.TBase<TxnAbortedException, TxnAbortedException._Fields>, java.io.Serializable, Cloneable, Comparable<TxnAbortedException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnAbortedException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java
index 266fbe1..0828397 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TxnInfo implements org.apache.thrift.TBase<TxnInfo, TxnInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TxnInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnInfo");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java
index 18db1b8..50da426 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class TxnOpenException extends TException implements org.apache.thrift.TBase<TxnOpenException, TxnOpenException._Fields>, java.io.Serializable, Cloneable, Comparable<TxnOpenException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnOpenException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
index b330ce2..309abe4 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Type implements org.apache.thrift.TBase<Type, Type._Fields>, java.io.Serializable, Cloneable, Comparable<Type> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Type");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
index b7623ca..cdb1671 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class UnknownDBException extends TException implements org.apache.thrift.TBase<UnknownDBException, UnknownDBException._Fields>, java.io.Serializable, Cloneable, Comparable<UnknownDBException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnknownDBException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java
index bdd674b..c767367 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownPartitionException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class UnknownPartitionException extends TException implements org.apache.thrift.TBase<UnknownPartitionException, UnknownPartitionException._Fields>, java.io.Serializable, Cloneable, Comparable<UnknownPartitionException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnknownPartitionException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
index 768eb65..1d0f347 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class UnknownTableException extends TException implements org.apache.thrift.TBase<UnknownTableException, UnknownTableException._Fields>, java.io.Serializable, Cloneable, Comparable<UnknownTableException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnknownTableException");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java
index 395c15f..568a744 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class UnlockRequest implements org.apache.thrift.TBase<UnlockRequest, UnlockRequest._Fields>, java.io.Serializable, Cloneable, Comparable<UnlockRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnlockRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
index f3cbb74..8d0daa5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class Version implements org.apache.thrift.TBase<Version, Version._Fields>, java.io.Serializable, Cloneable, Comparable<Version> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Version");
 


[41/52] [abbrv] hive git commit: HIVE-11694: Exclude hbase-metastore for hadoop-1 (Daniel Dai reviewed by Thejas Nair)

Posted by se...@apache.org.
HIVE-11694: Exclude hbase-metastore for hadoop-1 (Daniel Dai reviewed by Thejas Nair)


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

Branch: refs/heads/llap
Commit: 3d170cae5f41b0a3928117328aeaac3aefbc7cef
Parents: fbbb7cf
Author: Daniel Dai <da...@hortonworks.com>
Authored: Tue Sep 1 12:43:42 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Tue Sep 1 12:43:42 2015 -0700

----------------------------------------------------------------------
 metastore/pom.xml | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3d170cae/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index ccec9f1..255726c 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -159,6 +159,23 @@
   <profiles>
     <profile>
       <id>hadoop-1</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <version>2.3.2</version>
+            <configuration>
+              <excludes>
+                <exclude>**/hbase/**</exclude>
+              </excludes>
+              <testExcludes>
+                <exclude>**/hbase/**</exclude>
+              </testExcludes>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>


[18/52] [abbrv] hive git commit: HIVE-11568 : merge master into branch (Sergey Shelukhin) ADDENDUM MERGE

Posted by se...@apache.org.
HIVE-11568 : merge master into branch (Sergey Shelukhin) ADDENDUM MERGE


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

Branch: refs/heads/llap
Commit: 2fe60861db72a4128448245f8031d1839e5e3f8e
Parents: c528294 3071ce9
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Aug 14 16:13:32 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Aug 14 16:13:32 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/DDLSemanticAnalyzer.java   | 11 +++++++++++
 .../clientnegative/alter_table_wrong_location.q     |  4 ++++
 .../clientnegative/alter_table_wrong_location.q.out |  9 +++++++++
 .../apache/hive/spark/client/SparkClientImpl.java   |  4 ++++
 testutils/ptest2/pom.xml                            | 16 +---------------
 testutils/ptest2/src/main/resources/log4j2.xml      |  1 +
 6 files changed, 30 insertions(+), 15 deletions(-)
----------------------------------------------------------------------



[32/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
new file mode 100644
index 0000000..3d69606
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -0,0 +1,548 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<GetFileMetadataByExprRequest, GetFileMetadataByExprRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetFileMetadataByExprRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFileMetadataByExprRequest");
+
+  private static final org.apache.thrift.protocol.TField FILE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("fileIds", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField EXPR_FIELD_DESC = new org.apache.thrift.protocol.TField("expr", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetFileMetadataByExprRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetFileMetadataByExprRequestTupleSchemeFactory());
+  }
+
+  private List<Long> fileIds; // required
+  private ByteBuffer expr; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    FILE_IDS((short)1, "fileIds"),
+    EXPR((short)2, "expr");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FILE_IDS
+          return FILE_IDS;
+        case 2: // EXPR
+          return EXPR;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FILE_IDS, new org.apache.thrift.meta_data.FieldMetaData("fileIds", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.EXPR, new org.apache.thrift.meta_data.FieldMetaData("expr", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFileMetadataByExprRequest.class, metaDataMap);
+  }
+
+  public GetFileMetadataByExprRequest() {
+  }
+
+  public GetFileMetadataByExprRequest(
+    List<Long> fileIds,
+    ByteBuffer expr)
+  {
+    this();
+    this.fileIds = fileIds;
+    this.expr = org.apache.thrift.TBaseHelper.copyBinary(expr);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetFileMetadataByExprRequest(GetFileMetadataByExprRequest other) {
+    if (other.isSetFileIds()) {
+      List<Long> __this__fileIds = new ArrayList<Long>(other.fileIds);
+      this.fileIds = __this__fileIds;
+    }
+    if (other.isSetExpr()) {
+      this.expr = org.apache.thrift.TBaseHelper.copyBinary(other.expr);
+    }
+  }
+
+  public GetFileMetadataByExprRequest deepCopy() {
+    return new GetFileMetadataByExprRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.fileIds = null;
+    this.expr = null;
+  }
+
+  public int getFileIdsSize() {
+    return (this.fileIds == null) ? 0 : this.fileIds.size();
+  }
+
+  public java.util.Iterator<Long> getFileIdsIterator() {
+    return (this.fileIds == null) ? null : this.fileIds.iterator();
+  }
+
+  public void addToFileIds(long elem) {
+    if (this.fileIds == null) {
+      this.fileIds = new ArrayList<Long>();
+    }
+    this.fileIds.add(elem);
+  }
+
+  public List<Long> getFileIds() {
+    return this.fileIds;
+  }
+
+  public void setFileIds(List<Long> fileIds) {
+    this.fileIds = fileIds;
+  }
+
+  public void unsetFileIds() {
+    this.fileIds = null;
+  }
+
+  /** Returns true if field fileIds is set (has been assigned a value) and false otherwise */
+  public boolean isSetFileIds() {
+    return this.fileIds != null;
+  }
+
+  public void setFileIdsIsSet(boolean value) {
+    if (!value) {
+      this.fileIds = null;
+    }
+  }
+
+  public byte[] getExpr() {
+    setExpr(org.apache.thrift.TBaseHelper.rightSize(expr));
+    return expr == null ? null : expr.array();
+  }
+
+  public ByteBuffer bufferForExpr() {
+    return org.apache.thrift.TBaseHelper.copyBinary(expr);
+  }
+
+  public void setExpr(byte[] expr) {
+    this.expr = expr == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(expr, expr.length));
+  }
+
+  public void setExpr(ByteBuffer expr) {
+    this.expr = org.apache.thrift.TBaseHelper.copyBinary(expr);
+  }
+
+  public void unsetExpr() {
+    this.expr = null;
+  }
+
+  /** Returns true if field expr is set (has been assigned a value) and false otherwise */
+  public boolean isSetExpr() {
+    return this.expr != null;
+  }
+
+  public void setExprIsSet(boolean value) {
+    if (!value) {
+      this.expr = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FILE_IDS:
+      if (value == null) {
+        unsetFileIds();
+      } else {
+        setFileIds((List<Long>)value);
+      }
+      break;
+
+    case EXPR:
+      if (value == null) {
+        unsetExpr();
+      } else {
+        setExpr((ByteBuffer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FILE_IDS:
+      return getFileIds();
+
+    case EXPR:
+      return getExpr();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case FILE_IDS:
+      return isSetFileIds();
+    case EXPR:
+      return isSetExpr();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetFileMetadataByExprRequest)
+      return this.equals((GetFileMetadataByExprRequest)that);
+    return false;
+  }
+
+  public boolean equals(GetFileMetadataByExprRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_fileIds = true && this.isSetFileIds();
+    boolean that_present_fileIds = true && that.isSetFileIds();
+    if (this_present_fileIds || that_present_fileIds) {
+      if (!(this_present_fileIds && that_present_fileIds))
+        return false;
+      if (!this.fileIds.equals(that.fileIds))
+        return false;
+    }
+
+    boolean this_present_expr = true && this.isSetExpr();
+    boolean that_present_expr = true && that.isSetExpr();
+    if (this_present_expr || that_present_expr) {
+      if (!(this_present_expr && that_present_expr))
+        return false;
+      if (!this.expr.equals(that.expr))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_fileIds = true && (isSetFileIds());
+    list.add(present_fileIds);
+    if (present_fileIds)
+      list.add(fileIds);
+
+    boolean present_expr = true && (isSetExpr());
+    list.add(present_expr);
+    if (present_expr)
+      list.add(expr);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetFileMetadataByExprRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetFileIds()).compareTo(other.isSetFileIds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFileIds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileIds, other.fileIds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetExpr()).compareTo(other.isSetExpr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetExpr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.expr, other.expr);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetFileMetadataByExprRequest(");
+    boolean first = true;
+
+    sb.append("fileIds:");
+    if (this.fileIds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fileIds);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("expr:");
+    if (this.expr == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.expr, sb);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetFileIds()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'fileIds' is unset! Struct:" + toString());
+    }
+
+    if (!isSetExpr()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'expr' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetFileMetadataByExprRequestStandardSchemeFactory implements SchemeFactory {
+    public GetFileMetadataByExprRequestStandardScheme getScheme() {
+      return new GetFileMetadataByExprRequestStandardScheme();
+    }
+  }
+
+  private static class GetFileMetadataByExprRequestStandardScheme extends StandardScheme<GetFileMetadataByExprRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // FILE_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list542 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list542.size);
+                long _elem543;
+                for (int _i544 = 0; _i544 < _list542.size; ++_i544)
+                {
+                  _elem543 = iprot.readI64();
+                  struct.fileIds.add(_elem543);
+                }
+                iprot.readListEnd();
+              }
+              struct.setFileIdsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // EXPR
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.expr = iprot.readBinary();
+              struct.setExprIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.fileIds != null) {
+        oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
+          for (long _iter545 : struct.fileIds)
+          {
+            oprot.writeI64(_iter545);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.expr != null) {
+        oprot.writeFieldBegin(EXPR_FIELD_DESC);
+        oprot.writeBinary(struct.expr);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetFileMetadataByExprRequestTupleSchemeFactory implements SchemeFactory {
+    public GetFileMetadataByExprRequestTupleScheme getScheme() {
+      return new GetFileMetadataByExprRequestTupleScheme();
+    }
+  }
+
+  private static class GetFileMetadataByExprRequestTupleScheme extends TupleScheme<GetFileMetadataByExprRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.fileIds.size());
+        for (long _iter546 : struct.fileIds)
+        {
+          oprot.writeI64(_iter546);
+        }
+      }
+      oprot.writeBinary(struct.expr);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list547.size);
+        long _elem548;
+        for (int _i549 = 0; _i549 < _list547.size; ++_i549)
+        {
+          _elem548 = iprot.readI64();
+          struct.fileIds.add(_elem548);
+        }
+      }
+      struct.setFileIdsIsSet(true);
+      struct.expr = iprot.readBinary();
+      struct.setExprIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
new file mode 100644
index 0000000..3ac9921
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -0,0 +1,703 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetFileMetadataByExprResult, GetFileMetadataByExprResult._Fields>, java.io.Serializable, Cloneable, Comparable<GetFileMetadataByExprResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFileMetadataByExprResult");
+
+  private static final org.apache.thrift.protocol.TField METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("metadata", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField IS_SUPPORTED_FIELD_DESC = new org.apache.thrift.protocol.TField("isSupported", org.apache.thrift.protocol.TType.BOOL, (short)2);
+  private static final org.apache.thrift.protocol.TField UNKNOWN_FILE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("unknownFileIds", org.apache.thrift.protocol.TType.LIST, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetFileMetadataByExprResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetFileMetadataByExprResultTupleSchemeFactory());
+  }
+
+  private Map<Long,MetadataPpdResult> metadata; // required
+  private boolean isSupported; // required
+  private List<Long> unknownFileIds; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    METADATA((short)1, "metadata"),
+    IS_SUPPORTED((short)2, "isSupported"),
+    UNKNOWN_FILE_IDS((short)3, "unknownFileIds");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // METADATA
+          return METADATA;
+        case 2: // IS_SUPPORTED
+          return IS_SUPPORTED;
+        case 3: // UNKNOWN_FILE_IDS
+          return UNKNOWN_FILE_IDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ISSUPPORTED_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.METADATA, new org.apache.thrift.meta_data.FieldMetaData("metadata", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetadataPpdResult.class))));
+    tmpMap.put(_Fields.IS_SUPPORTED, new org.apache.thrift.meta_data.FieldMetaData("isSupported", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.UNKNOWN_FILE_IDS, new org.apache.thrift.meta_data.FieldMetaData("unknownFileIds", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFileMetadataByExprResult.class, metaDataMap);
+  }
+
+  public GetFileMetadataByExprResult() {
+  }
+
+  public GetFileMetadataByExprResult(
+    Map<Long,MetadataPpdResult> metadata,
+    boolean isSupported,
+    List<Long> unknownFileIds)
+  {
+    this();
+    this.metadata = metadata;
+    this.isSupported = isSupported;
+    setIsSupportedIsSet(true);
+    this.unknownFileIds = unknownFileIds;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetFileMetadataByExprResult(GetFileMetadataByExprResult other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetMetadata()) {
+      Map<Long,MetadataPpdResult> __this__metadata = new HashMap<Long,MetadataPpdResult>(other.metadata.size());
+      for (Map.Entry<Long, MetadataPpdResult> other_element : other.metadata.entrySet()) {
+
+        Long other_element_key = other_element.getKey();
+        MetadataPpdResult other_element_value = other_element.getValue();
+
+        Long __this__metadata_copy_key = other_element_key;
+
+        MetadataPpdResult __this__metadata_copy_value = new MetadataPpdResult(other_element_value);
+
+        __this__metadata.put(__this__metadata_copy_key, __this__metadata_copy_value);
+      }
+      this.metadata = __this__metadata;
+    }
+    this.isSupported = other.isSupported;
+    if (other.isSetUnknownFileIds()) {
+      List<Long> __this__unknownFileIds = new ArrayList<Long>(other.unknownFileIds);
+      this.unknownFileIds = __this__unknownFileIds;
+    }
+  }
+
+  public GetFileMetadataByExprResult deepCopy() {
+    return new GetFileMetadataByExprResult(this);
+  }
+
+  @Override
+  public void clear() {
+    this.metadata = null;
+    setIsSupportedIsSet(false);
+    this.isSupported = false;
+    this.unknownFileIds = null;
+  }
+
+  public int getMetadataSize() {
+    return (this.metadata == null) ? 0 : this.metadata.size();
+  }
+
+  public void putToMetadata(long key, MetadataPpdResult val) {
+    if (this.metadata == null) {
+      this.metadata = new HashMap<Long,MetadataPpdResult>();
+    }
+    this.metadata.put(key, val);
+  }
+
+  public Map<Long,MetadataPpdResult> getMetadata() {
+    return this.metadata;
+  }
+
+  public void setMetadata(Map<Long,MetadataPpdResult> metadata) {
+    this.metadata = metadata;
+  }
+
+  public void unsetMetadata() {
+    this.metadata = null;
+  }
+
+  /** Returns true if field metadata is set (has been assigned a value) and false otherwise */
+  public boolean isSetMetadata() {
+    return this.metadata != null;
+  }
+
+  public void setMetadataIsSet(boolean value) {
+    if (!value) {
+      this.metadata = null;
+    }
+  }
+
+  public boolean isIsSupported() {
+    return this.isSupported;
+  }
+
+  public void setIsSupported(boolean isSupported) {
+    this.isSupported = isSupported;
+    setIsSupportedIsSet(true);
+  }
+
+  public void unsetIsSupported() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID);
+  }
+
+  /** Returns true if field isSupported is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsSupported() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID);
+  }
+
+  public void setIsSupportedIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID, value);
+  }
+
+  public int getUnknownFileIdsSize() {
+    return (this.unknownFileIds == null) ? 0 : this.unknownFileIds.size();
+  }
+
+  public java.util.Iterator<Long> getUnknownFileIdsIterator() {
+    return (this.unknownFileIds == null) ? null : this.unknownFileIds.iterator();
+  }
+
+  public void addToUnknownFileIds(long elem) {
+    if (this.unknownFileIds == null) {
+      this.unknownFileIds = new ArrayList<Long>();
+    }
+    this.unknownFileIds.add(elem);
+  }
+
+  public List<Long> getUnknownFileIds() {
+    return this.unknownFileIds;
+  }
+
+  public void setUnknownFileIds(List<Long> unknownFileIds) {
+    this.unknownFileIds = unknownFileIds;
+  }
+
+  public void unsetUnknownFileIds() {
+    this.unknownFileIds = null;
+  }
+
+  /** Returns true if field unknownFileIds is set (has been assigned a value) and false otherwise */
+  public boolean isSetUnknownFileIds() {
+    return this.unknownFileIds != null;
+  }
+
+  public void setUnknownFileIdsIsSet(boolean value) {
+    if (!value) {
+      this.unknownFileIds = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case METADATA:
+      if (value == null) {
+        unsetMetadata();
+      } else {
+        setMetadata((Map<Long,MetadataPpdResult>)value);
+      }
+      break;
+
+    case IS_SUPPORTED:
+      if (value == null) {
+        unsetIsSupported();
+      } else {
+        setIsSupported((Boolean)value);
+      }
+      break;
+
+    case UNKNOWN_FILE_IDS:
+      if (value == null) {
+        unsetUnknownFileIds();
+      } else {
+        setUnknownFileIds((List<Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case METADATA:
+      return getMetadata();
+
+    case IS_SUPPORTED:
+      return Boolean.valueOf(isIsSupported());
+
+    case UNKNOWN_FILE_IDS:
+      return getUnknownFileIds();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case METADATA:
+      return isSetMetadata();
+    case IS_SUPPORTED:
+      return isSetIsSupported();
+    case UNKNOWN_FILE_IDS:
+      return isSetUnknownFileIds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetFileMetadataByExprResult)
+      return this.equals((GetFileMetadataByExprResult)that);
+    return false;
+  }
+
+  public boolean equals(GetFileMetadataByExprResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_metadata = true && this.isSetMetadata();
+    boolean that_present_metadata = true && that.isSetMetadata();
+    if (this_present_metadata || that_present_metadata) {
+      if (!(this_present_metadata && that_present_metadata))
+        return false;
+      if (!this.metadata.equals(that.metadata))
+        return false;
+    }
+
+    boolean this_present_isSupported = true;
+    boolean that_present_isSupported = true;
+    if (this_present_isSupported || that_present_isSupported) {
+      if (!(this_present_isSupported && that_present_isSupported))
+        return false;
+      if (this.isSupported != that.isSupported)
+        return false;
+    }
+
+    boolean this_present_unknownFileIds = true && this.isSetUnknownFileIds();
+    boolean that_present_unknownFileIds = true && that.isSetUnknownFileIds();
+    if (this_present_unknownFileIds || that_present_unknownFileIds) {
+      if (!(this_present_unknownFileIds && that_present_unknownFileIds))
+        return false;
+      if (!this.unknownFileIds.equals(that.unknownFileIds))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_metadata = true && (isSetMetadata());
+    list.add(present_metadata);
+    if (present_metadata)
+      list.add(metadata);
+
+    boolean present_isSupported = true;
+    list.add(present_isSupported);
+    if (present_isSupported)
+      list.add(isSupported);
+
+    boolean present_unknownFileIds = true && (isSetUnknownFileIds());
+    list.add(present_unknownFileIds);
+    if (present_unknownFileIds)
+      list.add(unknownFileIds);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetFileMetadataByExprResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetMetadata()).compareTo(other.isSetMetadata());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMetadata()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metadata, other.metadata);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIsSupported()).compareTo(other.isSetIsSupported());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIsSupported()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isSupported, other.isSupported);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetUnknownFileIds()).compareTo(other.isSetUnknownFileIds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUnknownFileIds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.unknownFileIds, other.unknownFileIds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetFileMetadataByExprResult(");
+    boolean first = true;
+
+    sb.append("metadata:");
+    if (this.metadata == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.metadata);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("isSupported:");
+    sb.append(this.isSupported);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("unknownFileIds:");
+    if (this.unknownFileIds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.unknownFileIds);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetMetadata()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'metadata' is unset! Struct:" + toString());
+    }
+
+    if (!isSetIsSupported()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'isSupported' is unset! Struct:" + toString());
+    }
+
+    if (!isSetUnknownFileIds()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'unknownFileIds' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetFileMetadataByExprResultStandardSchemeFactory implements SchemeFactory {
+    public GetFileMetadataByExprResultStandardScheme getScheme() {
+      return new GetFileMetadataByExprResultStandardScheme();
+    }
+  }
+
+  private static class GetFileMetadataByExprResultStandardScheme extends StandardScheme<GetFileMetadataByExprResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // METADATA
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map524 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map524.size);
+                long _key525;
+                MetadataPpdResult _val526;
+                for (int _i527 = 0; _i527 < _map524.size; ++_i527)
+                {
+                  _key525 = iprot.readI64();
+                  _val526 = new MetadataPpdResult();
+                  _val526.read(iprot);
+                  struct.metadata.put(_key525, _val526);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setMetadataIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // IS_SUPPORTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isSupported = iprot.readBool();
+              struct.setIsSupportedIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // UNKNOWN_FILE_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list528 = iprot.readListBegin();
+                struct.unknownFileIds = new ArrayList<Long>(_list528.size);
+                long _elem529;
+                for (int _i530 = 0; _i530 < _list528.size; ++_i530)
+                {
+                  _elem529 = iprot.readI64();
+                  struct.unknownFileIds.add(_elem529);
+                }
+                iprot.readListEnd();
+              }
+              struct.setUnknownFileIdsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.metadata != null) {
+        oprot.writeFieldBegin(METADATA_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, struct.metadata.size()));
+          for (Map.Entry<Long, MetadataPpdResult> _iter531 : struct.metadata.entrySet())
+          {
+            oprot.writeI64(_iter531.getKey());
+            _iter531.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(IS_SUPPORTED_FIELD_DESC);
+      oprot.writeBool(struct.isSupported);
+      oprot.writeFieldEnd();
+      if (struct.unknownFileIds != null) {
+        oprot.writeFieldBegin(UNKNOWN_FILE_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.unknownFileIds.size()));
+          for (long _iter532 : struct.unknownFileIds)
+          {
+            oprot.writeI64(_iter532);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetFileMetadataByExprResultTupleSchemeFactory implements SchemeFactory {
+    public GetFileMetadataByExprResultTupleScheme getScheme() {
+      return new GetFileMetadataByExprResultTupleScheme();
+    }
+  }
+
+  private static class GetFileMetadataByExprResultTupleScheme extends TupleScheme<GetFileMetadataByExprResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.metadata.size());
+        for (Map.Entry<Long, MetadataPpdResult> _iter533 : struct.metadata.entrySet())
+        {
+          oprot.writeI64(_iter533.getKey());
+          _iter533.getValue().write(oprot);
+        }
+      }
+      oprot.writeBool(struct.isSupported);
+      {
+        oprot.writeI32(struct.unknownFileIds.size());
+        for (long _iter534 : struct.unknownFileIds)
+        {
+          oprot.writeI64(_iter534);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map535 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map535.size);
+        long _key536;
+        MetadataPpdResult _val537;
+        for (int _i538 = 0; _i538 < _map535.size; ++_i538)
+        {
+          _key536 = iprot.readI64();
+          _val537 = new MetadataPpdResult();
+          _val537.read(iprot);
+          struct.metadata.put(_key536, _val537);
+        }
+      }
+      struct.setMetadataIsSet(true);
+      struct.isSupported = iprot.readBool();
+      struct.setIsSupportedIsSet(true);
+      {
+        org.apache.thrift.protocol.TList _list539 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.unknownFileIds = new ArrayList<Long>(_list539.size);
+        long _elem540;
+        for (int _i541 = 0; _i541 < _list539.size; ++_i541)
+        {
+          _elem540 = iprot.readI64();
+          struct.unknownFileIds.add(_elem540);
+        }
+      }
+      struct.setUnknownFileIdsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
new file mode 100644
index 0000000..e4cd1c4
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -0,0 +1,438 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMetadataRequest, GetFileMetadataRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetFileMetadataRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFileMetadataRequest");
+
+  private static final org.apache.thrift.protocol.TField FILE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("fileIds", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetFileMetadataRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetFileMetadataRequestTupleSchemeFactory());
+  }
+
+  private List<Long> fileIds; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    FILE_IDS((short)1, "fileIds");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FILE_IDS
+          return FILE_IDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FILE_IDS, new org.apache.thrift.meta_data.FieldMetaData("fileIds", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFileMetadataRequest.class, metaDataMap);
+  }
+
+  public GetFileMetadataRequest() {
+  }
+
+  public GetFileMetadataRequest(
+    List<Long> fileIds)
+  {
+    this();
+    this.fileIds = fileIds;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetFileMetadataRequest(GetFileMetadataRequest other) {
+    if (other.isSetFileIds()) {
+      List<Long> __this__fileIds = new ArrayList<Long>(other.fileIds);
+      this.fileIds = __this__fileIds;
+    }
+  }
+
+  public GetFileMetadataRequest deepCopy() {
+    return new GetFileMetadataRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.fileIds = null;
+  }
+
+  public int getFileIdsSize() {
+    return (this.fileIds == null) ? 0 : this.fileIds.size();
+  }
+
+  public java.util.Iterator<Long> getFileIdsIterator() {
+    return (this.fileIds == null) ? null : this.fileIds.iterator();
+  }
+
+  public void addToFileIds(long elem) {
+    if (this.fileIds == null) {
+      this.fileIds = new ArrayList<Long>();
+    }
+    this.fileIds.add(elem);
+  }
+
+  public List<Long> getFileIds() {
+    return this.fileIds;
+  }
+
+  public void setFileIds(List<Long> fileIds) {
+    this.fileIds = fileIds;
+  }
+
+  public void unsetFileIds() {
+    this.fileIds = null;
+  }
+
+  /** Returns true if field fileIds is set (has been assigned a value) and false otherwise */
+  public boolean isSetFileIds() {
+    return this.fileIds != null;
+  }
+
+  public void setFileIdsIsSet(boolean value) {
+    if (!value) {
+      this.fileIds = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FILE_IDS:
+      if (value == null) {
+        unsetFileIds();
+      } else {
+        setFileIds((List<Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FILE_IDS:
+      return getFileIds();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case FILE_IDS:
+      return isSetFileIds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetFileMetadataRequest)
+      return this.equals((GetFileMetadataRequest)that);
+    return false;
+  }
+
+  public boolean equals(GetFileMetadataRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_fileIds = true && this.isSetFileIds();
+    boolean that_present_fileIds = true && that.isSetFileIds();
+    if (this_present_fileIds || that_present_fileIds) {
+      if (!(this_present_fileIds && that_present_fileIds))
+        return false;
+      if (!this.fileIds.equals(that.fileIds))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_fileIds = true && (isSetFileIds());
+    list.add(present_fileIds);
+    if (present_fileIds)
+      list.add(fileIds);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetFileMetadataRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetFileIds()).compareTo(other.isSetFileIds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFileIds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileIds, other.fileIds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetFileMetadataRequest(");
+    boolean first = true;
+
+    sb.append("fileIds:");
+    if (this.fileIds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fileIds);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetFileIds()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'fileIds' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetFileMetadataRequestStandardSchemeFactory implements SchemeFactory {
+    public GetFileMetadataRequestStandardScheme getScheme() {
+      return new GetFileMetadataRequestStandardScheme();
+    }
+  }
+
+  private static class GetFileMetadataRequestStandardScheme extends StandardScheme<GetFileMetadataRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // FILE_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list560 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list560.size);
+                long _elem561;
+                for (int _i562 = 0; _i562 < _list560.size; ++_i562)
+                {
+                  _elem561 = iprot.readI64();
+                  struct.fileIds.add(_elem561);
+                }
+                iprot.readListEnd();
+              }
+              struct.setFileIdsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.fileIds != null) {
+        oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
+          for (long _iter563 : struct.fileIds)
+          {
+            oprot.writeI64(_iter563);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetFileMetadataRequestTupleSchemeFactory implements SchemeFactory {
+    public GetFileMetadataRequestTupleScheme getScheme() {
+      return new GetFileMetadataRequestTupleScheme();
+    }
+  }
+
+  private static class GetFileMetadataRequestTupleScheme extends TupleScheme<GetFileMetadataRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.fileIds.size());
+        for (long _iter564 : struct.fileIds)
+        {
+          oprot.writeI64(_iter564);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list565 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list565.size);
+        long _elem566;
+        for (int _i567 = 0; _i567 < _list565.size; ++_i567)
+        {
+          _elem566 = iprot.readI64();
+          struct.fileIds.add(_elem566);
+        }
+      }
+      struct.setFileIdsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
new file mode 100644
index 0000000..a7d01e1
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -0,0 +1,540 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
+public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMetadataResult, GetFileMetadataResult._Fields>, java.io.Serializable, Cloneable, Comparable<GetFileMetadataResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFileMetadataResult");
+
+  private static final org.apache.thrift.protocol.TField METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("metadata", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField IS_SUPPORTED_FIELD_DESC = new org.apache.thrift.protocol.TField("isSupported", org.apache.thrift.protocol.TType.BOOL, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetFileMetadataResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetFileMetadataResultTupleSchemeFactory());
+  }
+
+  private Map<Long,ByteBuffer> metadata; // required
+  private boolean isSupported; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    METADATA((short)1, "metadata"),
+    IS_SUPPORTED((short)2, "isSupported");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // METADATA
+          return METADATA;
+        case 2: // IS_SUPPORTED
+          return IS_SUPPORTED;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ISSUPPORTED_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.METADATA, new org.apache.thrift.meta_data.FieldMetaData("metadata", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
+    tmpMap.put(_Fields.IS_SUPPORTED, new org.apache.thrift.meta_data.FieldMetaData("isSupported", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFileMetadataResult.class, metaDataMap);
+  }
+
+  public GetFileMetadataResult() {
+  }
+
+  public GetFileMetadataResult(
+    Map<Long,ByteBuffer> metadata,
+    boolean isSupported)
+  {
+    this();
+    this.metadata = metadata;
+    this.isSupported = isSupported;
+    setIsSupportedIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetFileMetadataResult(GetFileMetadataResult other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetMetadata()) {
+      Map<Long,ByteBuffer> __this__metadata = new HashMap<Long,ByteBuffer>(other.metadata);
+      this.metadata = __this__metadata;
+    }
+    this.isSupported = other.isSupported;
+  }
+
+  public GetFileMetadataResult deepCopy() {
+    return new GetFileMetadataResult(this);
+  }
+
+  @Override
+  public void clear() {
+    this.metadata = null;
+    setIsSupportedIsSet(false);
+    this.isSupported = false;
+  }
+
+  public int getMetadataSize() {
+    return (this.metadata == null) ? 0 : this.metadata.size();
+  }
+
+  public void putToMetadata(long key, ByteBuffer val) {
+    if (this.metadata == null) {
+      this.metadata = new HashMap<Long,ByteBuffer>();
+    }
+    this.metadata.put(key, val);
+  }
+
+  public Map<Long,ByteBuffer> getMetadata() {
+    return this.metadata;
+  }
+
+  public void setMetadata(Map<Long,ByteBuffer> metadata) {
+    this.metadata = metadata;
+  }
+
+  public void unsetMetadata() {
+    this.metadata = null;
+  }
+
+  /** Returns true if field metadata is set (has been assigned a value) and false otherwise */
+  public boolean isSetMetadata() {
+    return this.metadata != null;
+  }
+
+  public void setMetadataIsSet(boolean value) {
+    if (!value) {
+      this.metadata = null;
+    }
+  }
+
+  public boolean isIsSupported() {
+    return this.isSupported;
+  }
+
+  public void setIsSupported(boolean isSupported) {
+    this.isSupported = isSupported;
+    setIsSupportedIsSet(true);
+  }
+
+  public void unsetIsSupported() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID);
+  }
+
+  /** Returns true if field isSupported is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsSupported() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID);
+  }
+
+  public void setIsSupportedIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case METADATA:
+      if (value == null) {
+        unsetMetadata();
+      } else {
+        setMetadata((Map<Long,ByteBuffer>)value);
+      }
+      break;
+
+    case IS_SUPPORTED:
+      if (value == null) {
+        unsetIsSupported();
+      } else {
+        setIsSupported((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case METADATA:
+      return getMetadata();
+
+    case IS_SUPPORTED:
+      return Boolean.valueOf(isIsSupported());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case METADATA:
+      return isSetMetadata();
+    case IS_SUPPORTED:
+      return isSetIsSupported();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetFileMetadataResult)
+      return this.equals((GetFileMetadataResult)that);
+    return false;
+  }
+
+  public boolean equals(GetFileMetadataResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_metadata = true && this.isSetMetadata();
+    boolean that_present_metadata = true && that.isSetMetadata();
+    if (this_present_metadata || that_present_metadata) {
+      if (!(this_present_metadata && that_present_metadata))
+        return false;
+      if (!this.metadata.equals(that.metadata))
+        return false;
+    }
+
+    boolean this_present_isSupported = true;
+    boolean that_present_isSupported = true;
+    if (this_present_isSupported || that_present_isSupported) {
+      if (!(this_present_isSupported && that_present_isSupported))
+        return false;
+      if (this.isSupported != that.isSupported)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_metadata = true && (isSetMetadata());
+    list.add(present_metadata);
+    if (present_metadata)
+      list.add(metadata);
+
+    boolean present_isSupported = true;
+    list.add(present_isSupported);
+    if (present_isSupported)
+      list.add(isSupported);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetFileMetadataResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetMetadata()).compareTo(other.isSetMetadata());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMetadata()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metadata, other.metadata);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIsSupported()).compareTo(other.isSetIsSupported());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIsSupported()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isSupported, other.isSupported);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetFileMetadataResult(");
+    boolean first = true;
+
+    sb.append("metadata:");
+    if (this.metadata == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.metadata);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("isSupported:");
+    sb.append(this.isSupported);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetMetadata()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'metadata' is unset! Struct:" + toString());
+    }
+
+    if (!isSetIsSupported()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'isSupported' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetFileMetadataResultStandardSchemeFactory implements SchemeFactory {
+    public GetFileMetadataResultStandardScheme getScheme() {
+      return new GetFileMetadataResultStandardScheme();
+    }
+  }
+
+  private static class GetFileMetadataResultStandardScheme extends StandardScheme<GetFileMetadataResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // METADATA
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map550 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map550.size);
+                long _key551;
+                ByteBuffer _val552;
+                for (int _i553 = 0; _i553 < _map550.size; ++_i553)
+                {
+                  _key551 = iprot.readI64();
+                  _val552 = iprot.readBinary();
+                  struct.metadata.put(_key551, _val552);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setMetadataIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // IS_SUPPORTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isSupported = iprot.readBool();
+              struct.setIsSupportedIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.metadata != null) {
+        oprot.writeFieldBegin(METADATA_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
+          for (Map.Entry<Long, ByteBuffer> _iter554 : struct.metadata.entrySet())
+          {
+            oprot.writeI64(_iter554.getKey());
+            oprot.writeBinary(_iter554.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(IS_SUPPORTED_FIELD_DESC);
+      oprot.writeBool(struct.isSupported);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetFileMetadataResultTupleSchemeFactory implements SchemeFactory {
+    public GetFileMetadataResultTupleScheme getScheme() {
+      return new GetFileMetadataResultTupleScheme();
+    }
+  }
+
+  private static class GetFileMetadataResultTupleScheme extends TupleScheme<GetFileMetadataResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.metadata.size());
+        for (Map.Entry<Long, ByteBuffer> _iter555 : struct.metadata.entrySet())
+        {
+          oprot.writeI64(_iter555.getKey());
+          oprot.writeBinary(_iter555.getValue());
+        }
+      }
+      oprot.writeBool(struct.isSupported);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map556 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map556.size);
+        long _key557;
+        ByteBuffer _val558;
+        for (int _i559 = 0; _i559 < _map556.size; ++_i559)
+        {
+          _key557 = iprot.readI64();
+          _val558 = iprot.readBinary();
+          struct.metadata.put(_key557, _val558);
+        }
+      }
+      struct.setMetadataIsSet(true);
+      struct.isSupported = iprot.readBool();
+      struct.setIsSupportedIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
index 8d2b62e..ad1af91 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenTxnsInfoResponse, GetOpenTxnsInfoResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetOpenTxnsInfoResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOpenTxnsInfoResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
index 3d70d93..fb6d841 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsResponse, GetOpenTxnsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetOpenTxnsResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOpenTxnsResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java
index 1d34a40..6111cb9 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GetPrincipalsInRoleRequest implements org.apache.thrift.TBase<GetPrincipalsInRoleRequest, GetPrincipalsInRoleRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetPrincipalsInRoleRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPrincipalsInRoleRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java
index 3db7c94..abe22af 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPrincipalsInRoleResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GetPrincipalsInRoleResponse implements org.apache.thrift.TBase<GetPrincipalsInRoleResponse, GetPrincipalsInRoleResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetPrincipalsInRoleResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPrincipalsInRoleResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java
index fb9eb88..42154b3 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GetRoleGrantsForPrincipalRequest implements org.apache.thrift.TBase<GetRoleGrantsForPrincipalRequest, GetRoleGrantsForPrincipalRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetRoleGrantsForPrincipalRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetRoleGrantsForPrincipalRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java
index 42dfa01..2df6f63 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRoleGrantsForPrincipalResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GetRoleGrantsForPrincipalResponse implements org.apache.thrift.TBase<GetRoleGrantsForPrincipalResponse, GetRoleGrantsForPrincipalResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetRoleGrantsForPrincipalResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetRoleGrantsForPrincipalResponse");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java
index 8a042f7..10282e7 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeRequest.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GrantRevokePrivilegeRequest implements org.apache.thrift.TBase<GrantRevokePrivilegeRequest, GrantRevokePrivilegeRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GrantRevokePrivilegeRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GrantRevokePrivilegeRequest");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java
index 113a07f..6a123e9 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GrantRevokePrivilegeResponse.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-17")
 public class GrantRevokePrivilegeResponse implements org.apache.thrift.TBase<GrantRevokePrivilegeResponse, GrantRevokePrivilegeResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GrantRevokePrivilegeResponse> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GrantRevokePrivilegeResponse");
 


[15/52] [abbrv] hive git commit: HIVE-11568 : merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java
index 305e979,1292a64..e8cb821
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchObjectException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class NoSuchObjectException extends TException implements org.apache.thrift.TBase<NoSuchObjectException, NoSuchObjectException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class NoSuchObjectException extends TException implements org.apache.thrift.TBase<NoSuchObjectException, NoSuchObjectException._Fields>, java.io.Serializable, Cloneable, Comparable<NoSuchObjectException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NoSuchObjectException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java
index 92dbb7f,d1c430d..9997b93
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class NoSuchTxnException extends TException implements org.apache.thrift.TBase<NoSuchTxnException, NoSuchTxnException._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class NoSuchTxnException extends TException implements org.apache.thrift.TBase<NoSuchTxnException, NoSuchTxnException._Fields>, java.io.Serializable, Cloneable, Comparable<NoSuchTxnException> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NoSuchTxnException");
  
    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
index f196c1c,bcf4f51..6f594c5
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class NotificationEvent implements org.apache.thrift.TBase<NotificationEvent, NotificationEvent._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class NotificationEvent implements org.apache.thrift.TBase<NotificationEvent, NotificationEvent._Fields>, java.io.Serializable, Cloneable, Comparable<NotificationEvent> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotificationEvent");
  
    private static final org.apache.thrift.protocol.TField EVENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("eventId", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
index 6a8c8ab,c2bc4e8..0c6dc01
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class NotificationEventRequest implements org.apache.thrift.TBase<NotificationEventRequest, NotificationEventRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class NotificationEventRequest implements org.apache.thrift.TBase<NotificationEventRequest, NotificationEventRequest._Fields>, java.io.Serializable, Cloneable, Comparable<NotificationEventRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotificationEventRequest");
  
    private static final org.apache.thrift.protocol.TField LAST_EVENT_FIELD_DESC = new org.apache.thrift.protocol.TField("lastEvent", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index 0123e87,24f9ce4..3295c3c
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class NotificationEventResponse implements org.apache.thrift.TBase<NotificationEventResponse, NotificationEventResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class NotificationEventResponse implements org.apache.thrift.TBase<NotificationEventResponse, NotificationEventResponse._Fields>, java.io.Serializable, Cloneable, Comparable<NotificationEventResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotificationEventResponse");
  
    private static final org.apache.thrift.protocol.TField EVENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("events", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
index 0d05378,c5f9ccf..a09575d
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class OpenTxnRequest implements org.apache.thrift.TBase<OpenTxnRequest, OpenTxnRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class OpenTxnRequest implements org.apache.thrift.TBase<OpenTxnRequest, OpenTxnRequest._Fields>, java.io.Serializable, Cloneable, Comparable<OpenTxnRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenTxnRequest");
  
    private static final org.apache.thrift.protocol.TField NUM_TXNS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_txns", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
index f5efbe3,c233422..d874654
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsResponse, OpenTxnsResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsResponse, OpenTxnsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<OpenTxnsResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenTxnsResponse");
  
    private static final org.apache.thrift.protocol.TField TXN_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("txn_ids", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
index 55d5174,da7bd55..d83b83d
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, java.io.Serializable, Cloneable, Comparable<Order> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Order");
  
    private static final org.apache.thrift.protocol.TField COL_FIELD_DESC = new org.apache.thrift.protocol.TField("col", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
index 7d29d09,224d28e..e38798a
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Partition implements org.apache.thrift.TBase<Partition, Partition._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Partition implements org.apache.thrift.TBase<Partition, Partition._Fields>, java.io.Serializable, Cloneable, Comparable<Partition> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Partition");
  
    private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
index bcd92b8,c50a100..6ccebb3
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PartitionListComposingSpec implements org.apache.thrift.TBase<PartitionListComposingSpec, PartitionListComposingSpec._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PartitionListComposingSpec implements org.apache.thrift.TBase<PartitionListComposingSpec, PartitionListComposingSpec._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionListComposingSpec> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionListComposingSpec");
  
    private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
index 06cc8fb,b5251af..8d2f1b4
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PartitionSpec implements org.apache.thrift.TBase<PartitionSpec, PartitionSpec._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PartitionSpec implements org.apache.thrift.TBase<PartitionSpec, PartitionSpec._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionSpec> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionSpec");
  
    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
index 18ab134,5574e0b..08d8548
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PartitionSpecWithSharedSD implements org.apache.thrift.TBase<PartitionSpecWithSharedSD, PartitionSpecWithSharedSD._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PartitionSpecWithSharedSD implements org.apache.thrift.TBase<PartitionSpecWithSharedSD, PartitionSpecWithSharedSD._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionSpecWithSharedSD> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionSpecWithSharedSD");
  
    private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
index 193d9e9,e1ec73e..57ff72e
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PartitionWithoutSD implements org.apache.thrift.TBase<PartitionWithoutSD, PartitionWithoutSD._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PartitionWithoutSD implements org.apache.thrift.TBase<PartitionWithoutSD, PartitionWithoutSD._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionWithoutSD> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionWithoutSD");
  
    private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
index c6fa8a2,6149c31..b10f3c8
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PartitionsByExprRequest implements org.apache.thrift.TBase<PartitionsByExprRequest, PartitionsByExprRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PartitionsByExprRequest implements org.apache.thrift.TBase<PartitionsByExprRequest, PartitionsByExprRequest._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionsByExprRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionsByExprRequest");
  
    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
index 525ce0e,740f7bd..3a0376d
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PartitionsByExprResult implements org.apache.thrift.TBase<PartitionsByExprResult, PartitionsByExprResult._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PartitionsByExprResult implements org.apache.thrift.TBase<PartitionsByExprResult, PartitionsByExprResult._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionsByExprResult> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionsByExprResult");
  
    private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
index d224453,5d1ee87..bfa77f6
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PartitionsStatsRequest implements org.apache.thrift.TBase<PartitionsStatsRequest, PartitionsStatsRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PartitionsStatsRequest implements org.apache.thrift.TBase<PartitionsStatsRequest, PartitionsStatsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionsStatsRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionsStatsRequest");
  
    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
index c9ae14e,da33014..757f209
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PartitionsStatsResult implements org.apache.thrift.TBase<PartitionsStatsResult, PartitionsStatsResult._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PartitionsStatsResult implements org.apache.thrift.TBase<PartitionsStatsResult, PartitionsStatsResult._Fields>, java.io.Serializable, Cloneable, Comparable<PartitionsStatsResult> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PartitionsStatsResult");
  
    private static final org.apache.thrift.protocol.TField PART_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("partStats", org.apache.thrift.protocol.TType.MAP, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java
index 0c9518a,8f29f50..889a41c
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PrincipalPrivilegeSet implements org.apache.thrift.TBase<PrincipalPrivilegeSet, PrincipalPrivilegeSet._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PrincipalPrivilegeSet implements org.apache.thrift.TBase<PrincipalPrivilegeSet, PrincipalPrivilegeSet._Fields>, java.io.Serializable, Cloneable, Comparable<PrincipalPrivilegeSet> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PrincipalPrivilegeSet");
  
    private static final org.apache.thrift.protocol.TField USER_PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("userPrivileges", org.apache.thrift.protocol.TType.MAP, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java
index 4285ed8,2fd819c..741ace7
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PrivilegeBag implements org.apache.thrift.TBase<PrivilegeBag, PrivilegeBag._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PrivilegeBag implements org.apache.thrift.TBase<PrivilegeBag, PrivilegeBag._Fields>, java.io.Serializable, Cloneable, Comparable<PrivilegeBag> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PrivilegeBag");
  
    private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java
index 5869457,c04e196..ba52582
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeGrantInfo.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class PrivilegeGrantInfo implements org.apache.thrift.TBase<PrivilegeGrantInfo, PrivilegeGrantInfo._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class PrivilegeGrantInfo implements org.apache.thrift.TBase<PrivilegeGrantInfo, PrivilegeGrantInfo._Fields>, java.io.Serializable, Cloneable, Comparable<PrivilegeGrantInfo> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PrivilegeGrantInfo");
  
    private static final org.apache.thrift.protocol.TField PRIVILEGE_FIELD_DESC = new org.apache.thrift.protocol.TField("privilege", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
index c230eab,3b3df25..cffcf91
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ResourceUri implements org.apache.thrift.TBase<ResourceUri, ResourceUri._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ResourceUri implements org.apache.thrift.TBase<ResourceUri, ResourceUri._Fields>, java.io.Serializable, Cloneable, Comparable<ResourceUri> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ResourceUri");
  
    private static final org.apache.thrift.protocol.TField RESOURCE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("resourceType", org.apache.thrift.protocol.TType.I32, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java
index 35fcf58,5c882d2..b9052a3
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Role.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Role implements org.apache.thrift.TBase<Role, Role._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Role implements org.apache.thrift.TBase<Role, Role._Fields>, java.io.Serializable, Cloneable, Comparable<Role> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Role");
  
    private static final org.apache.thrift.protocol.TField ROLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("roleName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java
index 8993268,c4beb08..6e3c200
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RolePrincipalGrant.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class RolePrincipalGrant implements org.apache.thrift.TBase<RolePrincipalGrant, RolePrincipalGrant._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class RolePrincipalGrant implements org.apache.thrift.TBase<RolePrincipalGrant, RolePrincipalGrant._Fields>, java.io.Serializable, Cloneable, Comparable<RolePrincipalGrant> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RolePrincipalGrant");
  
    private static final org.apache.thrift.protocol.TField ROLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("roleName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
index 31ee943,8772180..85af5c5
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Schema implements org.apache.thrift.TBase<Schema, Schema._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Schema implements org.apache.thrift.TBase<Schema, Schema._Fields>, java.io.Serializable, Cloneable, Comparable<Schema> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Schema");
  
    private static final org.apache.thrift.protocol.TField FIELD_SCHEMAS_FIELD_DESC = new org.apache.thrift.protocol.TField("fieldSchemas", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
index 24d65bb,b30e698..73853a2
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class SerDeInfo implements org.apache.thrift.TBase<SerDeInfo, SerDeInfo._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class SerDeInfo implements org.apache.thrift.TBase<SerDeInfo, SerDeInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SerDeInfo> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SerDeInfo");
  
    private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
index e62e410,7da298c..d09e413
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class SetPartitionsStatsRequest implements org.apache.thrift.TBase<SetPartitionsStatsRequest, SetPartitionsStatsRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class SetPartitionsStatsRequest implements org.apache.thrift.TBase<SetPartitionsStatsRequest, SetPartitionsStatsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<SetPartitionsStatsRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SetPartitionsStatsRequest");
  
    private static final org.apache.thrift.protocol.TField COL_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("colStats", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java
index 28ad1c9,7756384..7d4d12c
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ShowCompactRequest implements org.apache.thrift.TBase<ShowCompactRequest, ShowCompactRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ShowCompactRequest implements org.apache.thrift.TBase<ShowCompactRequest, ShowCompactRequest._Fields>, java.io.Serializable, Cloneable, Comparable<ShowCompactRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactRequest");
  
  

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index c882b7b,dd1e857..7112f26
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactResponse, ShowCompactResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactResponse, ShowCompactResponse._Fields>, java.io.Serializable, Cloneable, Comparable<ShowCompactResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactResponse");
  
    private static final org.apache.thrift.protocol.TField COMPACTS_FIELD_DESC = new org.apache.thrift.protocol.TField("compacts", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
index 365a401,cd7e79e..810b140
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowCompactResponseElement, ShowCompactResponseElement._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowCompactResponseElement, ShowCompactResponseElement._Fields>, java.io.Serializable, Cloneable, Comparable<ShowCompactResponseElement> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactResponseElement");
  
    private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java
index 7a1fd6f,122c070..c13fda4
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ShowLocksRequest implements org.apache.thrift.TBase<ShowLocksRequest, ShowLocksRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ShowLocksRequest implements org.apache.thrift.TBase<ShowLocksRequest, ShowLocksRequest._Fields>, java.io.Serializable, Cloneable, Comparable<ShowLocksRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksRequest");
  
  

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index 42cfe8c,52b0bbc..2289195
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksResponse, ShowLocksResponse._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksResponse, ShowLocksResponse._Fields>, java.io.Serializable, Cloneable, Comparable<ShowLocksResponse> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksResponse");
  
    private static final org.apache.thrift.protocol.TField LOCKS_FIELD_DESC = new org.apache.thrift.protocol.TField("locks", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java
index 2f7c24f,8be9b05..ba17ea7
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ShowLocksResponseElement implements org.apache.thrift.TBase<ShowLocksResponseElement, ShowLocksResponseElement._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class ShowLocksResponseElement implements org.apache.thrift.TBase<ShowLocksResponseElement, ShowLocksResponseElement._Fields>, java.io.Serializable, Cloneable, Comparable<ShowLocksResponseElement> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksResponseElement");
  
    private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
index ab5c0ed,bc64495..5cb5e2b
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class SkewedInfo implements org.apache.thrift.TBase<SkewedInfo, SkewedInfo._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class SkewedInfo implements org.apache.thrift.TBase<SkewedInfo, SkewedInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SkewedInfo> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SkewedInfo");
  
    private static final org.apache.thrift.protocol.TField SKEWED_COL_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("skewedColNames", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
index 813b4f0,165a879..6b5cf87
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class StorageDescriptor implements org.apache.thrift.TBase<StorageDescriptor, StorageDescriptor._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class StorageDescriptor implements org.apache.thrift.TBase<StorageDescriptor, StorageDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<StorageDescriptor> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StorageDescriptor");
  
    private static final org.apache.thrift.protocol.TField COLS_FIELD_DESC = new org.apache.thrift.protocol.TField("cols", org.apache.thrift.protocol.TType.LIST, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java
index db3274a,9906ff3..2160bc8
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StringColumnStatsData.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class StringColumnStatsData implements org.apache.thrift.TBase<StringColumnStatsData, StringColumnStatsData._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class StringColumnStatsData implements org.apache.thrift.TBase<StringColumnStatsData, StringColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<StringColumnStatsData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StringColumnStatsData");
  
    private static final org.apache.thrift.protocol.TField MAX_COL_LEN_FIELD_DESC = new org.apache.thrift.protocol.TField("maxColLen", org.apache.thrift.protocol.TType.I64, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
index 484bd6a,51b9e38..ca16924
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, java.io.Serializable, Cloneable, Comparable<Table> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Table");
  
    private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
index 2073829,1edcaf9..11d3b03
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequest, TableStatsRequest._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequest, TableStatsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<TableStatsRequest> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TableStatsRequest");
  
    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c528294b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
index 541370f,25a1f25..f1104e1
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
@@@ -31,7 -33,9 +33,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResult, TableStatsResult._Fields>, java.io.Serializable, Cloneable {
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-3")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-14")
+ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResult, TableStatsResult._Fields>, java.io.Serializable, Cloneable, Comparable<TableStatsResult> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TableStatsResult");
  
    private static final org.apache.thrift.protocol.TField TABLE_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("tableStats", org.apache.thrift.protocol.TType.LIST, (short)1);


[20/52] [abbrv] hive git commit: HIVE-10289: Support filter on non-first partition key and non-string partition key (Daniel Dai reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseFilterPlanUtil.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseFilterPlanUtil.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseFilterPlanUtil.java
index ec99685..9762309 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseFilterPlanUtil.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseFilterPlanUtil.java
@@ -20,15 +20,30 @@ package org.apache.hadoop.hive.metastore.hbase;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.LeafNode;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeNode;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeVisitor;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
@@ -147,7 +162,7 @@ class HBaseFilterPlanUtil {
   public static class ScanPlan extends FilterPlan {
 
     public static class ScanMarker {
-      final byte[] bytes;
+      final String value;
       /**
        * If inclusive = true, it means that the
        * marker includes those bytes.
@@ -155,20 +170,24 @@ class HBaseFilterPlanUtil {
        * or ends at the next possible byte array
        */
       final boolean isInclusive;
-      ScanMarker(byte [] b, boolean i){
-        this.bytes = b;
+      final String type;
+      ScanMarker(String obj, boolean i, String type){
+        this.value = obj;
         this.isInclusive = i;
+        this.type = type;
       }
       @Override
       public String toString() {
-        return "ScanMarker [bytes=" + Arrays.toString(bytes) + ", isInclusive=" + isInclusive + "]";
+        return "ScanMarker [" + "value=" + value.toString() + ", isInclusive=" + isInclusive +
+            ", type=" + type + "]";
       }
       @Override
       public int hashCode() {
         final int prime = 31;
         int result = 1;
-        result = prime * result + Arrays.hashCode(bytes);
+        result = prime * result + value.hashCode();
         result = prime * result + (isInclusive ? 1231 : 1237);
+        result = prime * result + type.hashCode();
         return result;
       }
       @Override
@@ -180,48 +199,118 @@ class HBaseFilterPlanUtil {
         if (getClass() != obj.getClass())
           return false;
         ScanMarker other = (ScanMarker) obj;
-        if (!Arrays.equals(bytes, other.bytes))
+        if (!value.equals(other.value))
           return false;
         if (isInclusive != other.isInclusive)
           return false;
+        if (type != other.type)
+          return false;
         return true;
       }
     }
-    // represent Scan start
-    private ScanMarker startMarker = new ScanMarker(null, false);
-    // represent Scan end
-    private ScanMarker endMarker = new ScanMarker(null, false);
-
-    private ScanFilter filter;
-
-    public ScanFilter getFilter() {
-      return filter;
+    public static class ScanMarkerPair {
+      public ScanMarkerPair(ScanMarker startMarker, ScanMarker endMarker) {
+        this.startMarker = startMarker;
+        this.endMarker = endMarker;
+      }
+      ScanMarker startMarker;
+      ScanMarker endMarker;
+    }
+    // represent Scan start, partition key name -> scanMarkerPair
+    Map<String, ScanMarkerPair> markers = new HashMap<String, ScanMarkerPair>();
+    List<Operator> ops = new ArrayList<Operator>();
+
+    // Get the number of partition key prefixes which can be used in the scan range.
+    // For example, if partition key is (year, month, state)
+    // 1. year = 2015 and month >= 1 and month < 5
+    //    year + month can be used in scan range, majorParts = 2
+    // 2. year = 2015 and state = 'CA'
+    //    only year can be used in scan range, majorParts = 1
+    // 3. month = 10 and state = 'CA'
+    //    nothing can be used in scan range, majorParts = 0
+    private int getMajorPartsCount(List<FieldSchema> parts) {
+      int majorPartsCount = 0;
+      while (majorPartsCount<parts.size() && markers.containsKey(parts.get(majorPartsCount).getName())) {
+        ScanMarkerPair pair = markers.get(parts.get(majorPartsCount).getName());
+        majorPartsCount++;
+        if (pair.startMarker!=null && pair.endMarker!=null && pair.startMarker.value.equals(pair
+            .endMarker.value) && pair.startMarker.isInclusive && pair.endMarker.isInclusive) {
+          // is equal
+          continue;
+        } else {
+          break;
+        }
+      }
+      return majorPartsCount;
     }
+    public Filter getFilter(List<FieldSchema> parts) {
+      int majorPartsCount = getMajorPartsCount(parts);
+      Set<String> majorKeys = new HashSet<String>();
+      for (int i=0;i<majorPartsCount;i++) {
+        majorKeys.add(parts.get(i).getName());
+      }
 
-    public void setFilter(ScanFilter filter) {
-      this.filter = filter;
-    }
+      List<String> names = HBaseUtils.getPartitionNames(parts);
+      List<PartitionKeyComparator.Range> ranges = new ArrayList<PartitionKeyComparator.Range>();
+      for (Map.Entry<String, ScanMarkerPair> entry : markers.entrySet()) {
+        if (names.contains(entry.getKey()) && !majorKeys.contains(entry.getKey())) {
+          PartitionKeyComparator.Mark startMark = null;
+          if (entry.getValue().startMarker != null) {
+            startMark = new PartitionKeyComparator.Mark(entry.getValue().startMarker.value,
+                entry.getValue().startMarker.isInclusive);
+          }
+          PartitionKeyComparator.Mark endMark = null;
+          if (entry.getValue().endMarker != null) {
+            startMark = new PartitionKeyComparator.Mark(entry.getValue().endMarker.value,
+                entry.getValue().endMarker.isInclusive);
+          }
+          PartitionKeyComparator.Range range = new PartitionKeyComparator.Range(
+              entry.getKey(), startMark, endMark);
+          ranges.add(range);
+        }
+      }
 
-    public ScanMarker getStartMarker() {
-      return startMarker;
+      if (ranges.isEmpty() && ops.isEmpty()) {
+        return null;
+      } else {
+        return new RowFilter(CompareFilter.CompareOp.EQUAL, new PartitionKeyComparator(
+            StringUtils.join(names, ","), StringUtils.join(HBaseUtils.getPartitionKeyTypes(parts), ","),
+            ranges, ops));
+      }
     }
 
-    public void setStartMarker(ScanMarker startMarker) {
-      this.startMarker = startMarker;
-    }
-    public void setStartMarker(byte[] start, boolean isInclusive) {
-      setStartMarker(new ScanMarker(start, isInclusive));
+    public void setStartMarker(String keyName, String keyType, String start, boolean isInclusive) {
+      if (markers.containsKey(keyName)) {
+        markers.get(keyName).startMarker = new ScanMarker(start, isInclusive, keyType);
+      } else {
+        ScanMarkerPair marker = new ScanMarkerPair(new ScanMarker(start, isInclusive, keyType), null);
+        markers.put(keyName, marker);
+      }
     }
 
-    public ScanMarker getEndMarker() {
-      return endMarker;
+    public ScanMarker getStartMarker(String keyName) {
+      if (markers.containsKey(keyName)) {
+        return markers.get(keyName).startMarker;
+      } else {
+        return null;
+      }
     }
 
-    public void setEndMarker(ScanMarker endMarker) {
-      this.endMarker = endMarker;
+    public void setEndMarker(String keyName, String keyType, String end, boolean isInclusive) {
+      if (markers.containsKey(keyName)) {
+        markers.get(keyName).endMarker = new ScanMarker(end, isInclusive, keyType);
+      } else {
+        ScanMarkerPair marker = new ScanMarkerPair(null, new ScanMarker(end, isInclusive, keyType));
+        markers.put(keyName, marker);
+      }
     }
-    public void setEndMarker(byte[] end, boolean isInclusive) {
-      setEndMarker(new ScanMarker(end, isInclusive));
+
+    public ScanMarker getEndMarker(String keyName) {
+      if (markers.containsKey(keyName)) {
+        return markers.get(keyName).endMarker;
+      } else {
+        return null;
+      }
     }
 
     @Override
@@ -236,28 +325,33 @@ class HBaseFilterPlanUtil {
     private ScanPlan and(ScanPlan other) {
       // create combined FilterPlan based on existing lhs and rhs plan
       ScanPlan newPlan = new ScanPlan();
+      newPlan.markers.putAll(markers);
+
+      for (String keyName : other.markers.keySet()) {
+        if (newPlan.markers.containsKey(keyName)) {
+          // create new scan start
+          ScanMarker greaterStartMarker = getComparedMarker(this.getStartMarker(keyName),
+              other.getStartMarker(keyName), true);
+          if (greaterStartMarker != null) {
+            newPlan.setStartMarker(keyName, greaterStartMarker.type, greaterStartMarker.value, greaterStartMarker.isInclusive);
+          }
+
+          // create new scan end
+          ScanMarker lesserEndMarker = getComparedMarker(this.getEndMarker(keyName), other.getEndMarker(keyName),
+              false);
+          if (lesserEndMarker != null) {
+            newPlan.setEndMarker(keyName, lesserEndMarker.type, lesserEndMarker.value, lesserEndMarker.isInclusive);
+          }
+        } else {
+          newPlan.markers.put(keyName, other.markers.get(keyName));
+        }
+      }
 
-      // create new scan start
-      ScanMarker greaterStartMarker = getComparedMarker(this.getStartMarker(),
-          other.getStartMarker(), true);
-      newPlan.setStartMarker(greaterStartMarker);
-
-      // create new scan end
-      ScanMarker lesserEndMarker = getComparedMarker(this.getEndMarker(), other.getEndMarker(),
-          false);
-      newPlan.setEndMarker(lesserEndMarker);
-
-      // create new filter plan
-      newPlan.setFilter(createCombinedFilter(this.getFilter(), other.getFilter()));
-
+      newPlan.ops.addAll(ops);
+      newPlan.ops.addAll(other.ops);
       return newPlan;
     }
 
-    private ScanFilter createCombinedFilter(ScanFilter filter1, ScanFilter filter2) {
-      // TODO create combined filter - filter1 && filter2
-      return null;
-    }
-
     /**
      * @param lStartMarker
      * @param rStartMarker
@@ -268,13 +362,23 @@ class HBaseFilterPlanUtil {
     static ScanMarker getComparedMarker(ScanMarker lStartMarker, ScanMarker rStartMarker,
         boolean getGreater) {
       // if one of them has null bytes, just return other
-      if(lStartMarker.bytes == null) {
+      if(lStartMarker == null) {
         return rStartMarker;
-      } else if (rStartMarker.bytes == null) {
+      } else if (rStartMarker == null) {
         return lStartMarker;
       }
-
-      int compareRes = compare(lStartMarker.bytes, rStartMarker.bytes);
+      TypeInfo expectedType =
+          TypeInfoUtils.getTypeInfoFromTypeString(lStartMarker.type);
+      ObjectInspector outputOI =
+          TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(expectedType);
+      Converter lConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaStringObjectInspector, outputOI);
+      Converter rConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaStringObjectInspector, outputOI);
+      Comparable lValue = (Comparable)lConverter.convert(lStartMarker.value);
+      Comparable rValue = (Comparable)rConverter.convert(rStartMarker.value);
+
+      int compareRes = lValue.compareTo(rValue);
       if (compareRes == 0) {
         // bytes are equal, now compare the isInclusive flags
         if (lStartMarker.isInclusive == rStartMarker.isInclusive) {
@@ -287,7 +391,7 @@ class HBaseFilterPlanUtil {
           isInclusive = false;
         }
         // else
-        return new ScanMarker(lStartMarker.bytes, isInclusive);
+        return new ScanMarker(lStartMarker.value, isInclusive, lStartMarker.type);
       }
       if (getGreater) {
         return compareRes == 1 ? lStartMarker : rStartMarker;
@@ -313,42 +417,74 @@ class HBaseFilterPlanUtil {
     /**
      * @return row suffix - This is appended to db + table, to generate start row for the Scan
      */
-    public byte[] getStartRowSuffix() {
-      if (startMarker.isInclusive) {
-        return startMarker.bytes;
-      } else {
-        return HBaseUtils.getEndPrefix(startMarker.bytes);
+    public byte[] getStartRowSuffix(String dbName, String tableName, List<FieldSchema> parts) {
+      int majorPartsCount = getMajorPartsCount(parts);
+      List<String> majorPartTypes = new ArrayList<String>();
+      List<String> components = new ArrayList<String>();
+      boolean endPrefix = false;
+      for (int i=0;i<majorPartsCount;i++) {
+        majorPartTypes.add(parts.get(i).getType());
+        ScanMarker marker = markers.get(parts.get(i).getName()).startMarker;
+        if (marker != null) {
+          components.add(marker.value);
+          if (i==majorPartsCount-1) {
+            endPrefix = !marker.isInclusive;
+          }
+        } else {
+          components.add(null);
+          if (i==majorPartsCount-1) {
+            endPrefix = false;
+          }
+        }
       }
+      byte[] bytes = HBaseUtils.buildPartitionKey(dbName, tableName, majorPartTypes, components, endPrefix);
+      return bytes;
     }
 
     /**
      * @return row suffix - This is appended to db + table, to generate end row for the Scan
      */
-    public byte[] getEndRowSuffix() {
-      if (endMarker.isInclusive) {
-        return HBaseUtils.getEndPrefix(endMarker.bytes);
-      } else {
-        return endMarker.bytes;
+    public byte[] getEndRowSuffix(String dbName, String tableName, List<FieldSchema> parts) {
+      int majorPartsCount = getMajorPartsCount(parts);
+      List<String> majorPartTypes = new ArrayList<String>();
+      List<String> components = new ArrayList<String>();
+      boolean endPrefix = false;
+      for (int i=0;i<majorPartsCount;i++) {
+        majorPartTypes.add(parts.get(i).getType());
+        ScanMarker marker = markers.get(parts.get(i).getName()).endMarker;
+        if (marker != null) {
+          components.add(marker.value);
+          if (i==majorPartsCount-1) {
+            endPrefix = marker.isInclusive;
+          }
+        } else {
+          components.add(null);
+          if (i==majorPartsCount-1) {
+            endPrefix = true;
+          }
+        }
+      }
+      byte[] bytes = HBaseUtils.buildPartitionKey(dbName, tableName, majorPartTypes, components, endPrefix);
+      if (components.isEmpty()) {
+        bytes[bytes.length-1]++;
       }
+      return bytes;
     }
 
     @Override
     public String toString() {
-      return "ScanPlan [startMarker=" + startMarker + ", endMarker=" + endMarker + ", filter="
-          + filter + "]";
+      StringBuffer sb = new StringBuffer();
+      sb.append("ScanPlan:\n");
+      for (Map.Entry<String, ScanMarkerPair> entry : markers.entrySet()) {
+        sb.append("key=" + entry.getKey() + "[startMarker=" + entry.getValue().startMarker
+            + ", endMarker=" + entry.getValue().endMarker + "]");
+      }
+      return sb.toString();
     }
 
   }
 
   /**
-   * represent a plan that can be used to create a hbase filter and then set in
-   * Scan.setFilter()
-   */
-  public static class ScanFilter {
-    // TODO: implement this
-  }
-
-  /**
    * Visitor for ExpressionTree.
    * It first generates the ScanPlan for the leaf nodes. The higher level nodes are
    * either AND or OR operations. It then calls FilterPlan.and and FilterPlan.or with
@@ -369,9 +505,12 @@ class HBaseFilterPlanUtil {
     // temporary params for current left and right side plans, for AND, OR
     private FilterPlan rPlan;
 
-    private final String firstPartcolumn;
-    public PartitionFilterGenerator(String firstPartitionColumn) {
-      this.firstPartcolumn = firstPartitionColumn;
+    private Map<String, String> nameToType = new HashMap<String, String>();
+
+    public PartitionFilterGenerator(List<FieldSchema> parts) {
+      for (FieldSchema part : parts) {
+        nameToType.put(part.getName(), part.getType());
+      }
     }
 
     FilterPlan getPlan() {
@@ -414,63 +553,37 @@ class HBaseFilterPlanUtil {
     public void visit(LeafNode node) throws MetaException {
       ScanPlan leafPlan = new ScanPlan();
       curPlan = leafPlan;
-      if (!isFirstParitionColumn(node.keyName)) {
-        leafPlan.setFilter(generateScanFilter(node));
-        return;
-      }
-      if (!(node.value instanceof String)) {
-        // only string type is supported currently
-        // treat conditions on other types as true
-        return;
-      }
 
       // this is a condition on first partition column, so might influence the
       // start and end of the scan
       final boolean INCLUSIVE = true;
       switch (node.operator) {
       case EQUALS:
-        leafPlan.setStartMarker(toBytes(node.value), INCLUSIVE);
-        leafPlan.setEndMarker(toBytes(node.value), INCLUSIVE);
+        leafPlan.setStartMarker(node.keyName, nameToType.get(node.keyName), node.value.toString(), INCLUSIVE);
+        leafPlan.setEndMarker(node.keyName, nameToType.get(node.keyName), node.value.toString(), INCLUSIVE);
         break;
       case GREATERTHAN:
-        leafPlan.setStartMarker(toBytes(node.value), !INCLUSIVE);
+        leafPlan.setStartMarker(node.keyName, nameToType.get(node.keyName), node.value.toString(), !INCLUSIVE);
         break;
       case GREATERTHANOREQUALTO:
-        leafPlan.setStartMarker(toBytes(node.value), INCLUSIVE);
+        leafPlan.setStartMarker(node.keyName, nameToType.get(node.keyName), node.value.toString(), INCLUSIVE);
         break;
       case LESSTHAN:
-        leafPlan.setEndMarker(toBytes(node.value), !INCLUSIVE);
+        leafPlan.setEndMarker(node.keyName, nameToType.get(node.keyName), node.value.toString(), !INCLUSIVE);
         break;
       case LESSTHANOREQUALTO:
-        leafPlan.setEndMarker(toBytes(node.value), INCLUSIVE);
+        leafPlan.setEndMarker(node.keyName, nameToType.get(node.keyName), node.value.toString(), INCLUSIVE);
         break;
       case LIKE:
+        leafPlan.ops.add(new Operator(Operator.Type.LIKE, node.keyName, node.value.toString()));
+        break;
       case NOTEQUALS:
       case NOTEQUALS2:
-        // TODO: create filter plan for these
-        hasUnsupportedCondition = true;
+        leafPlan.ops.add(new Operator(Operator.Type.NOTEQUALS, node.keyName, node.value.toString()));
         break;
       }
     }
 
-    @VisibleForTesting
-    static byte[] toBytes(Object value) {
-      // TODO: actually implement this
-      // We need to determine the actual type and use appropriate
-      // serialization format for that type
-      return ((String) value).getBytes(HBaseUtils.ENCODING);
-    }
-
-    private ScanFilter generateScanFilter(LeafNode node) {
-      // TODO Auto-generated method stub
-      hasUnsupportedCondition = true;
-      return null;
-    }
-
-    private boolean isFirstParitionColumn(String keyName) {
-      return keyName.equalsIgnoreCase(firstPartcolumn);
-    }
-
     private boolean hasUnsupportedCondition() {
       return hasUnsupportedCondition;
     }
@@ -486,12 +599,12 @@ class HBaseFilterPlanUtil {
     }
   }
 
-  public static PlanResult getFilterPlan(ExpressionTree exprTree, String firstPartitionColumn) throws MetaException {
+  public static PlanResult getFilterPlan(ExpressionTree exprTree, List<FieldSchema> parts) throws MetaException {
     if (exprTree == null) {
       // TODO: if exprTree is null, we should do what ObjectStore does. See HIVE-10102
       return new PlanResult(new ScanPlan(), true);
     }
-    PartitionFilterGenerator pGenerator = new PartitionFilterGenerator(firstPartitionColumn);
+    PartitionFilterGenerator pGenerator = new PartitionFilterGenerator(parts);
     exprTree.accept(pGenerator);
     return new PlanResult(pGenerator.getPlan(), pGenerator.hasUnsupportedCondition());
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index ca1582e..66c46a5 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -19,8 +19,10 @@
 package org.apache.hadoop.hive.metastore.hbase;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -43,6 +45,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
@@ -51,6 +54,7 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator;
 import org.apache.hive.common.util.BloomFilter;
 
 import java.io.IOException;
@@ -493,12 +497,12 @@ public class HBaseReadWrite {
    * @return a list of partition objects.
    * @throws IOException
    */
-   List<Partition> getPartitions(String dbName, String tableName, List<List<String>> partValLists)
-       throws IOException {
+   List<Partition> getPartitions(String dbName, String tableName, List<String> partTypes,
+       List<List<String>> partValLists) throws IOException {
      List<Partition> parts = new ArrayList<>(partValLists.size());
      List<Get> gets = new ArrayList<>(partValLists.size());
      for (List<String> partVals : partValLists) {
-       byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName, partVals);
+       byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName, partTypes, partVals);
        Get get = new Get(key);
        get.addColumn(CATALOG_CF, CATALOG_COL);
        gets.add(get);
@@ -526,7 +530,8 @@ public class HBaseReadWrite {
    */
   void putPartition(Partition partition) throws IOException {
     byte[] hash = putStorageDescriptor(partition.getSd());
-    byte[][] serialized = HBaseUtils.serializePartition(partition, hash);
+    byte[][] serialized = HBaseUtils.serializePartition(partition,
+        HBaseUtils.getPartitionKeyTypes(getTable(partition.getDbName(), partition.getTableName()).getPartitionKeys()), hash);
     store(PART_TABLE, serialized[0], CATALOG_CF, CATALOG_COL, serialized[1]);
     partCache.put(partition.getDbName(), partition.getTableName(), partition);
   }
@@ -547,7 +552,8 @@ public class HBaseReadWrite {
       decrementStorageDescriptorRefCount(oldPart.getSd());
       hash = putStorageDescriptor(newPart.getSd());
     }
-    byte[][] serialized = HBaseUtils.serializePartition(newPart, hash);
+    byte[][] serialized = HBaseUtils.serializePartition(newPart,
+        HBaseUtils.getPartitionKeyTypes(getTable(newPart.getDbName(), newPart.getTableName()).getPartitionKeys()), hash);
     store(PART_TABLE, serialized[0], CATALOG_CF, CATALOG_COL, serialized[1]);
     partCache.put(newPart.getDbName(), newPart.getTableName(), newPart);
     if (!oldPart.getTableName().equals(newPart.getTableName())) {
@@ -565,7 +571,9 @@ public class HBaseReadWrite {
     List<Put> puts = new ArrayList<>(partitions.size());
     for (Partition partition : partitions) {
       byte[] hash = putStorageDescriptor(partition.getSd());
-      byte[][] serialized = HBaseUtils.serializePartition(partition, hash);
+      List<String> partTypes = HBaseUtils.getPartitionKeyTypes(
+          getTable(partition.getDbName(), partition.getTableName()).getPartitionKeys());
+      byte[][] serialized = HBaseUtils.serializePartition(partition, partTypes, hash);
       Put p = new Put(serialized[0]);
       p.add(CATALOG_CF, CATALOG_COL, serialized[1]);
       puts.add(p);
@@ -591,7 +599,9 @@ public class HBaseReadWrite {
         decrementStorageDescriptorRefCount(oldParts.get(i).getSd());
         hash = putStorageDescriptor(newParts.get(i).getSd());
       }
-      byte[][] serialized = HBaseUtils.serializePartition(newParts.get(i), hash);
+      Partition newPart = newParts.get(i);
+      byte[][] serialized = HBaseUtils.serializePartition(newPart,
+          HBaseUtils.getPartitionKeyTypes(getTable(newPart.getDbName(), newPart.getTableName()).getPartitionKeys()), hash);
       Put p = new Put(serialized[0]);
       p.add(CATALOG_CF, CATALOG_COL, serialized[1]);
       puts.add(p);
@@ -624,8 +634,9 @@ public class HBaseReadWrite {
           ? new ArrayList<>(cached).subList(0, maxPartitions)
           : new ArrayList<>(cached);
     }
-    byte[] keyPrefix = HBaseUtils.buildKeyWithTrailingSeparator(dbName, tableName);
-    List<Partition> parts = scanPartitionsWithFilter(keyPrefix, HBaseUtils.getEndPrefix(keyPrefix), -1, null);
+    byte[] keyPrefix = HBaseUtils.buildPartitionKey(dbName, tableName, new ArrayList<String>(),
+        new ArrayList<String>(), false);
+    List<Partition> parts = scanPartitionsWithFilter(dbName, tableName, keyPrefix, HBaseUtils.getEndPrefix(keyPrefix), -1, null);
     partCache.put(dbName, tableName, parts, true);
     return maxPartitions < parts.size() ? parts.subList(0, maxPartitions) : parts;
   }
@@ -672,72 +683,68 @@ public class HBaseReadWrite {
     if (table == null) {
       throw new NoSuchObjectException("Unable to find table " + dbName + "." + tableName);
     }
-    if (partVals.size() == table.getPartitionKeys().size()) {
-      keyPrefix = HBaseUtils.buildKey(keyElements.toArray(new String[keyElements.size()]));
-    } else {
-      keyPrefix = HBaseUtils.buildKeyWithTrailingSeparator(keyElements.toArray(
-          new String[keyElements.size()]));
-    }
+    keyPrefix = HBaseUtils.buildPartitionKey(dbName, tableName,
+        HBaseUtils.getPartitionKeyTypes(table.getPartitionKeys().subList(0, keyElements.size()-2)),
+          keyElements.subList(0, keyElements.size()-2));
 
     // Now, build a filter out of the remaining keys
-    String regex = null;
+    List<PartitionKeyComparator.Range> ranges = new ArrayList<PartitionKeyComparator.Range>();
+    List<Operator> ops = new ArrayList<Operator>();
     if (!(partVals.size() == table.getPartitionKeys().size() && firstStar == -1)) {
-      StringBuilder buf = new StringBuilder(".*");
+
       for (int i = Math.max(0, firstStar);
            i < table.getPartitionKeys().size() && i < partVals.size(); i++) {
-        buf.append(HBaseUtils.KEY_SEPARATOR);
+
         if ("*".equals(partVals.get(i))) {
-          buf.append("[^");
-          buf.append(HBaseUtils.KEY_SEPARATOR);
-          buf.append("]+");
+          PartitionKeyComparator.Range range = new PartitionKeyComparator.Range(
+              table.getPartitionKeys().get(i).getName(),
+              new PartitionKeyComparator.Mark(partVals.get(i), true),
+              new PartitionKeyComparator.Mark(partVals.get(i), true));
+          ranges.add(range);
         } else {
-          buf.append(partVals.get(i));
+          PartitionKeyComparator.Operator op = new PartitionKeyComparator.Operator(
+              PartitionKeyComparator.Operator.Type.LIKE,
+              table.getPartitionKeys().get(i).getName(),
+              ".*");
         }
       }
-      if (partVals.size() < table.getPartitionKeys().size()) {
-        buf.append(HBaseUtils.KEY_SEPARATOR);
-        buf.append(".*");
-      }
-      regex = buf.toString();
     }
 
     Filter filter = null;
-    if (regex != null) {
-      filter = new RowFilter(CompareFilter.CompareOp.EQUAL, new RegexStringComparator(regex));
+    if (!ranges.isEmpty() || !ops.isEmpty()) {
+      filter = new RowFilter(CompareFilter.CompareOp.EQUAL, new PartitionKeyComparator(
+          StringUtils.join(HBaseUtils.getPartitionNames(table.getPartitionKeys()), ","),
+          StringUtils.join(HBaseUtils.getPartitionKeyTypes(table.getPartitionKeys()), ","),
+          ranges, ops));
     }
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Scanning partitions with prefix <" + new String(keyPrefix) + "> and filter <" +
-          regex + ">");
+          filter + ">");
     }
 
-    List<Partition> parts = scanPartitionsWithFilter(keyPrefix, HBaseUtils.getEndPrefix(keyPrefix), maxPartitions, filter);
+    List<Partition> parts = scanPartitionsWithFilter(dbName, tableName, keyPrefix,
+        HBaseUtils.getEndPrefix(keyPrefix), maxPartitions, filter);
     partCache.put(dbName, tableName, parts, false);
     return parts;
   }
 
   List<Partition> scanPartitions(String dbName, String tableName, byte[] keyStart, byte[] keyEnd,
       Filter filter, int maxPartitions) throws IOException, NoSuchObjectException {
-    List<String> keyElements = new ArrayList<>();
-    keyElements.add(dbName);
-    keyElements.add(tableName);
-
-    byte[] keyPrefix =
-        HBaseUtils.buildKeyWithTrailingSeparator(keyElements.toArray(new String[keyElements.size()]));
-    byte[] startRow = ArrayUtils.addAll(keyPrefix, keyStart);
+    byte[] startRow = keyStart;
     byte[] endRow;
     if (keyEnd == null || keyEnd.length == 0) {
       // stop when current db+table entries are over
-      endRow = HBaseUtils.getEndPrefix(keyPrefix);
+      endRow = HBaseUtils.getEndPrefix(startRow);
     } else {
-      endRow = ArrayUtils.addAll(keyPrefix, keyEnd);
+      endRow = keyEnd;
     }
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Scanning partitions with start row <" + new String(startRow) + "> and end row <"
           + new String(endRow) + ">");
     }
-    return scanPartitionsWithFilter(startRow, endRow, maxPartitions, filter);
+    return scanPartitionsWithFilter(dbName, tableName, startRow, endRow, maxPartitions, filter);
   }
 
 
@@ -762,7 +769,8 @@ public class HBaseReadWrite {
       Partition p = getPartition(dbName, tableName, partVals, false);
       decrementStorageDescriptorRefCount(p.getSd());
     }
-    byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName, partVals);
+    byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName,
+        HBaseUtils.getPartitionKeyTypes(getTable(dbName, tableName).getPartitionKeys()), partVals);
     delete(PART_TABLE, key, null, null);
   }
 
@@ -770,7 +778,8 @@ public class HBaseReadWrite {
                                  boolean populateCache) throws IOException {
     Partition cached = partCache.get(dbName, tableName, partVals);
     if (cached != null) return cached;
-    byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName, partVals);
+    byte[] key = HBaseUtils.buildPartitionKey(dbName, tableName,
+        HBaseUtils.getPartitionKeyTypes(getTable(dbName, tableName).getPartitionKeys()), partVals);
     byte[] serialized = read(PART_TABLE, key, CATALOG_CF, CATALOG_COL);
     if (serialized == null) return null;
     HBaseUtils.StorageDescriptorParts sdParts =
@@ -781,17 +790,18 @@ public class HBaseReadWrite {
     return sdParts.containingPartition;
   }
 
-  private List<Partition> scanPartitionsWithFilter(byte[] startRow, byte [] endRow,
-      int maxResults, Filter filter)
+  private List<Partition> scanPartitionsWithFilter(String dbName, String tableName,
+      byte[] startRow, byte [] endRow, int maxResults, Filter filter)
       throws IOException {
     Iterator<Result> iter =
         scan(PART_TABLE, startRow, endRow, CATALOG_CF, CATALOG_COL, filter);
+    List<FieldSchema> tablePartitions = getTable(dbName, tableName).getPartitionKeys();
     List<Partition> parts = new ArrayList<>();
     int numToFetch = maxResults < 0 ? Integer.MAX_VALUE : maxResults;
     for (int i = 0; i < numToFetch && iter.hasNext(); i++) {
       Result result = iter.next();
-      HBaseUtils.StorageDescriptorParts sdParts = HBaseUtils.deserializePartition(result.getRow(),
-          result.getValue(CATALOG_CF, CATALOG_COL));
+      HBaseUtils.StorageDescriptorParts sdParts = HBaseUtils.deserializePartition(dbName, tableName,
+          tablePartitions, result.getRow(), result.getValue(CATALOG_CF, CATALOG_COL));
       StorageDescriptor sd = getStorageDescriptor(sdParts.sdHash);
       HBaseUtils.assembleStorageDescriptor(sd, sdParts);
       parts.add(sdParts.containingPartition);
@@ -1558,7 +1568,9 @@ public class HBaseReadWrite {
 
     for (int i = 0; i < partNames.size(); i++) {
       valToPartMap.put(partVals.get(i), partNames.get(i));
-      byte[] partKey = HBaseUtils.buildPartitionKey(dbName, tblName, partVals.get(i));
+      byte[] partKey = HBaseUtils.buildPartitionKey(dbName, tblName,
+          HBaseUtils.getPartitionKeyTypes(getTable(dbName, tblName).getPartitionKeys()),
+          partVals.get(i));
       Get get = new Get(partKey);
       for (byte[] colName : colNameBytes) {
         get.addColumn(STATS_CF, colName);
@@ -1690,9 +1702,11 @@ public class HBaseReadWrite {
     return keys;
   }
 
-  private byte[] getStatisticsKey(String dbName, String tableName, List<String> partVals) {
+  private byte[] getStatisticsKey(String dbName, String tableName, List<String> partVals) throws IOException {
     return partVals == null ? HBaseUtils.buildKey(dbName, tableName) : HBaseUtils
-        .buildPartitionKey(dbName, tableName, partVals);
+        .buildPartitionKey(dbName, tableName,
+            HBaseUtils.getPartitionKeyTypes(getTable(dbName, tableName).getPartitionKeys()),
+            partVals);
   }
 
   private String getStatisticsTable(List<String> partVals) {

http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index 0204f37..717e094 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -541,7 +541,8 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Partition> oldParts = getHBase().getPartitions(db_name, tbl_name, part_vals_list);
+      List<Partition> oldParts = getHBase().getPartitions(db_name, tbl_name,
+          HBaseUtils.getPartitionKeyTypes(getTable(db_name, tbl_name).getPartitionKeys()), part_vals_list);
       getHBase().replacePartitions(oldParts, new_parts);
       for (List<String> part_vals : part_vals_list) {
         getHBase().getStatsCache().invalidate(db_name, tbl_name,
@@ -634,10 +635,8 @@ public class HBaseStore implements RawStore {
     if (table == null) {
       throw new NoSuchObjectException("Unable to find table " + dbName + "." + tblName);
     }
-    String firstPartitionColumn = table.getPartitionKeys().get(0).getName();
     // general hbase filter plan from expression tree
-    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(exprTree, firstPartitionColumn);
-
+    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(exprTree, table.getPartitionKeys());
     if (LOG.isDebugEnabled()) {
       LOG.debug("Hbase Filter Plan generated : " + planRes.plan);
     }
@@ -648,7 +647,9 @@ public class HBaseStore implements RawStore {
     for (ScanPlan splan : planRes.plan.getPlans()) {
       try {
         List<Partition> parts = getHBase().scanPartitions(dbName, tblName,
-            splan.getStartRowSuffix(), splan.getEndRowSuffix(), null, -1);
+            splan.getStartRowSuffix(dbName, tblName, table.getPartitionKeys()),
+            splan.getEndRowSuffix(dbName, tblName, table.getPartitionKeys()),
+            splan.getFilter(table.getPartitionKeys()), -1);
         boolean reachedMax = false;
         for (Partition part : parts) {
           mergedParts.put(part.getValues(), part);

http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index 62bb4de..b6fa591 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -18,11 +18,14 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
+import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
@@ -50,6 +53,19 @@ import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
+import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDeWithEndPrefix;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.BytesWritable;
 import org.apache.hive.common.util.BloomFilter;
 
 import java.io.IOException;
@@ -63,6 +79,7 @@ import java.util.Deque;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.SortedMap;
 import java.util.SortedSet;
 import java.util.TreeMap;
@@ -712,15 +729,31 @@ class HBaseUtils {
     return sd;
   }
 
+  static List<String> getPartitionKeyTypes(List<FieldSchema> parts) {
+    com.google.common.base.Function<FieldSchema, String> fieldSchemaToType =
+        new com.google.common.base.Function<FieldSchema, String>() {
+      public String apply(FieldSchema fs) { return fs.getType(); }
+    };
+    return Lists.transform(parts, fieldSchemaToType);
+  }
+
+  static List<String> getPartitionNames(List<FieldSchema> parts) {
+    com.google.common.base.Function<FieldSchema, String> fieldSchemaToName =
+        new com.google.common.base.Function<FieldSchema, String>() {
+      public String apply(FieldSchema fs) { return fs.getName(); }
+    };
+    return Lists.transform(parts, fieldSchemaToName);
+  }
+
   /**
    * Serialize a partition
    * @param part partition object
    * @param sdHash hash that is being used as a key for the enclosed storage descriptor
    * @return First element is the key, second is the serialized partition
    */
-  static byte[][] serializePartition(Partition part, byte[] sdHash) {
+  static byte[][] serializePartition(Partition part, List<String> partTypes, byte[] sdHash) {
     byte[][] result = new byte[2][];
-    result[0] = buildPartitionKey(part.getDbName(), part.getTableName(), part.getValues());
+    result[0] = buildPartitionKey(part.getDbName(), part.getTableName(), partTypes, part.getValues());
     HbaseMetastoreProto.Partition.Builder builder = HbaseMetastoreProto.Partition.newBuilder();
     builder
         .setCreateTime(part.getCreateTime())
@@ -735,11 +768,54 @@ class HBaseUtils {
     return result;
   }
 
-  static byte[] buildPartitionKey(String dbName, String tableName, List<String> partVals) {
-    Deque<String> keyParts = new ArrayDeque<>(partVals);
-    keyParts.addFirst(tableName);
-    keyParts.addFirst(dbName);
-    return buildKey(keyParts.toArray(new String[keyParts.size()]));
+  static byte[] buildPartitionKey(String dbName, String tableName, List<String> partTypes, List<String> partVals) {
+    return buildPartitionKey(dbName, tableName, partTypes, partVals, false);
+  }
+
+  static byte[] buildPartitionKey(String dbName, String tableName, List<String> partTypes, List<String> partVals, boolean endPrefix) {
+    Object[] components = new Object[partVals.size()];
+    for (int i=0;i<partVals.size();i++) {
+      TypeInfo expectedType =
+          TypeInfoUtils.getTypeInfoFromTypeString(partTypes.get(i));
+      ObjectInspector outputOI =
+          TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(expectedType);
+      Converter converter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaStringObjectInspector, outputOI);
+      components[i] = converter.convert(partVals.get(i));
+    }
+
+    return buildSerializedPartitionKey(dbName, tableName, partTypes, components, endPrefix);
+  }
+
+  static byte[] buildSerializedPartitionKey(String dbName, String tableName, List<String> partTypes, Object[] components, boolean endPrefix) {
+    ObjectInspector javaStringOI =
+        PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(PrimitiveCategory.STRING);
+    Object[] data = new Object[components.length+2];
+    List<ObjectInspector> fois = new ArrayList<ObjectInspector>(components.length+2);
+    boolean[] endPrefixes = new boolean[components.length+2];
+
+    data[0] = dbName;
+    fois.add(javaStringOI);
+    endPrefixes[0] = false;
+    data[1] = tableName;
+    fois.add(javaStringOI);
+    endPrefixes[1] = false;
+
+    for (int i = 0; i < components.length; i++) {
+      data[i+2] = components[i];
+      TypeInfo expectedType =
+          TypeInfoUtils.getTypeInfoFromTypeString(partTypes.get(i));
+      ObjectInspector outputOI =
+          TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(expectedType);
+      fois.add(outputOI);
+    }
+    Output output = new Output();
+    try {
+      BinarySortableSerDeWithEndPrefix.serializeStruct(output, data, fois, endPrefix);
+    } catch (SerDeException e) {
+      throw new RuntimeException("Cannot serialize partition " + StringUtils.join(components, ","));
+    }
+    return Arrays.copyOf(output.getData(), output.getLength());
   }
 
   static class StorageDescriptorParts {
@@ -771,11 +847,10 @@ class HBaseUtils {
    * @param serialized the value fetched from HBase
    * @return A struct that contains the partition plus parts of the storage descriptor
    */
-  static StorageDescriptorParts deserializePartition(byte[] key, byte[] serialized)
-      throws InvalidProtocolBufferException {
-    String[] keys = deserializeKey(key);
-    return deserializePartition(keys[0], keys[1],
-        Arrays.asList(Arrays.copyOfRange(keys, 2, keys.length)), serialized);
+  static StorageDescriptorParts deserializePartition(String dbName, String tableName, List<FieldSchema> partitions,
+      byte[] key, byte[] serialized) throws InvalidProtocolBufferException {
+    List keys = deserializePartitionKey(partitions, key);
+    return deserializePartition(dbName, tableName, keys, serialized);
   }
 
   /**
@@ -811,6 +886,36 @@ class HBaseUtils {
     return k.split(KEY_SEPARATOR_STR);
   }
 
+  private static List<String> deserializePartitionKey(List<FieldSchema> partitions, byte[] key) {
+    StringBuffer names = new StringBuffer();
+    names.append("dbName,tableName,");
+    StringBuffer types = new StringBuffer();
+    types.append("string,string,");
+    for (int i=0;i<partitions.size();i++) {
+      names.append(partitions.get(i).getName());
+      types.append(TypeInfoUtils.getTypeInfoFromTypeString(partitions.get(i).getType()));
+      if (i!=partitions.size()-1) {
+        names.append(",");
+        types.append(",");
+      }
+    }
+    BinarySortableSerDe serDe = new BinarySortableSerDe();
+    Properties props = new Properties();
+    props.setProperty(serdeConstants.LIST_COLUMNS, names.toString());
+    props.setProperty(serdeConstants.LIST_COLUMN_TYPES, types.toString());
+    try {
+      serDe.initialize(new Configuration(), props);
+      List deserializedkeys = ((List)serDe.deserialize(new BytesWritable(key))).subList(2, partitions.size()+2);
+      List<String> partitionKeys = new ArrayList<String>();
+      for (Object deserializedKey : deserializedkeys) {
+        partitionKeys.add(deserializedKey.toString());
+      }
+      return partitionKeys;
+    } catch (SerDeException e) {
+      throw new RuntimeException("Error when deserialize key", e);
+    }
+  }
+
   /**
    * Serialize a table
    * @param table table object

http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/PartitionKeyComparator.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/PartitionKeyComparator.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/PartitionKeyComparator.java
new file mode 100644
index 0000000..01fe403
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/PartitionKeyComparator.java
@@ -0,0 +1,292 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hive.metastore.hbase;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.filter.ByteArrayComparable;
+import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PartitionKeyComparator.Operator.Type;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.BytesWritable;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+public class PartitionKeyComparator extends ByteArrayComparable {
+  private static final Log LOG = LogFactory.getLog(PartitionKeyComparator.class);
+  static class Mark {
+    Mark(String value, boolean inclusive) {
+      this.value = value;
+      this.inclusive = inclusive;
+    }
+    String value;
+    boolean inclusive;
+    public String toString() {
+      return value + (inclusive?"_":"");
+    }
+  }
+  static class Range {
+    Range(String keyName, Mark start, Mark end) {
+      this.keyName = keyName;
+      this.start = start;
+      this.end = end;
+    }
+    String keyName;
+    Mark start;
+    Mark end;
+    public String toString() {
+      return "" + keyName + ":" + (start!=null?start.toString():"") + (end!=null?end.toString():"");
+    }
+  }
+  // Cache the information derived from ranges for performance, including
+  // range in native datatype
+  static class NativeRange {
+    int pos;
+    Comparable start;
+    Comparable end;
+  }
+  static class Operator {
+    public Operator(Type type, String keyName, String val) {
+      this.type = type;
+      this.keyName = keyName;
+      this.val = val;
+    }
+    enum Type {
+      LIKE, NOTEQUALS
+    };
+    Type type;
+    String keyName;
+    String val;
+  }
+  static class NativeOperator {
+    int pos;
+    Comparable val;
+  }
+  String names;
+  String types;
+  List<Range> ranges;
+  List<NativeRange> nativeRanges;
+  List<Operator> ops;
+  List<NativeOperator> nativeOps;
+  Properties serdeProps;
+  public PartitionKeyComparator(String names, String types, List<Range> ranges, List<Operator> ops) {
+    super(null);
+    this.names = names;
+    this.types = types;
+    this.ranges = ranges;
+    this.ops = ops;
+    serdeProps = new Properties();
+    serdeProps.setProperty(serdeConstants.LIST_COLUMNS, "dbName,tableName," + names);
+    serdeProps.setProperty(serdeConstants.LIST_COLUMN_TYPES, "string,string," + types);
+
+    this.nativeRanges = new ArrayList<NativeRange>(this.ranges.size());
+    for (int i=0;i<ranges.size();i++) {
+      Range range = ranges.get(i);
+      NativeRange nativeRange = new NativeRange();;
+      nativeRanges.add(i, nativeRange);
+      nativeRange.pos = Arrays.asList(names.split(",")).indexOf(range.keyName);
+      TypeInfo expectedType =
+          TypeInfoUtils.getTypeInfoFromTypeString(types.split(",")[nativeRange.pos]);
+      ObjectInspector outputOI =
+          TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(expectedType);
+      nativeRange.start = null;
+      if (range.start != null) {
+        Converter converter = ObjectInspectorConverters.getConverter(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, outputOI);
+        nativeRange.start = (Comparable)converter.convert(range.start.value);
+      }
+      nativeRange.end = null;
+      if (range.end != null) {
+        Converter converter = ObjectInspectorConverters.getConverter(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, outputOI);
+        nativeRange.end = (Comparable)converter.convert(range.end.value);
+      }
+    }
+
+    this.nativeOps = new ArrayList<NativeOperator>(this.ops.size());
+    for (int i=0;i<ops.size();i++) {
+      Operator op = ops.get(i);
+      NativeOperator nativeOp = new NativeOperator();
+      nativeOps.add(i, nativeOp);
+      nativeOp.pos = ArrayUtils.indexOf(names.split(","), op.keyName);
+      TypeInfo expectedType =
+          TypeInfoUtils.getTypeInfoFromTypeString(types.split(",")[nativeOp.pos]);
+      ObjectInspector outputOI =
+          TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(expectedType);
+      Converter converter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaStringObjectInspector, outputOI);
+      nativeOp.val = (Comparable)converter.convert(op.val);
+    }
+  }
+
+  public static PartitionKeyComparator parseFrom(final byte [] bytes) {
+    HbaseMetastoreProto.PartitionKeyComparator proto;
+    try {
+      proto = HbaseMetastoreProto.PartitionKeyComparator.parseFrom(bytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new RuntimeException(e);
+    }
+    List<Range> ranges = new ArrayList<Range>();
+    for (HbaseMetastoreProto.PartitionKeyComparator.Range range : proto.getRangeList()) {
+      Mark start = null;
+      if (range.hasStart()) {
+        start = new Mark(range.getStart().getValue(), range.getStart().getInclusive());
+      }
+      Mark end = null;
+      if (range.hasEnd()) {
+        end = new Mark(range.getEnd().getValue(), range.getEnd().getInclusive());
+      }
+      ranges.add(new Range(range.getKey(), start, end));
+    }
+    List<Operator> ops = new ArrayList<Operator>();
+    for (HbaseMetastoreProto.PartitionKeyComparator.Operator op : proto.getOpList()) {
+      ops.add(new Operator(Operator.Type.valueOf(op.getType().name()), op.getKey(),
+          op.getVal()));
+    }
+    return new PartitionKeyComparator(proto.getNames(), proto.getTypes(), ranges, ops);
+  }
+
+  @Override
+  public byte[] toByteArray() {
+    HbaseMetastoreProto.PartitionKeyComparator.Builder builder = 
+        HbaseMetastoreProto.PartitionKeyComparator.newBuilder();
+    builder.setNames(names);
+    builder.setTypes(types);
+    for (int i=0;i<ranges.size();i++) {
+      Range range = ranges.get(i);
+      HbaseMetastoreProto.PartitionKeyComparator.Mark startMark = null;
+      if (range.start != null) {
+        startMark = HbaseMetastoreProto.PartitionKeyComparator.Mark.newBuilder()
+          .setValue(range.start.value)
+          .setInclusive(range.start.inclusive)
+          .build();
+      }
+      HbaseMetastoreProto.PartitionKeyComparator.Mark endMark = null;
+      if (range.end != null) {
+        endMark = HbaseMetastoreProto.PartitionKeyComparator.Mark.newBuilder()
+          .setValue(range.end.value)
+          .setInclusive(range.end.inclusive)
+          .build();
+      }
+        
+      HbaseMetastoreProto.PartitionKeyComparator.Range.Builder rangeBuilder = 
+        HbaseMetastoreProto.PartitionKeyComparator.Range.newBuilder();
+      rangeBuilder.setKey(range.keyName);
+      if (startMark != null) {
+        rangeBuilder.setStart(startMark);
+      }
+      if (endMark != null) {
+        rangeBuilder.setEnd(endMark);
+      }
+      builder.addRange(rangeBuilder.build());
+    }
+    for (int i=0;i<ops.size();i++) {
+      Operator op = ops.get(i);
+      builder.addOp(HbaseMetastoreProto.PartitionKeyComparator.Operator.newBuilder()
+        .setKey(op.keyName)
+        .setType(Type.valueOf(op.type.toString()))
+        .setVal(op.val).build());
+    }
+    return builder.build().toByteArray();
+  }
+
+  @Override
+  public int compareTo(byte[] value, int offset, int length) {
+    byte[] bytes = Arrays.copyOfRange(value, offset, offset + length);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Get key " + new String(bytes));
+    }
+    BinarySortableSerDe serDe = new BinarySortableSerDe();
+    List deserializedkeys = null;
+    try {
+      serDe.initialize(new Configuration(), serdeProps);
+      deserializedkeys = ((List)serDe.deserialize(new BytesWritable(bytes))).subList(2, 2 + names.split(",").length);
+    } catch (SerDeException e) {
+      // don't bother with failed deserialization, continue with next key
+      return 1;
+    }
+    for (int i=0;i<ranges.size();i++) {
+      Range range = ranges.get(i);
+      NativeRange nativeRange = nativeRanges.get(i);
+
+      Comparable partVal = (Comparable)deserializedkeys.get(nativeRange.pos);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Try to match range " + partVal + ", start " + nativeRange.start + ", end "
+            + nativeRange.end);
+      }
+      if (range.start == null || range.start.inclusive && partVal.compareTo(nativeRange.start)>=0 ||
+          !range.start.inclusive && partVal.compareTo(nativeRange.start)>0) {
+        if (range.end == null || range.end.inclusive && partVal.compareTo(nativeRange.end)<=0 ||
+            !range.end.inclusive && partVal.compareTo(nativeRange.end)<0) {
+          continue;
+        }
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Fail to match range " + range.keyName + "-" + partVal + "[" + nativeRange.start
+            + "," + nativeRange.end + "]");
+      }
+      return 1;
+    }
+
+    for (int i=0;i<ops.size();i++) {
+      Operator op = ops.get(i);
+      NativeOperator nativeOp = nativeOps.get(i);
+      switch (op.type) {
+      case LIKE:
+        if (!deserializedkeys.get(nativeOp.pos).toString().matches(op.val)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Fail to match operator " + op.keyName + "(" + deserializedkeys.get(nativeOp.pos)
+                + ") LIKE " + nativeOp.val);
+          }
+          return 1;
+        }
+        break;
+      case NOTEQUALS:
+        if (nativeOp.val.equals(deserializedkeys.get(nativeOp.pos))) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Fail to match operator " + op.keyName + "(" + deserializedkeys.get(nativeOp.pos)
+                + ")!=" + nativeOp.val);
+          }
+          return 1;
+        }
+        break;
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("All conditions satisfied:" + deserializedkeys);
+    }
+    return 0;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
----------------------------------------------------------------------
diff --git a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
index cba3671..0d0ef89 100644
--- a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
+++ b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
@@ -255,3 +255,28 @@ message Table {
   optional PrincipalPrivilegeSet privileges = 13;
   optional bool is_temporary = 14;
 }
+
+message PartitionKeyComparator {
+  required string names = 1;
+  required string types = 2;
+  message Mark {
+    required string value = 1;
+    required bool inclusive = 2;
+  }
+  message Range {
+    required string key = 1;
+    optional Mark start = 2;
+    optional Mark end = 3;
+  }
+  message Operator {
+    enum Type {
+      LIKE = 0;
+      NOTEQUALS = 1;
+    }
+    required Type type = 1;
+    required string key = 2;
+    required string val = 3;
+  }
+  repeated Operator op = 3;
+  repeated Range range = 4;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseFilterPlanUtil.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseFilterPlanUtil.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseFilterPlanUtil.java
index 5943d14..06884b3 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseFilterPlanUtil.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseFilterPlanUtil.java
@@ -18,12 +18,17 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.FilterPlan;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.MultiScanPlan;
-import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.PartitionFilterGenerator;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.PlanResult;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.ScanPlan;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.ScanPlan.ScanMarker;
@@ -35,6 +40,8 @@ import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeNode;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.primitives.Shorts;
+
 public class TestHBaseFilterPlanUtil {
   final boolean INCLUSIVE = true;
 
@@ -68,31 +75,28 @@ public class TestHBaseFilterPlanUtil {
     ScanMarker r;
 
     // equal plans
-    l = new ScanMarker(new byte[] { 1, 2 }, INCLUSIVE);
-    r = new ScanMarker(new byte[] { 1, 2 }, INCLUSIVE);
+    l = new ScanMarker("1", INCLUSIVE, "int");
+    r = new ScanMarker("1", INCLUSIVE, "int");
     assertFirstGreater(l, r);
 
-    l = new ScanMarker(new byte[] { 1, 2 }, !INCLUSIVE);
-    r = new ScanMarker(new byte[] { 1, 2 }, !INCLUSIVE);
+    l = new ScanMarker("1", !INCLUSIVE, "int");
+    r = new ScanMarker("1", !INCLUSIVE, "int");
     assertFirstGreater(l, r);
 
-    l = new ScanMarker(null, !INCLUSIVE);
-    r = new ScanMarker(null, !INCLUSIVE);
-    assertFirstGreater(l, r);
+    assertFirstGreater(null, null);
 
     // create l is greater because of inclusive flag
-    l = new ScanMarker(new byte[] { 1, 2 }, !INCLUSIVE);
-    r = new ScanMarker(null, !INCLUSIVE);
+    l = new ScanMarker("1", !INCLUSIVE, "int");
     // the rule for null vs non-null is different
     // non-null is both smaller and greater than null
-    Assert.assertEquals(l, ScanPlan.getComparedMarker(l, r, true));
-    Assert.assertEquals(l, ScanPlan.getComparedMarker(r, l, true));
-    Assert.assertEquals(l, ScanPlan.getComparedMarker(l, r, false));
-    Assert.assertEquals(l, ScanPlan.getComparedMarker(r, l, false));
+    Assert.assertEquals(l, ScanPlan.getComparedMarker(l, null, true));
+    Assert.assertEquals(l, ScanPlan.getComparedMarker(null, l, true));
+    Assert.assertEquals(l, ScanPlan.getComparedMarker(l, null, false));
+    Assert.assertEquals(l, ScanPlan.getComparedMarker(null, l, false));
 
     // create l that is greater because of the bytes
-    l = new ScanMarker(new byte[] { 1, 2, 0 }, INCLUSIVE);
-    r = new ScanMarker(new byte[] { 1, 2 }, INCLUSIVE);
+    l = new ScanMarker("2", INCLUSIVE, "int");
+    r = new ScanMarker("1", INCLUSIVE, "int");
     assertFirstGreater(l, r);
 
   }
@@ -111,36 +115,30 @@ public class TestHBaseFilterPlanUtil {
   public void testScanPlanAnd() {
     ScanPlan l = new ScanPlan();
     ScanPlan r = new ScanPlan();
-    l.setStartMarker(new ScanMarker(new byte[] { 10 }, INCLUSIVE));
-    r.setStartMarker(new ScanMarker(new byte[] { 10 }, INCLUSIVE));
+    l.setStartMarker("a", "int", "10", INCLUSIVE);
+    r.setStartMarker("a", "int", "10", INCLUSIVE);
 
     ScanPlan res;
     // both equal
     res = l.and(r).getPlans().get(0);
-    Assert.assertEquals(new ScanMarker(new byte[] { 10 }, INCLUSIVE), res.getStartMarker());
+    Assert.assertEquals(new ScanMarker("10", INCLUSIVE, "int"), res.markers.get("a").startMarker);
 
     // add equal end markers as well, and test AND again
-    l.setEndMarker(new ScanMarker(new byte[] { 20 }, INCLUSIVE));
-    r.setEndMarker(new ScanMarker(new byte[] { 20 }, INCLUSIVE));
+    l.setEndMarker("a", "int", "20", INCLUSIVE);
+    r.setEndMarker("a", "int", "20", INCLUSIVE);
     res = l.and(r).getPlans().get(0);
-    Assert.assertEquals(new ScanMarker(new byte[] { 10 }, INCLUSIVE), res.getStartMarker());
-    Assert.assertEquals(new ScanMarker(new byte[] { 20 }, INCLUSIVE), res.getEndMarker());
-
-    l.setEndMarker(new ScanMarker(null, INCLUSIVE));
-    r.setStartMarker(new ScanMarker(null, !INCLUSIVE));
-    // markers with non null bytes are both lesser and greator
-    Assert.assertEquals(l.getStartMarker(), res.getStartMarker());
-    Assert.assertEquals(r.getEndMarker(), res.getEndMarker());
+    Assert.assertEquals(new ScanMarker("10", INCLUSIVE, "int"), res.markers.get("a").startMarker);
+    Assert.assertEquals(new ScanMarker("20", INCLUSIVE, "int"), res.markers.get("a").endMarker);
 
-    l.setStartMarker(new ScanMarker(new byte[] { 10, 11 }, !INCLUSIVE));
-    l.setEndMarker(new ScanMarker(new byte[] { 20, 21 }, INCLUSIVE));
+    l.setStartMarker("a", "int", "10", !INCLUSIVE);
+    l.setEndMarker("a", "int", "20", INCLUSIVE);
 
-    r.setStartMarker(new ScanMarker(new byte[] { 10, 10 }, INCLUSIVE));
-    r.setEndMarker(new ScanMarker(new byte[] { 15 }, INCLUSIVE));
+    r.setStartMarker("a", "int", "10", INCLUSIVE);
+    r.setEndMarker("a", "int", "15", INCLUSIVE);
     res = l.and(r).getPlans().get(0);
     // start of l is greater, end of r is smaller
-    Assert.assertEquals(l.getStartMarker(), res.getStartMarker());
-    Assert.assertEquals(r.getEndMarker(), res.getEndMarker());
+    Assert.assertEquals(l.markers.get("a").startMarker, res.markers.get("a").startMarker);
+    Assert.assertEquals(r.markers.get("a").endMarker, res.markers.get("a").endMarker);
 
   }
 
@@ -151,13 +149,13 @@ public class TestHBaseFilterPlanUtil {
   public void testScanPlanOr() {
     ScanPlan l = new ScanPlan();
     ScanPlan r = new ScanPlan();
-    l.setStartMarker(new ScanMarker(new byte[] { 10 }, INCLUSIVE));
-    r.setStartMarker(new ScanMarker(new byte[] { 11 }, INCLUSIVE));
+    l.setStartMarker("a", "int", "1", INCLUSIVE);
+    r.setStartMarker("a", "int", "11", INCLUSIVE);
 
     FilterPlan res1 = l.or(r);
     Assert.assertEquals(2, res1.getPlans().size());
-    res1.getPlans().get(0).getStartMarker().equals(l.getStartMarker());
-    res1.getPlans().get(1).getStartMarker().equals(r.getStartMarker());
+    res1.getPlans().get(0).markers.get("a").startMarker.equals(l.markers.get("a").startMarker);
+    res1.getPlans().get(1).markers.get("a").startMarker.equals(r.markers.get("a").startMarker);
 
     FilterPlan res2 = res1.or(r);
     Assert.assertEquals(3, res2.getPlans().size());
@@ -223,72 +221,71 @@ public class TestHBaseFilterPlanUtil {
 
     final String KEY = "k1";
     final String VAL = "v1";
-    final byte[] VAL_BYTES = PartitionFilterGenerator.toBytes(VAL);
+    final String OTHERKEY = "k2";
     LeafNode l = new LeafNode();
     l.keyName = KEY;
     l.value = VAL;
-    final ScanMarker DEFAULT_SCANMARKER = new ScanMarker(null, false);
+    final ScanMarker DEFAULT_SCANMARKER = null;
+    List<FieldSchema> parts = new ArrayList<FieldSchema>();
+    parts.add(new FieldSchema(KEY, "int", null));
+    parts.add(new FieldSchema(OTHERKEY, "int", null));
 
     l.operator = Operator.EQUALS;
-    verifyPlan(l, KEY, new ScanMarker(VAL_BYTES, INCLUSIVE), new ScanMarker(VAL_BYTES, INCLUSIVE));
+    verifyPlan(l, parts, KEY, new ScanMarker(VAL, INCLUSIVE, "int"), new ScanMarker(VAL, INCLUSIVE, "int"));
 
     l.operator = Operator.GREATERTHAN;
-    verifyPlan(l, KEY, new ScanMarker(VAL_BYTES, !INCLUSIVE), DEFAULT_SCANMARKER);
+    verifyPlan(l, parts, KEY, new ScanMarker(VAL, !INCLUSIVE, "int"), DEFAULT_SCANMARKER);
 
     l.operator = Operator.GREATERTHANOREQUALTO;
-    verifyPlan(l, KEY, new ScanMarker(VAL_BYTES, INCLUSIVE), DEFAULT_SCANMARKER);
+    verifyPlan(l, parts, KEY, new ScanMarker(VAL, INCLUSIVE, "int"), DEFAULT_SCANMARKER);
 
     l.operator = Operator.LESSTHAN;
-    verifyPlan(l, KEY, DEFAULT_SCANMARKER, new ScanMarker(VAL_BYTES, !INCLUSIVE));
+    verifyPlan(l, parts, KEY, DEFAULT_SCANMARKER, new ScanMarker(VAL, !INCLUSIVE, "int"));
 
     l.operator = Operator.LESSTHANOREQUALTO;
-    verifyPlan(l, KEY, DEFAULT_SCANMARKER, new ScanMarker(VAL_BYTES, INCLUSIVE));
-
-    // following leaf node plans should currently have true for 'has unsupported condition',
-    // because of the unsupported operator
-    l.operator = Operator.NOTEQUALS;
-    verifyPlan(l, KEY, DEFAULT_SCANMARKER, DEFAULT_SCANMARKER, true);
-
-    l.operator = Operator.NOTEQUALS2;
-    verifyPlan(l, KEY, DEFAULT_SCANMARKER, DEFAULT_SCANMARKER, true);
-
-    l.operator = Operator.LIKE;
-    verifyPlan(l, KEY, DEFAULT_SCANMARKER, DEFAULT_SCANMARKER, true);
+    verifyPlan(l, parts, KEY, DEFAULT_SCANMARKER, new ScanMarker(VAL, INCLUSIVE, "int"));
 
     // following leaf node plans should currently have true for 'has unsupported condition',
     // because of the condition is not on first key
     l.operator = Operator.EQUALS;
-    verifyPlan(l, "NOT_FIRST_PART", DEFAULT_SCANMARKER, DEFAULT_SCANMARKER, true);
-
-    l.operator = Operator.NOTEQUALS;
-    verifyPlan(l, "NOT_FIRST_PART", DEFAULT_SCANMARKER, DEFAULT_SCANMARKER, true);
+    verifyPlan(l, parts, OTHERKEY, DEFAULT_SCANMARKER, DEFAULT_SCANMARKER, false);
 
     // if tree is null, it should return equivalent of full scan, and true
     // for 'has unsupported condition'
-    verifyPlan(null, KEY, DEFAULT_SCANMARKER, DEFAULT_SCANMARKER, true);
+    verifyPlan(null, parts, KEY, DEFAULT_SCANMARKER, DEFAULT_SCANMARKER, true);
 
   }
 
-  private void verifyPlan(TreeNode l, String keyName, ScanMarker startMarker, ScanMarker endMarker)
+  private void verifyPlan(TreeNode l, List<FieldSchema> parts, String keyName, ScanMarker startMarker, ScanMarker endMarker)
       throws MetaException {
-    verifyPlan(l, keyName, startMarker, endMarker, false);
+    verifyPlan(l, parts, keyName, startMarker, endMarker, false);
   }
 
-  private void verifyPlan(TreeNode l, String keyName, ScanMarker startMarker, ScanMarker endMarker,
+  private void verifyPlan(TreeNode l, List<FieldSchema> parts, String keyName, ScanMarker startMarker, ScanMarker endMarker,
       boolean hasUnsupportedCondition) throws MetaException {
     ExpressionTree e = null;
     if (l != null) {
       e = new ExpressionTree();
       e.setRootForTest(l);
     }
-    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(e, keyName);
+    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(e, parts);
     FilterPlan plan = planRes.plan;
     Assert.assertEquals("Has unsupported condition", hasUnsupportedCondition,
         planRes.hasUnsupportedCondition);
     Assert.assertEquals(1, plan.getPlans().size());
     ScanPlan splan = plan.getPlans().get(0);
-    Assert.assertEquals(startMarker, splan.getStartMarker());
-    Assert.assertEquals(endMarker, splan.getEndMarker());
+    if (startMarker != null) {
+      Assert.assertEquals(startMarker, splan.markers.get(keyName).startMarker);
+    } else {
+      Assert.assertTrue(splan.markers.get(keyName)==null ||
+          splan.markers.get(keyName).startMarker==null);
+    }
+    if (endMarker != null) {
+      Assert.assertEquals(endMarker, splan.markers.get(keyName).endMarker);
+    } else {
+      Assert.assertTrue(splan.markers.get(keyName)==null ||
+          splan.markers.get(keyName).endMarker==null);
+    }
   }
 
   /**
@@ -302,12 +299,13 @@ public class TestHBaseFilterPlanUtil {
     final String KEY = "k1";
     final String VAL1 = "10";
     final String VAL2 = "11";
-    final byte[] VAL1_BYTES = PartitionFilterGenerator.toBytes(VAL1);
-    final byte[] VAL2_BYTES = PartitionFilterGenerator.toBytes(VAL2);
     LeafNode l = new LeafNode();
     l.keyName = KEY;
     l.value = VAL1;
-    final ScanMarker DEFAULT_SCANMARKER = new ScanMarker(null, false);
+    final ScanMarker DEFAULT_SCANMARKER = null;
+
+    List<FieldSchema> parts = new ArrayList<FieldSchema>();
+    parts.add(new FieldSchema("k1", "int", null));
 
     LeafNode r = new LeafNode();
     r.keyName = KEY;
@@ -318,19 +316,19 @@ public class TestHBaseFilterPlanUtil {
     // verify plan for - k1 >= '10' and k1 < '11'
     l.operator = Operator.GREATERTHANOREQUALTO;
     r.operator = Operator.LESSTHAN;
-    verifyPlan(tn, KEY, new ScanMarker(VAL1_BYTES, INCLUSIVE), new ScanMarker(VAL2_BYTES,
-        !INCLUSIVE));
+    verifyPlan(tn, parts, KEY, new ScanMarker(VAL1, INCLUSIVE, "int"), new ScanMarker(VAL2,
+        !INCLUSIVE, "int"));
 
     // verify plan for - k1 >= '10' and k1 > '11'
     l.operator = Operator.GREATERTHANOREQUALTO;
     r.operator = Operator.GREATERTHAN;
-    verifyPlan(tn, KEY, new ScanMarker(VAL2_BYTES, !INCLUSIVE), DEFAULT_SCANMARKER);
+    verifyPlan(tn, parts, KEY, new ScanMarker(VAL2, !INCLUSIVE, "int"), DEFAULT_SCANMARKER);
 
     // verify plan for - k1 >= '10' or k1 > '11'
     tn = new TreeNode(l, LogicalOperator.OR, r);
     ExpressionTree e = new ExpressionTree();
     e.setRootForTest(tn);
-    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(e, KEY);
+    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(e, parts);
     Assert.assertEquals(2, planRes.plan.getPlans().size());
     Assert.assertEquals(false, planRes.hasUnsupportedCondition);
 
@@ -338,7 +336,7 @@ public class TestHBaseFilterPlanUtil {
     TreeNode tn2 = new TreeNode(l, LogicalOperator.AND, tn);
     e = new ExpressionTree();
     e.setRootForTest(tn2);
-    planRes = HBaseFilterPlanUtil.getFilterPlan(e, KEY);
+    planRes = HBaseFilterPlanUtil.getFilterPlan(e, parts);
     Assert.assertEquals(2, planRes.plan.getPlans().size());
     Assert.assertEquals(false, planRes.hasUnsupportedCondition);
 
@@ -351,11 +349,135 @@ public class TestHBaseFilterPlanUtil {
     TreeNode tn3 = new TreeNode(tn2, LogicalOperator.OR, klike);
     e = new ExpressionTree();
     e.setRootForTest(tn3);
-    planRes = HBaseFilterPlanUtil.getFilterPlan(e, KEY);
+    planRes = HBaseFilterPlanUtil.getFilterPlan(e, parts);
     Assert.assertEquals(3, planRes.plan.getPlans().size());
-    Assert.assertEquals(true, planRes.hasUnsupportedCondition);
+    Assert.assertEquals(false, planRes.hasUnsupportedCondition);
+
 
+  }
 
+  @Test
+  public void testPartitionKeyScannerAllString() throws Exception {
+    List<FieldSchema> parts = new ArrayList<FieldSchema>();
+    parts.add(new FieldSchema("year", "string", null));
+    parts.add(new FieldSchema("month", "string", null));
+    parts.add(new FieldSchema("state", "string", null));
+
+    // One prefix key and one minor key range
+    ExpressionTree exprTree = PartFilterExprUtil.getFilterParser("year = 2015 and state = 'CA'").tree;
+    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(exprTree, parts);
+
+    Assert.assertEquals(planRes.plan.getPlans().size(), 1);
+
+    ScanPlan sp = planRes.plan.getPlans().get(0);
+    byte[] startRowSuffix = sp.getStartRowSuffix("testdb", "testtb", parts);
+    byte[] endRowSuffix = sp.getEndRowSuffix("testdb", "testtb", parts);
+    RowFilter filter = (RowFilter)sp.getFilter(parts);
+
+    // scan range contains the major key year, rowfilter contains minor key state
+    Assert.assertTrue(Bytes.contains(startRowSuffix, "2015".getBytes()));
+    Assert.assertTrue(Bytes.contains(endRowSuffix, "2015".getBytes()));
+    Assert.assertFalse(Bytes.contains(startRowSuffix, "CA".getBytes()));
+    Assert.assertFalse(Bytes.contains(endRowSuffix, "CA".getBytes()));
+
+    PartitionKeyComparator comparator = (PartitionKeyComparator)filter.getComparator();
+    Assert.assertEquals(comparator.ranges.size(), 1);
+    Assert.assertEquals(comparator.ranges.get(0).keyName, "state");
+
+    // Two prefix key and one LIKE operator
+    exprTree = PartFilterExprUtil.getFilterParser("year = 2015 and month > 10 "
+        + "and month <= 11 and state like 'C%'").tree;
+    planRes = HBaseFilterPlanUtil.getFilterPlan(exprTree, parts);
+
+    Assert.assertEquals(planRes.plan.getPlans().size(), 1);
+
+    sp = planRes.plan.getPlans().get(0);
+    startRowSuffix = sp.getStartRowSuffix("testdb", "testtb", parts);
+    endRowSuffix = sp.getEndRowSuffix("testdb", "testtb", parts);
+    filter = (RowFilter)sp.getFilter(parts);
+
+    // scan range contains the major key value year/month, rowfilter contains LIKE operator
+    Assert.assertTrue(Bytes.contains(startRowSuffix, "2015".getBytes()));
+    Assert.assertTrue(Bytes.contains(endRowSuffix, "2015".getBytes()));
+    Assert.assertTrue(Bytes.contains(startRowSuffix, "10".getBytes()));
+    Assert.assertTrue(Bytes.contains(endRowSuffix, "11".getBytes()));
+
+    comparator = (PartitionKeyComparator)filter.getComparator();
+    Assert.assertEquals(comparator.ops.size(), 1);
+    Assert.assertEquals(comparator.ops.get(0).keyName, "state");
+
+    // One prefix key, one minor key range and one LIKE operator
+    exprTree = PartFilterExprUtil.getFilterParser("year >= 2014 and month > 10 "
+        + "and month <= 11 and state like 'C%'").tree;
+    planRes = HBaseFilterPlanUtil.getFilterPlan(exprTree, parts);
+
+    Assert.assertEquals(planRes.plan.getPlans().size(), 1);
+
+    sp = planRes.plan.getPlans().get(0);
+    startRowSuffix = sp.getStartRowSuffix("testdb", "testtb", parts);
+    endRowSuffix = sp.getEndRowSuffix("testdb", "testtb", parts);
+    filter = (RowFilter)sp.getFilter(parts);
+
+    // scan range contains the major key value year (low bound), rowfilter contains minor key state
+    // and LIKE operator
+    Assert.assertTrue(Bytes.contains(startRowSuffix, "2014".getBytes()));
+
+    comparator = (PartitionKeyComparator)filter.getComparator();
+    Assert.assertEquals(comparator.ranges.size(), 1);
+    Assert.assertEquals(comparator.ranges.get(0).keyName, "month");
+    Assert.assertEquals(comparator.ops.size(), 1);
+    Assert.assertEquals(comparator.ops.get(0).keyName, "state");
+
+    // Condition contains or
+    exprTree = PartFilterExprUtil.getFilterParser("year = 2014 and (month > 10 "
+        + "or month < 3)").tree;
+    planRes = HBaseFilterPlanUtil.getFilterPlan(exprTree, parts);
+
+    sp = planRes.plan.getPlans().get(0);
+    startRowSuffix = sp.getStartRowSuffix("testdb", "testtb", parts);
+    endRowSuffix = sp.getEndRowSuffix("testdb", "testtb", parts);
+    filter = (RowFilter)sp.getFilter(parts);
+
+    // The first ScanPlan contains year = 2014 and month > 10
+    Assert.assertTrue(Bytes.contains(startRowSuffix, "2014".getBytes()));
+    Assert.assertTrue(Bytes.contains(endRowSuffix, "2014".getBytes()));
+    Assert.assertTrue(Bytes.contains(startRowSuffix, "10".getBytes()));
+
+    sp = planRes.plan.getPlans().get(1);
+    startRowSuffix = sp.getStartRowSuffix("testdb", "testtb", parts);
+    endRowSuffix = sp.getEndRowSuffix("testdb", "testtb", parts);
+    filter = (RowFilter)sp.getFilter(parts);
+
+    // The first ScanPlan contains year = 2014 and month < 3
+    Assert.assertTrue(Bytes.contains(startRowSuffix, "2014".getBytes()));
+    Assert.assertTrue(Bytes.contains(endRowSuffix, "2014".getBytes()));
+    Assert.assertTrue(Bytes.contains(endRowSuffix, "3".getBytes()));
   }
 
+  @Test
+  public void testPartitionKeyScannerMixedType() throws Exception {
+    List<FieldSchema> parts = new ArrayList<FieldSchema>();
+    parts.add(new FieldSchema("year", "int", null));
+    parts.add(new FieldSchema("month", "int", null));
+    parts.add(new FieldSchema("state", "string", null));
+
+    // One prefix key and one minor key range
+    ExpressionTree exprTree = PartFilterExprUtil.getFilterParser("year = 2015 and state = 'CA'").tree;
+    PlanResult planRes = HBaseFilterPlanUtil.getFilterPlan(exprTree, parts);
+
+    Assert.assertEquals(planRes.plan.getPlans().size(), 1);
+
+    ScanPlan sp = planRes.plan.getPlans().get(0);
+    byte[] startRowSuffix = sp.getStartRowSuffix("testdb", "testtb", parts);
+    byte[] endRowSuffix = sp.getEndRowSuffix("testdb", "testtb", parts);
+    RowFilter filter = (RowFilter)sp.getFilter(parts);
+
+    // scan range contains the major key year, rowfilter contains minor key state
+    Assert.assertTrue(Bytes.contains(startRowSuffix, Shorts.toByteArray((short)2015)));
+    Assert.assertTrue(Bytes.contains(endRowSuffix, Shorts.toByteArray((short)2016)));
+
+    PartitionKeyComparator comparator = (PartitionKeyComparator)filter.getComparator();
+    Assert.assertEquals(comparator.ranges.size(), 1);
+    Assert.assertEquals(comparator.ranges.get(0).keyName, "state");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5e16d53e/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java
new file mode 100644
index 0000000..ec43ae3
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.serde2.binarysortable;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+public class BinarySortableSerDeWithEndPrefix extends BinarySortableSerDe {
+  public static void serializeStruct(Output byteStream, Object[] fieldData,
+      List<ObjectInspector> fieldOis, boolean endPrefix) throws SerDeException {
+    for (int i = 0; i < fieldData.length; i++) {
+      serialize(byteStream, fieldData[i], fieldOis.get(i), false);
+    }
+    if (endPrefix) {
+      if (fieldData[fieldData.length-1]!=null) {
+        byteStream.getData()[byteStream.getLength()-1]++;
+      } else {
+        byteStream.getData()[byteStream.getLength()-1]+=2;
+      }
+    }
+  }
+}
\ No newline at end of file


[09/52] [abbrv] hive git commit: HIVE-11382 Invalidate aggregate column stats on alter partition (gates)

Posted by se...@apache.org.
HIVE-11382 Invalidate aggregate column stats on alter partition (gates)


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

Branch: refs/heads/llap
Commit: 7e7f461b0ba86e40224564e0ad1e320c4f6d62b3
Parents: 9d3d4eb
Author: Alan Gates <ga...@hortonworks.com>
Authored: Thu Jul 30 10:12:35 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Thu Jul 30 10:12:35 2015 -0700

----------------------------------------------------------------------
 .../TestHBaseAggrStatsCacheIntegration.java     | 192 +++++++++++++++++++
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   7 +
 2 files changed, 199 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7e7f461b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java
index 7e6a2ef..ad76b2e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggrStatsCacheIntegration.java
@@ -496,4 +496,196 @@ public class TestHBaseAggrStatsCacheIntegration extends HBaseIntegrationTests {
       store.backdoor().getStatsCache().wakeInvalidator();
     }
   }
+
+  @Test
+  public void alterInvalidation() throws Exception {
+    try {
+      String dbName = "default";
+      String tableName = "ai";
+      List<String> partVals1 = Arrays.asList("today");
+      List<String> partVals2 = Arrays.asList("yesterday");
+      List<String> partVals3 = Arrays.asList("tomorrow");
+      long now = System.currentTimeMillis();
+
+      List<FieldSchema> cols = new ArrayList<>();
+      cols.add(new FieldSchema("col1", "boolean", "nocomment"));
+      SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+      StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
+          serde, null, null, Collections.<String, String>emptyMap());
+      List<FieldSchema> partCols = new ArrayList<>();
+      partCols.add(new FieldSchema("ds", "string", ""));
+      Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
+          Collections.<String, String>emptyMap(), null, null, null);
+      store.createTable(table);
+
+      Partition[] partitions = new Partition[3];
+      int partnum = 0;
+      for (List<String> partVals : Arrays.asList(partVals1, partVals2, partVals3)) {
+        StorageDescriptor psd = new StorageDescriptor(sd);
+        psd.setLocation("file:/tmp/default/invalidation/ds=" + partVals.get(0));
+        Partition part = new Partition(partVals, dbName, tableName, (int) now, (int) now, psd,
+            Collections.<String, String>emptyMap());
+        partitions[partnum++] = part;
+        store.addPartition(part);
+
+        ColumnStatistics cs = new ColumnStatistics();
+        ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
+        desc.setLastAnalyzed(now);
+        desc.setPartName("ds=" + partVals.get(0));
+        cs.setStatsDesc(desc);
+        ColumnStatisticsObj obj = new ColumnStatisticsObj();
+        obj.setColName("col1");
+        obj.setColType("boolean");
+        ColumnStatisticsData data = new ColumnStatisticsData();
+        BooleanColumnStatsData bcsd = new BooleanColumnStatsData();
+        bcsd.setNumFalses(10);
+        bcsd.setNumTrues(20);
+        bcsd.setNumNulls(30);
+        data.setBooleanStats(bcsd);
+        obj.setStatsData(data);
+        cs.addToStatsObj(obj);
+
+        store.updatePartitionColumnStatistics(cs, partVals);
+      }
+
+      AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName,
+          Arrays.asList("ds=today", "ds=tomorrow"), Arrays.asList("col1"));
+      aggrStats = store.get_aggr_stats_for(dbName, tableName,
+          Arrays.asList("ds=today", "ds=yesterday"), Arrays.asList("col1"));
+
+      // Check that we had to build it from the stats
+      Assert.assertEquals(0, store.backdoor().getStatsCache().hbaseHits.getCnt());
+      Assert.assertEquals(2, store.backdoor().getStatsCache().totalGets.getCnt());
+      Assert.assertEquals(2, store.backdoor().getStatsCache().misses.getCnt());
+
+      // wake the invalidator and check again to make sure it isn't too aggressive about
+      // removing our stuff.
+      store.backdoor().getStatsCache().wakeInvalidator();
+
+      Partition newPart = new Partition(partitions[2]);
+      newPart.setLastAccessTime((int)System.currentTimeMillis());
+      store.alterPartition(dbName, tableName, partVals3, newPart);
+
+      store.backdoor().getStatsCache().setRunInvalidatorEvery(100);
+      store.backdoor().getStatsCache().wakeInvalidator();
+
+      aggrStats = store.get_aggr_stats_for(dbName, tableName,
+          Arrays.asList("ds=tomorrow", "ds=today"), Arrays.asList("col1"));
+
+      // Check that we missed, which means this aggregate was dropped from the cache.
+      Assert.assertEquals(0, store.backdoor().getStatsCache().hbaseHits.getCnt());
+      Assert.assertEquals(3, store.backdoor().getStatsCache().totalGets.getCnt());
+      Assert.assertEquals(3, store.backdoor().getStatsCache().misses.getCnt());
+
+      // Check that our other aggregate is still in the cache.
+      aggrStats = store.get_aggr_stats_for(dbName, tableName,
+          Arrays.asList("ds=yesterday", "ds=today"), Arrays.asList("col1"));
+
+      Assert.assertEquals(0, store.backdoor().getStatsCache().hbaseHits.getCnt());
+      Assert.assertEquals(4, store.backdoor().getStatsCache().totalGets.getCnt());
+      Assert.assertEquals(3, store.backdoor().getStatsCache().misses.getCnt());
+    } finally {
+      store.backdoor().getStatsCache().setRunInvalidatorEvery(5000);
+      store.backdoor().getStatsCache().setMaxTimeInCache(500000);
+      store.backdoor().getStatsCache().wakeInvalidator();
+    }
+  }
+
+  @Test
+  public void altersInvalidation() throws Exception {
+    try {
+      String dbName = "default";
+      String tableName = "asi";
+      List<String> partVals1 = Arrays.asList("today");
+      List<String> partVals2 = Arrays.asList("yesterday");
+      List<String> partVals3 = Arrays.asList("tomorrow");
+      long now = System.currentTimeMillis();
+
+      List<FieldSchema> cols = new ArrayList<>();
+      cols.add(new FieldSchema("col1", "boolean", "nocomment"));
+      SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+      StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
+          serde, null, null, Collections.<String, String>emptyMap());
+      List<FieldSchema> partCols = new ArrayList<>();
+      partCols.add(new FieldSchema("ds", "string", ""));
+      Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
+          Collections.<String, String>emptyMap(), null, null, null);
+      store.createTable(table);
+
+      Partition[] partitions = new Partition[3];
+      int partnum = 0;
+      for (List<String> partVals : Arrays.asList(partVals1, partVals2, partVals3)) {
+        StorageDescriptor psd = new StorageDescriptor(sd);
+        psd.setLocation("file:/tmp/default/invalidation/ds=" + partVals.get(0));
+        Partition part = new Partition(partVals, dbName, tableName, (int) now, (int) now, psd,
+            Collections.<String, String>emptyMap());
+        partitions[partnum++] = part;
+        store.addPartition(part);
+
+        ColumnStatistics cs = new ColumnStatistics();
+        ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
+        desc.setLastAnalyzed(now);
+        desc.setPartName("ds=" + partVals.get(0));
+        cs.setStatsDesc(desc);
+        ColumnStatisticsObj obj = new ColumnStatisticsObj();
+        obj.setColName("col1");
+        obj.setColType("boolean");
+        ColumnStatisticsData data = new ColumnStatisticsData();
+        BooleanColumnStatsData bcsd = new BooleanColumnStatsData();
+        bcsd.setNumFalses(10);
+        bcsd.setNumTrues(20);
+        bcsd.setNumNulls(30);
+        data.setBooleanStats(bcsd);
+        obj.setStatsData(data);
+        cs.addToStatsObj(obj);
+
+        store.updatePartitionColumnStatistics(cs, partVals);
+      }
+
+      AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName,
+          Arrays.asList("ds=today", "ds=tomorrow"), Arrays.asList("col1"));
+      aggrStats = store.get_aggr_stats_for(dbName, tableName,
+          Arrays.asList("ds=today", "ds=yesterday"), Arrays.asList("col1"));
+
+      // Check that we had to build it from the stats
+      Assert.assertEquals(0, store.backdoor().getStatsCache().hbaseHits.getCnt());
+      Assert.assertEquals(2, store.backdoor().getStatsCache().totalGets.getCnt());
+      Assert.assertEquals(2, store.backdoor().getStatsCache().misses.getCnt());
+
+      // wake the invalidator and check again to make sure it isn't too aggressive about
+      // removing our stuff.
+      store.backdoor().getStatsCache().wakeInvalidator();
+
+      Partition[] newParts = new Partition[2];
+      newParts[0] = new Partition(partitions[0]);
+      newParts[0].setLastAccessTime((int)System.currentTimeMillis());
+      newParts[1] = new Partition(partitions[2]);
+      newParts[1].setLastAccessTime((int) System.currentTimeMillis());
+      store.alterPartitions(dbName, tableName, Arrays.asList(partVals1, partVals3),
+          Arrays.asList(newParts));
+
+      store.backdoor().getStatsCache().setRunInvalidatorEvery(100);
+      store.backdoor().getStatsCache().wakeInvalidator();
+
+      aggrStats = store.get_aggr_stats_for(dbName, tableName,
+          Arrays.asList("ds=tomorrow", "ds=today"), Arrays.asList("col1"));
+
+      // Check that we missed, which means this aggregate was dropped from the cache.
+      Assert.assertEquals(0, store.backdoor().getStatsCache().hbaseHits.getCnt());
+      Assert.assertEquals(3, store.backdoor().getStatsCache().totalGets.getCnt());
+      Assert.assertEquals(3, store.backdoor().getStatsCache().misses.getCnt());
+
+      // Check that our other aggregate got dropped too
+      aggrStats = store.get_aggr_stats_for(dbName, tableName,
+          Arrays.asList("ds=yesterday", "ds=today"), Arrays.asList("col1"));
+
+      Assert.assertEquals(0, store.backdoor().getStatsCache().hbaseHits.getCnt());
+      Assert.assertEquals(4, store.backdoor().getStatsCache().totalGets.getCnt());
+      Assert.assertEquals(4, store.backdoor().getStatsCache().misses.getCnt());
+    } finally {
+      store.backdoor().getStatsCache().setRunInvalidatorEvery(5000);
+      store.backdoor().getStatsCache().setMaxTimeInCache(500000);
+      store.backdoor().getStatsCache().wakeInvalidator();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7e7f461b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index 744070d..f8042fc 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -522,6 +522,9 @@ public class HBaseStore implements RawStore {
     try {
       Partition oldPart = getHBase().getPartition(db_name, tbl_name, part_vals);
       getHBase().replacePartition(oldPart, new_part);
+      // Drop any cached stats that reference this partitions
+      getHBase().getStatsCache().invalidate(db_name, tbl_name,
+          buildExternalPartName(db_name, tbl_name, part_vals));
       commit = true;
     } catch (IOException e) {
       LOG.error("Unable to add partition", e);
@@ -540,6 +543,10 @@ public class HBaseStore implements RawStore {
     try {
       List<Partition> oldParts = getHBase().getPartitions(db_name, tbl_name, part_vals_list);
       getHBase().replacePartitions(oldParts, new_parts);
+      for (List<String> part_vals : part_vals_list) {
+        getHBase().getStatsCache().invalidate(db_name, tbl_name,
+            buildExternalPartName(db_name, tbl_name, part_vals));
+      }
       commit = true;
     } catch (IOException e) {
       LOG.error("Unable to add partition", e);


[43/52] [abbrv] hive git commit: HIVE-11731: Exclude hbase-metastore in itests for hadoop-1

Posted by se...@apache.org.
HIVE-11731: Exclude hbase-metastore in itests for hadoop-1


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

Branch: refs/heads/llap
Commit: 848b9770acb5cf47dd1f7307c88a1f80490fcf76
Parents: 8b0eded
Author: Daniel Dai <da...@hortonworks.com>
Authored: Thu Sep 3 21:56:59 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Thu Sep 3 21:56:59 2015 -0700

----------------------------------------------------------------------
 itests/hive-unit/pom.xml                        | 14 +++++++++++
 .../metastore/hbase/HBaseIntegrationTests.java  | 13 +---------
 itests/qtest/pom.xml                            |  7 ++++++
 itests/util/pom.xml                             | 20 ++++++++++++++++
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 25 ++++++--------------
 5 files changed, 49 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/848b9770/itests/hive-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml
index 26b5751..5295840 100644
--- a/itests/hive-unit/pom.xml
+++ b/itests/hive-unit/pom.xml
@@ -173,6 +173,20 @@
     </profile>
     <profile>
       <id>hadoop-1</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <version>2.3.2</version>
+            <configuration>
+              <testExcludes>
+                <exclude>**/metastore/hbase/**</exclude>
+              </testExcludes>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/848b9770/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
index c369058..02e481a 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/HBaseIntegrationTests.java
@@ -64,18 +64,7 @@ public class HBaseIntegrationTests {
     utility.startMiniCluster();
     conf = new HiveConf(utility.getConfiguration(), HBaseIntegrationTests.class);
     admin = utility.getHBaseAdmin();
-    for (String tableName : HBaseReadWrite.tableNames) {
-      List<byte[]> families = HBaseReadWrite.columnFamilies.get(tableName);
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
-      for (byte[] family : families) {
-        HColumnDescriptor columnDesc = new HColumnDescriptor(family);
-        if (testingTephra) columnDesc.setMaxVersions(Integer.MAX_VALUE);
-        desc.addFamily(columnDesc);
-      }
-      if (testingTephra) desc.addCoprocessor(TransactionProcessor.class.getName());
-      admin.createTable(desc);
-    }
-    admin.close();
+    HBaseStoreTestUtil.initHBaseMetastore(admin, null);
   }
 
   protected static void shutdownMiniCluster() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/848b9770/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index 122e3f6..739d06a 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -80,6 +80,13 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-it-unit</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-it-util</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hive/blob/848b9770/itests/util/pom.xml
----------------------------------------------------------------------
diff --git a/itests/util/pom.xml b/itests/util/pom.xml
index b0818d6..fdab72c 100644
--- a/itests/util/pom.xml
+++ b/itests/util/pom.xml
@@ -97,6 +97,20 @@
   <profiles>
     <profile>
       <id>hadoop-1</id>
+        <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <version>2.3.2</version>
+           <configuration>
+              <excludes>
+                <exclude>**/metastore/hbase/**</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+        </plugins>
+        </build>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -120,6 +134,12 @@
         </dependency>
         <dependency>
           <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <classifier>tests</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
           <artifactId>hbase-server</artifactId>
           <version>${hbase.hadoop1.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/hive/blob/848b9770/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 70df41e..24a85cf 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql;
 
-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT;
 import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
 
 import java.io.BufferedInputStream;
@@ -67,9 +66,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hive.cli.CliDriver;
@@ -81,9 +77,7 @@ import org.apache.hadoop.hive.common.io.SortPrintStream;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -357,19 +351,14 @@ public class QTestUtil {
     utility = new HBaseTestingUtility();
     utility.startMiniCluster();
     conf = new HiveConf(utility.getConfiguration(), Driver.class);
-    conf = new HiveConf(utility.getConfiguration(), Driver.class);
     HBaseAdmin admin = utility.getHBaseAdmin();
-    for (String tableName : HBaseReadWrite.tableNames) {
-      List<byte[]> families = HBaseReadWrite.columnFamilies.get(tableName);
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
-      for (byte[] family : families) {
-        HColumnDescriptor columnDesc = new HColumnDescriptor(family);
-        desc.addFamily(columnDesc);
-      }
-      admin.createTable(desc);
-    }
-    admin.close();
-    HBaseReadWrite.getInstance(conf);
+    // Need to use reflection here to make compilation pass since HBaseIntegrationTests
+    // is not compiled in hadoop-1. All HBaseMetastore tests run under hadoop-2, so this
+    // guarantee HBaseIntegrationTests exist when we hitting this code path
+    java.lang.reflect.Method initHBaseMetastoreMethod = Class.forName(
+        "org.apache.hadoop.hive.metastore.hbase.HBaseStoreTestUtil")
+        .getMethod("initHBaseMetastore", HBaseAdmin.class, HiveConf.class);
+    initHBaseMetastoreMethod.invoke(null, admin, conf);
   }
 
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,


[46/52] [abbrv] hive git commit: HIVE-11875: JDBC Driver does not honor delegation token mechanism when readings params from ZooKeeper (Vaibhav Gumashta reviewed by Jason Dere)

Posted by se...@apache.org.
HIVE-11875: JDBC Driver does not honor delegation token mechanism when readings params from ZooKeeper (Vaibhav Gumashta reviewed by Jason Dere)


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

Branch: refs/heads/llap
Commit: 514ab795ffd03a72803f878eac57e3cf82b80045
Parents: 2a65989
Author: Vaibhav Gumashta <vg...@apache.org>
Authored: Mon Sep 21 17:00:24 2015 -0700
Committer: Vaibhav Gumashta <vg...@apache.org>
Committed: Mon Sep 21 17:00:24 2015 -0700

----------------------------------------------------------------------
 .../hive/jdbc/ZooKeeperHiveClientHelper.java    | 32 ++++++++++++++------
 1 file changed, 22 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/514ab795/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java b/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java
index eeb3cf9..4712d2e 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java
@@ -137,20 +137,32 @@ class ZooKeeperHiveClientHelper {
             && !(connParams.getSessionVars().containsKey(JdbcConnectionParams.USE_SSL))) {
           connParams.getSessionVars().put(JdbcConnectionParams.USE_SSL, matcher.group(2));
         }
-        // Set authentication configs
-        // Note that in JDBC driver, we have 3 auth modes: NOSASL, Kerberos and password based
-        // The use of "JdbcConnectionParams.AUTH_TYPE=JdbcConnectionParams.AUTH_SIMPLE" picks NOSASL
-        // The presence of "JdbcConnectionParams.AUTH_PRINCIPAL=<principal>" picks Kerberos
-        // Otherwise password based (which includes NONE, PAM, LDAP, CUSTOM)
-        if ((matcher.group(1).equals("hive.server2.authentication"))
-            && !(connParams.getSessionVars().containsKey(JdbcConnectionParams.AUTH_TYPE))) {
-          if (matcher.group(2).equalsIgnoreCase("NOSASL")) {
+        /**
+         * Note: this is pretty messy, but sticking to the current implementation.
+         * Set authentication configs. Note that in JDBC driver, we have 3 auth modes: NOSASL,
+         * Kerberos (including delegation token mechanism) and password based.
+         * The use of JdbcConnectionParams.AUTH_TYPE==JdbcConnectionParams.AUTH_SIMPLE picks NOSASL.
+         * The presence of JdbcConnectionParams.AUTH_PRINCIPAL==<principal> picks Kerberos.
+         * If principal is absent, the presence of
+         * JdbcConnectionParams.AUTH_TYPE==JdbcConnectionParams.AUTH_TOKEN uses delegation token.
+         * Otherwise password based (which includes NONE, PAM, LDAP, CUSTOM)
+         */
+        if (matcher.group(1).equals("hive.server2.authentication")) {
+          // NOSASL
+          if (matcher.group(2).equalsIgnoreCase("NOSASL")
+              && !(connParams.getSessionVars().containsKey(JdbcConnectionParams.AUTH_TYPE) && connParams
+                  .getSessionVars().get(JdbcConnectionParams.AUTH_TYPE)
+                  .equalsIgnoreCase(JdbcConnectionParams.AUTH_SIMPLE))) {
             connParams.getSessionVars().put(JdbcConnectionParams.AUTH_TYPE,
                 JdbcConnectionParams.AUTH_SIMPLE);
           }
         }
-        // Set server's kerberos principal
-        if ((matcher.group(1).equals("hive.server2.authentication.kerberos.principal"))
+        // KERBEROS
+        // If delegation token is passed from the client side, do not set the principal
+        if (matcher.group(2).equalsIgnoreCase("hive.server2.authentication.kerberos.principal")
+            && !(connParams.getSessionVars().containsKey(JdbcConnectionParams.AUTH_TYPE) && connParams
+                .getSessionVars().get(JdbcConnectionParams.AUTH_TYPE)
+                .equalsIgnoreCase(JdbcConnectionParams.AUTH_TOKEN))
             && !(connParams.getSessionVars().containsKey(JdbcConnectionParams.AUTH_PRINCIPAL))) {
           connParams.getSessionVars().put(JdbcConnectionParams.AUTH_PRINCIPAL, matcher.group(2));
         }


[25/52] [abbrv] hive git commit: HIVE-11552 : implement basic methods for getting/putting file metadata (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/129bed52/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index dd75b01..f89320f 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1024,6 +1024,34 @@ class Iface(fb303.FacebookService.Iface):
   def flushCache(self):
     pass
 
+  def get_file_metadata_by_expr(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
+  def get_file_metadata(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
+  def put_file_metadata(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
+  def clear_file_metadata(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
 
 class Client(fb303.FacebookService.Client, Iface):
   """
@@ -5616,6 +5644,130 @@ class Client(fb303.FacebookService.Client, Iface):
     iprot.readMessageEnd()
     return
 
+  def get_file_metadata_by_expr(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_get_file_metadata_by_expr(req)
+    return self.recv_get_file_metadata_by_expr()
+
+  def send_get_file_metadata_by_expr(self, req):
+    self._oprot.writeMessageBegin('get_file_metadata_by_expr', TMessageType.CALL, self._seqid)
+    args = get_file_metadata_by_expr_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_file_metadata_by_expr(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_file_metadata_by_expr_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_file_metadata_by_expr failed: unknown result");
+
+  def get_file_metadata(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_get_file_metadata(req)
+    return self.recv_get_file_metadata()
+
+  def send_get_file_metadata(self, req):
+    self._oprot.writeMessageBegin('get_file_metadata', TMessageType.CALL, self._seqid)
+    args = get_file_metadata_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_file_metadata(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_file_metadata_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_file_metadata failed: unknown result");
+
+  def put_file_metadata(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_put_file_metadata(req)
+    return self.recv_put_file_metadata()
+
+  def send_put_file_metadata(self, req):
+    self._oprot.writeMessageBegin('put_file_metadata', TMessageType.CALL, self._seqid)
+    args = put_file_metadata_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_put_file_metadata(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = put_file_metadata_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "put_file_metadata failed: unknown result");
+
+  def clear_file_metadata(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_clear_file_metadata(req)
+    return self.recv_clear_file_metadata()
+
+  def send_clear_file_metadata(self, req):
+    self._oprot.writeMessageBegin('clear_file_metadata', TMessageType.CALL, self._seqid)
+    args = clear_file_metadata_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_clear_file_metadata(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = clear_file_metadata_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "clear_file_metadata failed: unknown result");
+
 
 class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
   def __init__(self, handler):
@@ -5745,6 +5897,10 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_current_notificationEventId"] = Processor.process_get_current_notificationEventId
     self._processMap["fire_listener_event"] = Processor.process_fire_listener_event
     self._processMap["flushCache"] = Processor.process_flushCache
+    self._processMap["get_file_metadata_by_expr"] = Processor.process_get_file_metadata_by_expr
+    self._processMap["get_file_metadata"] = Processor.process_get_file_metadata
+    self._processMap["put_file_metadata"] = Processor.process_put_file_metadata
+    self._processMap["clear_file_metadata"] = Processor.process_clear_file_metadata
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -7740,6 +7896,50 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_file_metadata_by_expr(self, seqid, iprot, oprot):
+    args = get_file_metadata_by_expr_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_file_metadata_by_expr_result()
+    result.success = self._handler.get_file_metadata_by_expr(args.req)
+    oprot.writeMessageBegin("get_file_metadata_by_expr", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_file_metadata(self, seqid, iprot, oprot):
+    args = get_file_metadata_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_file_metadata_result()
+    result.success = self._handler.get_file_metadata(args.req)
+    oprot.writeMessageBegin("get_file_metadata", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_put_file_metadata(self, seqid, iprot, oprot):
+    args = put_file_metadata_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = put_file_metadata_result()
+    result.success = self._handler.put_file_metadata(args.req)
+    oprot.writeMessageBegin("put_file_metadata", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_clear_file_metadata(self, seqid, iprot, oprot):
+    args = clear_file_metadata_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = clear_file_metadata_result()
+    result.success = self._handler.clear_file_metadata(args.req)
+    oprot.writeMessageBegin("clear_file_metadata", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
 
 # HELPER FUNCTIONS AND STRUCTURES
 
@@ -8626,10 +8826,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype472, _size469) = iprot.readListBegin()
-          for _i473 in xrange(_size469):
-            _elem474 = iprot.readString();
-            self.success.append(_elem474)
+          (_etype532, _size529) = iprot.readListBegin()
+          for _i533 in xrange(_size529):
+            _elem534 = iprot.readString();
+            self.success.append(_elem534)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8652,8 +8852,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter475 in self.success:
-        oprot.writeString(iter475)
+      for iter535 in self.success:
+        oprot.writeString(iter535)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -8758,10 +8958,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype479, _size476) = iprot.readListBegin()
-          for _i480 in xrange(_size476):
-            _elem481 = iprot.readString();
-            self.success.append(_elem481)
+          (_etype539, _size536) = iprot.readListBegin()
+          for _i540 in xrange(_size536):
+            _elem541 = iprot.readString();
+            self.success.append(_elem541)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8784,8 +8984,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter482 in self.success:
-        oprot.writeString(iter482)
+      for iter542 in self.success:
+        oprot.writeString(iter542)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -9555,12 +9755,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype484, _vtype485, _size483 ) = iprot.readMapBegin()
-          for _i487 in xrange(_size483):
-            _key488 = iprot.readString();
-            _val489 = Type()
-            _val489.read(iprot)
-            self.success[_key488] = _val489
+          (_ktype544, _vtype545, _size543 ) = iprot.readMapBegin()
+          for _i547 in xrange(_size543):
+            _key548 = iprot.readString();
+            _val549 = Type()
+            _val549.read(iprot)
+            self.success[_key548] = _val549
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -9583,9 +9783,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter490,viter491 in self.success.items():
-        oprot.writeString(kiter490)
-        viter491.write(oprot)
+      for kiter550,viter551 in self.success.items():
+        oprot.writeString(kiter550)
+        viter551.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -9728,11 +9928,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype495, _size492) = iprot.readListBegin()
-          for _i496 in xrange(_size492):
-            _elem497 = FieldSchema()
-            _elem497.read(iprot)
-            self.success.append(_elem497)
+          (_etype555, _size552) = iprot.readListBegin()
+          for _i556 in xrange(_size552):
+            _elem557 = FieldSchema()
+            _elem557.read(iprot)
+            self.success.append(_elem557)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9767,8 +9967,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter498 in self.success:
-        iter498.write(oprot)
+      for iter558 in self.success:
+        iter558.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -9935,11 +10135,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype502, _size499) = iprot.readListBegin()
-          for _i503 in xrange(_size499):
-            _elem504 = FieldSchema()
-            _elem504.read(iprot)
-            self.success.append(_elem504)
+          (_etype562, _size559) = iprot.readListBegin()
+          for _i563 in xrange(_size559):
+            _elem564 = FieldSchema()
+            _elem564.read(iprot)
+            self.success.append(_elem564)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9974,8 +10174,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter505 in self.success:
-        iter505.write(oprot)
+      for iter565 in self.success:
+        iter565.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -10128,11 +10328,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype509, _size506) = iprot.readListBegin()
-          for _i510 in xrange(_size506):
-            _elem511 = FieldSchema()
-            _elem511.read(iprot)
-            self.success.append(_elem511)
+          (_etype569, _size566) = iprot.readListBegin()
+          for _i570 in xrange(_size566):
+            _elem571 = FieldSchema()
+            _elem571.read(iprot)
+            self.success.append(_elem571)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10167,8 +10367,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter512 in self.success:
-        iter512.write(oprot)
+      for iter572 in self.success:
+        iter572.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -10335,11 +10535,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype516, _size513) = iprot.readListBegin()
-          for _i517 in xrange(_size513):
-            _elem518 = FieldSchema()
-            _elem518.read(iprot)
-            self.success.append(_elem518)
+          (_etype576, _size573) = iprot.readListBegin()
+          for _i577 in xrange(_size573):
+            _elem578 = FieldSchema()
+            _elem578.read(iprot)
+            self.success.append(_elem578)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10374,8 +10574,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter519 in self.success:
-        iter519.write(oprot)
+      for iter579 in self.success:
+        iter579.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11240,10 +11440,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype523, _size520) = iprot.readListBegin()
-          for _i524 in xrange(_size520):
-            _elem525 = iprot.readString();
-            self.success.append(_elem525)
+          (_etype583, _size580) = iprot.readListBegin()
+          for _i584 in xrange(_size580):
+            _elem585 = iprot.readString();
+            self.success.append(_elem585)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11266,8 +11466,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter526 in self.success:
-        oprot.writeString(iter526)
+      for iter586 in self.success:
+        oprot.writeString(iter586)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11391,10 +11591,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype530, _size527) = iprot.readListBegin()
-          for _i531 in xrange(_size527):
-            _elem532 = iprot.readString();
-            self.success.append(_elem532)
+          (_etype590, _size587) = iprot.readListBegin()
+          for _i591 in xrange(_size587):
+            _elem592 = iprot.readString();
+            self.success.append(_elem592)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11417,8 +11617,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter533 in self.success:
-        oprot.writeString(iter533)
+      for iter593 in self.success:
+        oprot.writeString(iter593)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11654,10 +11854,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype537, _size534) = iprot.readListBegin()
-          for _i538 in xrange(_size534):
-            _elem539 = iprot.readString();
-            self.tbl_names.append(_elem539)
+          (_etype597, _size594) = iprot.readListBegin()
+          for _i598 in xrange(_size594):
+            _elem599 = iprot.readString();
+            self.tbl_names.append(_elem599)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11678,8 +11878,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter540 in self.tbl_names:
-        oprot.writeString(iter540)
+      for iter600 in self.tbl_names:
+        oprot.writeString(iter600)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11740,11 +11940,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype544, _size541) = iprot.readListBegin()
-          for _i545 in xrange(_size541):
-            _elem546 = Table()
-            _elem546.read(iprot)
-            self.success.append(_elem546)
+          (_etype604, _size601) = iprot.readListBegin()
+          for _i605 in xrange(_size601):
+            _elem606 = Table()
+            _elem606.read(iprot)
+            self.success.append(_elem606)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11779,8 +11979,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter547 in self.success:
-        iter547.write(oprot)
+      for iter607 in self.success:
+        iter607.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11946,10 +12146,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype551, _size548) = iprot.readListBegin()
-          for _i552 in xrange(_size548):
-            _elem553 = iprot.readString();
-            self.success.append(_elem553)
+          (_etype611, _size608) = iprot.readListBegin()
+          for _i612 in xrange(_size608):
+            _elem613 = iprot.readString();
+            self.success.append(_elem613)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11984,8 +12184,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter554 in self.success:
-        oprot.writeString(iter554)
+      for iter614 in self.success:
+        oprot.writeString(iter614)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12955,11 +13155,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype558, _size555) = iprot.readListBegin()
-          for _i559 in xrange(_size555):
-            _elem560 = Partition()
-            _elem560.read(iprot)
-            self.new_parts.append(_elem560)
+          (_etype618, _size615) = iprot.readListBegin()
+          for _i619 in xrange(_size615):
+            _elem620 = Partition()
+            _elem620.read(iprot)
+            self.new_parts.append(_elem620)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12976,8 +13176,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter561 in self.new_parts:
-        iter561.write(oprot)
+      for iter621 in self.new_parts:
+        iter621.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13135,11 +13335,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype565, _size562) = iprot.readListBegin()
-          for _i566 in xrange(_size562):
-            _elem567 = PartitionSpec()
-            _elem567.read(iprot)
-            self.new_parts.append(_elem567)
+          (_etype625, _size622) = iprot.readListBegin()
+          for _i626 in xrange(_size622):
+            _elem627 = PartitionSpec()
+            _elem627.read(iprot)
+            self.new_parts.append(_elem627)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13156,8 +13356,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter568 in self.new_parts:
-        iter568.write(oprot)
+      for iter628 in self.new_parts:
+        iter628.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13331,10 +13531,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype572, _size569) = iprot.readListBegin()
-          for _i573 in xrange(_size569):
-            _elem574 = iprot.readString();
-            self.part_vals.append(_elem574)
+          (_etype632, _size629) = iprot.readListBegin()
+          for _i633 in xrange(_size629):
+            _elem634 = iprot.readString();
+            self.part_vals.append(_elem634)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13359,8 +13559,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter575 in self.part_vals:
-        oprot.writeString(iter575)
+      for iter635 in self.part_vals:
+        oprot.writeString(iter635)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13713,10 +13913,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype579, _size576) = iprot.readListBegin()
-          for _i580 in xrange(_size576):
-            _elem581 = iprot.readString();
-            self.part_vals.append(_elem581)
+          (_etype639, _size636) = iprot.readListBegin()
+          for _i640 in xrange(_size636):
+            _elem641 = iprot.readString();
+            self.part_vals.append(_elem641)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13747,8 +13947,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter582 in self.part_vals:
-        oprot.writeString(iter582)
+      for iter642 in self.part_vals:
+        oprot.writeString(iter642)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -14343,10 +14543,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype586, _size583) = iprot.readListBegin()
-          for _i587 in xrange(_size583):
-            _elem588 = iprot.readString();
-            self.part_vals.append(_elem588)
+          (_etype646, _size643) = iprot.readListBegin()
+          for _i647 in xrange(_size643):
+            _elem648 = iprot.readString();
+            self.part_vals.append(_elem648)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14376,8 +14576,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter589 in self.part_vals:
-        oprot.writeString(iter589)
+      for iter649 in self.part_vals:
+        oprot.writeString(iter649)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -14550,10 +14750,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype593, _size590) = iprot.readListBegin()
-          for _i594 in xrange(_size590):
-            _elem595 = iprot.readString();
-            self.part_vals.append(_elem595)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = iprot.readString();
+            self.part_vals.append(_elem655)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14589,8 +14789,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter596 in self.part_vals:
-        oprot.writeString(iter596)
+      for iter656 in self.part_vals:
+        oprot.writeString(iter656)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -15327,10 +15527,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype600, _size597) = iprot.readListBegin()
-          for _i601 in xrange(_size597):
-            _elem602 = iprot.readString();
-            self.part_vals.append(_elem602)
+          (_etype660, _size657) = iprot.readListBegin()
+          for _i661 in xrange(_size657):
+            _elem662 = iprot.readString();
+            self.part_vals.append(_elem662)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15355,8 +15555,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter603 in self.part_vals:
-        oprot.writeString(iter603)
+      for iter663 in self.part_vals:
+        oprot.writeString(iter663)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15515,11 +15715,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype605, _vtype606, _size604 ) = iprot.readMapBegin()
-          for _i608 in xrange(_size604):
-            _key609 = iprot.readString();
-            _val610 = iprot.readString();
-            self.partitionSpecs[_key609] = _val610
+          (_ktype665, _vtype666, _size664 ) = iprot.readMapBegin()
+          for _i668 in xrange(_size664):
+            _key669 = iprot.readString();
+            _val670 = iprot.readString();
+            self.partitionSpecs[_key669] = _val670
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -15556,9 +15756,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter611,viter612 in self.partitionSpecs.items():
-        oprot.writeString(kiter611)
-        oprot.writeString(viter612)
+      for kiter671,viter672 in self.partitionSpecs.items():
+        oprot.writeString(kiter671)
+        oprot.writeString(viter672)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -15773,10 +15973,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype616, _size613) = iprot.readListBegin()
-          for _i617 in xrange(_size613):
-            _elem618 = iprot.readString();
-            self.part_vals.append(_elem618)
+          (_etype676, _size673) = iprot.readListBegin()
+          for _i677 in xrange(_size673):
+            _elem678 = iprot.readString();
+            self.part_vals.append(_elem678)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15788,10 +15988,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype622, _size619) = iprot.readListBegin()
-          for _i623 in xrange(_size619):
-            _elem624 = iprot.readString();
-            self.group_names.append(_elem624)
+          (_etype682, _size679) = iprot.readListBegin()
+          for _i683 in xrange(_size679):
+            _elem684 = iprot.readString();
+            self.group_names.append(_elem684)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15816,8 +16016,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter625 in self.part_vals:
-        oprot.writeString(iter625)
+      for iter685 in self.part_vals:
+        oprot.writeString(iter685)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -15827,8 +16027,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter626 in self.group_names:
-        oprot.writeString(iter626)
+      for iter686 in self.group_names:
+        oprot.writeString(iter686)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16257,11 +16457,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype630, _size627) = iprot.readListBegin()
-          for _i631 in xrange(_size627):
-            _elem632 = Partition()
-            _elem632.read(iprot)
-            self.success.append(_elem632)
+          (_etype690, _size687) = iprot.readListBegin()
+          for _i691 in xrange(_size687):
+            _elem692 = Partition()
+            _elem692.read(iprot)
+            self.success.append(_elem692)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16290,8 +16490,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter633 in self.success:
-        iter633.write(oprot)
+      for iter693 in self.success:
+        iter693.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16385,10 +16585,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype637, _size634) = iprot.readListBegin()
-          for _i638 in xrange(_size634):
-            _elem639 = iprot.readString();
-            self.group_names.append(_elem639)
+          (_etype697, _size694) = iprot.readListBegin()
+          for _i698 in xrange(_size694):
+            _elem699 = iprot.readString();
+            self.group_names.append(_elem699)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16421,8 +16621,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter640 in self.group_names:
-        oprot.writeString(iter640)
+      for iter700 in self.group_names:
+        oprot.writeString(iter700)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16483,11 +16683,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype644, _size641) = iprot.readListBegin()
-          for _i645 in xrange(_size641):
-            _elem646 = Partition()
-            _elem646.read(iprot)
-            self.success.append(_elem646)
+          (_etype704, _size701) = iprot.readListBegin()
+          for _i705 in xrange(_size701):
+            _elem706 = Partition()
+            _elem706.read(iprot)
+            self.success.append(_elem706)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16516,8 +16716,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter647 in self.success:
-        iter647.write(oprot)
+      for iter707 in self.success:
+        iter707.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16675,11 +16875,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype651, _size648) = iprot.readListBegin()
-          for _i652 in xrange(_size648):
-            _elem653 = PartitionSpec()
-            _elem653.read(iprot)
-            self.success.append(_elem653)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = PartitionSpec()
+            _elem713.read(iprot)
+            self.success.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16708,8 +16908,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter654 in self.success:
-        iter654.write(oprot)
+      for iter714 in self.success:
+        iter714.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16864,10 +17064,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype658, _size655) = iprot.readListBegin()
-          for _i659 in xrange(_size655):
-            _elem660 = iprot.readString();
-            self.success.append(_elem660)
+          (_etype718, _size715) = iprot.readListBegin()
+          for _i719 in xrange(_size715):
+            _elem720 = iprot.readString();
+            self.success.append(_elem720)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16890,8 +17090,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter661 in self.success:
-        oprot.writeString(iter661)
+      for iter721 in self.success:
+        oprot.writeString(iter721)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -16967,10 +17167,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype665, _size662) = iprot.readListBegin()
-          for _i666 in xrange(_size662):
-            _elem667 = iprot.readString();
-            self.part_vals.append(_elem667)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = iprot.readString();
+            self.part_vals.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17000,8 +17200,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter668 in self.part_vals:
-        oprot.writeString(iter668)
+      for iter728 in self.part_vals:
+        oprot.writeString(iter728)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -17065,11 +17265,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype672, _size669) = iprot.readListBegin()
-          for _i673 in xrange(_size669):
-            _elem674 = Partition()
-            _elem674.read(iprot)
-            self.success.append(_elem674)
+          (_etype732, _size729) = iprot.readListBegin()
+          for _i733 in xrange(_size729):
+            _elem734 = Partition()
+            _elem734.read(iprot)
+            self.success.append(_elem734)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17098,8 +17298,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter675 in self.success:
-        iter675.write(oprot)
+      for iter735 in self.success:
+        iter735.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17186,10 +17386,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype679, _size676) = iprot.readListBegin()
-          for _i680 in xrange(_size676):
-            _elem681 = iprot.readString();
-            self.part_vals.append(_elem681)
+          (_etype739, _size736) = iprot.readListBegin()
+          for _i740 in xrange(_size736):
+            _elem741 = iprot.readString();
+            self.part_vals.append(_elem741)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17206,10 +17406,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype685, _size682) = iprot.readListBegin()
-          for _i686 in xrange(_size682):
-            _elem687 = iprot.readString();
-            self.group_names.append(_elem687)
+          (_etype745, _size742) = iprot.readListBegin()
+          for _i746 in xrange(_size742):
+            _elem747 = iprot.readString();
+            self.group_names.append(_elem747)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17234,8 +17434,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter688 in self.part_vals:
-        oprot.writeString(iter688)
+      for iter748 in self.part_vals:
+        oprot.writeString(iter748)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -17249,8 +17449,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter689 in self.group_names:
-        oprot.writeString(iter689)
+      for iter749 in self.group_names:
+        oprot.writeString(iter749)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17312,11 +17512,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype693, _size690) = iprot.readListBegin()
-          for _i694 in xrange(_size690):
-            _elem695 = Partition()
-            _elem695.read(iprot)
-            self.success.append(_elem695)
+          (_etype753, _size750) = iprot.readListBegin()
+          for _i754 in xrange(_size750):
+            _elem755 = Partition()
+            _elem755.read(iprot)
+            self.success.append(_elem755)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17345,8 +17545,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter696 in self.success:
-        iter696.write(oprot)
+      for iter756 in self.success:
+        iter756.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17427,10 +17627,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype700, _size697) = iprot.readListBegin()
-          for _i701 in xrange(_size697):
-            _elem702 = iprot.readString();
-            self.part_vals.append(_elem702)
+          (_etype760, _size757) = iprot.readListBegin()
+          for _i761 in xrange(_size757):
+            _elem762 = iprot.readString();
+            self.part_vals.append(_elem762)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17460,8 +17660,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter703 in self.part_vals:
-        oprot.writeString(iter703)
+      for iter763 in self.part_vals:
+        oprot.writeString(iter763)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -17525,10 +17725,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype707, _size704) = iprot.readListBegin()
-          for _i708 in xrange(_size704):
-            _elem709 = iprot.readString();
-            self.success.append(_elem709)
+          (_etype767, _size764) = iprot.readListBegin()
+          for _i768 in xrange(_size764):
+            _elem769 = iprot.readString();
+            self.success.append(_elem769)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17557,8 +17757,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter710 in self.success:
-        oprot.writeString(iter710)
+      for iter770 in self.success:
+        oprot.writeString(iter770)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17729,11 +17929,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype714, _size711) = iprot.readListBegin()
-          for _i715 in xrange(_size711):
-            _elem716 = Partition()
-            _elem716.read(iprot)
-            self.success.append(_elem716)
+          (_etype774, _size771) = iprot.readListBegin()
+          for _i775 in xrange(_size771):
+            _elem776 = Partition()
+            _elem776.read(iprot)
+            self.success.append(_elem776)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17762,8 +17962,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter717 in self.success:
-        iter717.write(oprot)
+      for iter777 in self.success:
+        iter777.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17934,11 +18134,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype721, _size718) = iprot.readListBegin()
-          for _i722 in xrange(_size718):
-            _elem723 = PartitionSpec()
-            _elem723.read(iprot)
-            self.success.append(_elem723)
+          (_etype781, _size778) = iprot.readListBegin()
+          for _i782 in xrange(_size778):
+            _elem783 = PartitionSpec()
+            _elem783.read(iprot)
+            self.success.append(_elem783)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17967,8 +18167,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter724 in self.success:
-        iter724.write(oprot)
+      for iter784 in self.success:
+        iter784.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18205,10 +18405,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype728, _size725) = iprot.readListBegin()
-          for _i729 in xrange(_size725):
-            _elem730 = iprot.readString();
-            self.names.append(_elem730)
+          (_etype788, _size785) = iprot.readListBegin()
+          for _i789 in xrange(_size785):
+            _elem790 = iprot.readString();
+            self.names.append(_elem790)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18233,8 +18433,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter731 in self.names:
-        oprot.writeString(iter731)
+      for iter791 in self.names:
+        oprot.writeString(iter791)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18293,11 +18493,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype735, _size732) = iprot.readListBegin()
-          for _i736 in xrange(_size732):
-            _elem737 = Partition()
-            _elem737.read(iprot)
-            self.success.append(_elem737)
+          (_etype795, _size792) = iprot.readListBegin()
+          for _i796 in xrange(_size792):
+            _elem797 = Partition()
+            _elem797.read(iprot)
+            self.success.append(_elem797)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18326,8 +18526,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter738 in self.success:
-        iter738.write(oprot)
+      for iter798 in self.success:
+        iter798.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18577,11 +18777,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype742, _size739) = iprot.readListBegin()
-          for _i743 in xrange(_size739):
-            _elem744 = Partition()
-            _elem744.read(iprot)
-            self.new_parts.append(_elem744)
+          (_etype802, _size799) = iprot.readListBegin()
+          for _i803 in xrange(_size799):
+            _elem804 = Partition()
+            _elem804.read(iprot)
+            self.new_parts.append(_elem804)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18606,8 +18806,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter745 in self.new_parts:
-        iter745.write(oprot)
+      for iter805 in self.new_parts:
+        iter805.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18946,10 +19146,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype749, _size746) = iprot.readListBegin()
-          for _i750 in xrange(_size746):
-            _elem751 = iprot.readString();
-            self.part_vals.append(_elem751)
+          (_etype809, _size806) = iprot.readListBegin()
+          for _i810 in xrange(_size806):
+            _elem811 = iprot.readString();
+            self.part_vals.append(_elem811)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18980,8 +19180,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter752 in self.part_vals:
-        oprot.writeString(iter752)
+      for iter812 in self.part_vals:
+        oprot.writeString(iter812)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -19123,10 +19323,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype756, _size753) = iprot.readListBegin()
-          for _i757 in xrange(_size753):
-            _elem758 = iprot.readString();
-            self.part_vals.append(_elem758)
+          (_etype816, _size813) = iprot.readListBegin()
+          for _i817 in xrange(_size813):
+            _elem818 = iprot.readString();
+            self.part_vals.append(_elem818)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19148,8 +19348,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter759 in self.part_vals:
-        oprot.writeString(iter759)
+      for iter819 in self.part_vals:
+        oprot.writeString(iter819)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -19507,10 +19707,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype763, _size760) = iprot.readListBegin()
-          for _i764 in xrange(_size760):
-            _elem765 = iprot.readString();
-            self.success.append(_elem765)
+          (_etype823, _size820) = iprot.readListBegin()
+          for _i824 in xrange(_size820):
+            _elem825 = iprot.readString();
+            self.success.append(_elem825)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19533,8 +19733,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter766 in self.success:
-        oprot.writeString(iter766)
+      for iter826 in self.success:
+        oprot.writeString(iter826)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19658,11 +19858,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype768, _vtype769, _size767 ) = iprot.readMapBegin()
-          for _i771 in xrange(_size767):
-            _key772 = iprot.readString();
-            _val773 = iprot.readString();
-            self.success[_key772] = _val773
+          (_ktype828, _vtype829, _size827 ) = iprot.readMapBegin()
+          for _i831 in xrange(_size827):
+            _key832 = iprot.readString();
+            _val833 = iprot.readString();
+            self.success[_key832] = _val833
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19685,9 +19885,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter774,viter775 in self.success.items():
-        oprot.writeString(kiter774)
-        oprot.writeString(viter775)
+      for kiter834,viter835 in self.success.items():
+        oprot.writeString(kiter834)
+        oprot.writeString(viter835)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19763,11 +19963,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype777, _vtype778, _size776 ) = iprot.readMapBegin()
-          for _i780 in xrange(_size776):
-            _key781 = iprot.readString();
-            _val782 = iprot.readString();
-            self.part_vals[_key781] = _val782
+          (_ktype837, _vtype838, _size836 ) = iprot.readMapBegin()
+          for _i840 in xrange(_size836):
+            _key841 = iprot.readString();
+            _val842 = iprot.readString();
+            self.part_vals[_key841] = _val842
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19797,9 +19997,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter783,viter784 in self.part_vals.items():
-        oprot.writeString(kiter783)
-        oprot.writeString(viter784)
+      for kiter843,viter844 in self.part_vals.items():
+        oprot.writeString(kiter843)
+        oprot.writeString(viter844)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -20013,11 +20213,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype786, _vtype787, _size785 ) = iprot.readMapBegin()
-          for _i789 in xrange(_size785):
-            _key790 = iprot.readString();
-            _val791 = iprot.readString();
-            self.part_vals[_key790] = _val791
+          (_ktype846, _vtype847, _size845 ) = iprot.readMapBegin()
+          for _i849 in xrange(_size845):
+            _key850 = iprot.readString();
+            _val851 = iprot.readString();
+            self.part_vals[_key850] = _val851
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -20047,9 +20247,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter792,viter793 in self.part_vals.items():
-        oprot.writeString(kiter792)
-        oprot.writeString(viter793)
+      for kiter852,viter853 in self.part_vals.items():
+        oprot.writeString(kiter852)
+        oprot.writeString(viter853)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -21104,11 +21304,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype797, _size794) = iprot.readListBegin()
-          for _i798 in xrange(_size794):
-            _elem799 = Index()
-            _elem799.read(iprot)
-            self.success.append(_elem799)
+          (_etype857, _size854) = iprot.readListBegin()
+          for _i858 in xrange(_size854):
+            _elem859 = Index()
+            _elem859.read(iprot)
+            self.success.append(_elem859)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21137,8 +21337,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter800 in self.success:
-        iter800.write(oprot)
+      for iter860 in self.success:
+        iter860.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21293,10 +21493,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype804, _size801) = iprot.readListBegin()
-          for _i805 in xrange(_size801):
-            _elem806 = iprot.readString();
-            self.success.append(_elem806)
+          (_etype864, _size861) = iprot.readListBegin()
+          for _i865 in xrange(_size861):
+            _elem866 = iprot.readString();
+            self.success.append(_elem866)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21319,8 +21519,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter807 in self.success:
-        oprot.writeString(iter807)
+      for iter867 in self.success:
+        oprot.writeString(iter867)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -23868,10 +24068,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype811, _size808) = iprot.readListBegin()
-          for _i812 in xrange(_size808):
-            _elem813 = iprot.readString();
-            self.success.append(_elem813)
+          (_etype871, _size868) = iprot.readListBegin()
+          for _i872 in xrange(_size868):
+            _elem873 = iprot.readString();
+            self.success.append(_elem873)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23894,8 +24094,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter814 in self.success:
-        oprot.writeString(iter814)
+      for iter874 in self.success:
+        oprot.writeString(iter874)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24583,10 +24783,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype818, _size815) = iprot.readListBegin()
-          for _i819 in xrange(_size815):
-            _elem820 = iprot.readString();
-            self.success.append(_elem820)
+          (_etype878, _size875) = iprot.readListBegin()
+          for _i879 in xrange(_size875):
+            _elem880 = iprot.readString();
+            self.success.append(_elem880)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24609,8 +24809,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter821 in self.success:
-        oprot.writeString(iter821)
+      for iter881 in self.success:
+        oprot.writeString(iter881)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25124,11 +25324,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype825, _size822) = iprot.readListBegin()
-          for _i826 in xrange(_size822):
-            _elem827 = Role()
-            _elem827.read(iprot)
-            self.success.append(_elem827)
+          (_etype885, _size882) = iprot.readListBegin()
+          for _i886 in xrange(_size882):
+            _elem887 = Role()
+            _elem887.read(iprot)
+            self.success.append(_elem887)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25151,8 +25351,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter828 in self.success:
-        iter828.write(oprot)
+      for iter888 in self.success:
+        iter888.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25661,10 +25861,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype832, _size829) = iprot.readListBegin()
-          for _i833 in xrange(_size829):
-            _elem834 = iprot.readString();
-            self.group_names.append(_elem834)
+          (_etype892, _size889) = iprot.readListBegin()
+          for _i893 in xrange(_size889):
+            _elem894 = iprot.readString();
+            self.group_names.append(_elem894)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25689,8 +25889,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter835 in self.group_names:
-        oprot.writeString(iter835)
+      for iter895 in self.group_names:
+        oprot.writeString(iter895)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25917,11 +26117,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype839, _size836) = iprot.readListBegin()
-          for _i840 in xrange(_size836):
-            _elem841 = HiveObjectPrivilege()
-            _elem841.read(iprot)
-            self.success.append(_elem841)
+          (_etype899, _size896) = iprot.readListBegin()
+          for _i900 in xrange(_size896):
+            _elem901 = HiveObjectPrivilege()
+            _elem901.read(iprot)
+            self.success.append(_elem901)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25944,8 +26144,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter842 in self.success:
-        iter842.write(oprot)
+      for iter902 in self.success:
+        iter902.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26443,10 +26643,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype846, _size843) = iprot.readListBegin()
-          for _i847 in xrange(_size843):
-            _elem848 = iprot.readString();
-            self.group_names.append(_elem848)
+          (_etype906, _size903) = iprot.readListBegin()
+          for _i907 in xrange(_size903):
+            _elem908 = iprot.readString();
+            self.group_names.append(_elem908)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26467,8 +26667,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter849 in self.group_names:
-        oprot.writeString(iter849)
+      for iter909 in self.group_names:
+        oprot.writeString(iter909)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26523,10 +26723,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype853, _size850) = iprot.readListBegin()
-          for _i854 in xrange(_size850):
-            _elem855 = iprot.readString();
-            self.success.append(_elem855)
+          (_etype913, _size910) = iprot.readListBegin()
+          for _i914 in xrange(_size910):
+            _elem915 = iprot.readString();
+            self.success.append(_elem915)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26549,8 +26749,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter856 in self.success:
-        oprot.writeString(iter856)
+      for iter916 in self.success:
+        oprot.writeString(iter916)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29395,3 +29595,527 @@ class flushCache_result:
 
   def __ne__(self, other):
     return not (self == other)
+
+class get_file_metadata_by_expr_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (GetFileMetadataByExprRequest, GetFileMetadataByExprRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, req=None,):
+    self.req = req
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.req = GetFileMetadataByExprRequest()
+          self.req.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_file_metadata_by_expr_args')
+    if self.req is not None:
+      oprot.writeFieldBegin('req', TType.STRUCT, 1)
+      self.req.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.req)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_file_metadata_by_expr_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (GetFileMetadataByExprResult, GetFileMetadataByExprResult.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = GetFileMetadataByExprResult()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_file_metadata_by_expr_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_file_metadata_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (GetFileMetadataRequest, GetFileMetadataRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, req=None,):
+    self.req = req
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.req = GetFileMetadataRequest()
+          self.req.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_file_metadata_args')
+    if self.req is not None:
+      oprot.writeFieldBegin('req', TType.STRUCT, 1)
+      self.req.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.req)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_file_metadata_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (GetFileMetadataResult, GetFileMetadataResult.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = GetFileMetadataResult()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_file_metadata_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class put_file_metadata_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (PutFileMetadataRequest, PutFileMetadataRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, req=None,):
+    self.req = req
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.req = PutFileMetadataRequest()
+          self.req.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('put_file_metadata_args')
+    if self.req is not None:
+      oprot.writeFieldBegin('req', TType.STRUCT, 1)
+      self.req.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.req)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class put_file_metadata_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (PutFileMetadataResult, PutFileMetadataResult.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = PutFileMetadataResult()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('put_file_metadata_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class clear_file_metadata_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (ClearFileMetadataRequest, ClearFileMetadataRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, req=None,):
+    self.req = req
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.req = ClearFileMetadataRequest()
+          self.req.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('clear_file_metadata_args')
+    if self.req is not None:
+      oprot.writeFieldBegin('req', TType.STRUCT, 1)
+      self.req.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.req)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class clear_file_metadata_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (ClearFileMetadataResult, ClearFileMetadataResult.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = ClearFileMetadataResult()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('clear_file_metadata_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)