You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2016/04/20 22:05:18 UTC

[01/30] hive git commit: HIVE-6535: JDBC: provide an async API to execute query and fetch results (Vaibhav Gumashta reviewed by Thejas Nair)

Repository: hive
Updated Branches:
  refs/heads/llap 5816ff303 -> e69bd1ee5


HIVE-6535: JDBC: provide an async API to execute query and fetch results (Vaibhav Gumashta 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/40e0c380
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/40e0c380
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/40e0c380

Branch: refs/heads/llap
Commit: 40e0c380764ba3332a00dbc77bb89e4dfa3f5aee
Parents: 2d28291
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Fri Apr 15 15:28:37 2016 -0700
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Fri Apr 15 15:28:37 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   | 85 +++++++++++++++++++-
 .../apache/hive/jdbc/HiveQueryResultSet.java    |  9 +++
 .../org/apache/hive/jdbc/HiveStatement.java     | 75 +++++++++++++----
 .../cli/operation/GetCatalogsOperation.java     |  5 +-
 .../cli/operation/GetColumnsOperation.java      |  5 +-
 .../cli/operation/GetFunctionsOperation.java    |  6 +-
 .../cli/operation/GetSchemasOperation.java      |  7 +-
 .../cli/operation/GetTableTypesOperation.java   |  7 +-
 .../cli/operation/GetTablesOperation.java       | 10 +--
 .../cli/operation/GetTypeInfoOperation.java     |  7 +-
 .../hive/service/cli/operation/Operation.java   |  8 +-
 .../service/cli/operation/SQLOperation.java     |  7 +-
 12 files changed, 193 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
index b427dc1..7028c25 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
@@ -19,7 +19,6 @@
 package org.apache.hive.jdbc;
 
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
@@ -109,6 +108,7 @@ public class TestJdbcDriver2 {
   private final HiveConf conf;
   public static String dataFileDir;
   private final Path dataFilePath;
+  private final int dataFileRowCount;
   private final Path dataTypeDataFilePath;
   private Connection con;
   private static boolean standAloneServer = false;
@@ -121,6 +121,7 @@ public class TestJdbcDriver2 {
     dataFileDir = conf.get("test.data.files").replace('\\', '/')
         .replace("c:", "");
     dataFilePath = new Path(dataFileDir, "kv1.txt");
+    dataFileRowCount = 500;
     dataTypeDataFilePath = new Path(dataFileDir, "datatypes.txt");
     standAloneServer = "true".equals(System
         .getProperty("test.service.standalone.server"));
@@ -2675,6 +2676,88 @@ public void testParseUrlHttpMode() throws SQLException, JdbcUriParseException,
     } finally {
       mycon.close();
     }
+  }
+
+  /**
+   * Test {@link HiveStatement#executeAsync(String)} for a select query
+   * @throws Exception
+   */
+  @Test
+  public void testSelectExecAsync() throws Exception {
+    HiveStatement stmt = (HiveStatement) con.createStatement();
+    ResultSet rs;
+    // Expected row count of the join query we'll run
+    int expectedCount = 1028;
+    int rowCount = 0;
+    boolean isResulSet =
+        stmt.executeAsync("select t1.value as v11, " + "t2.value as v12 from " + tableName
+            + " t1 join " + tableName + " t2 on t1.under_col = t2.under_col");
+    assertTrue(isResulSet);
+    rs = stmt.getResultSet();
+    assertNotNull(rs);
+    // ResultSet#next blocks until the async query is complete
+    while (rs.next()) {
+      String value = rs.getString(2);
+      rowCount++;
+      assertNotNull(value);
+    }
+    assertEquals(rowCount, expectedCount);
+    stmt.close();
+  }
 
+  /**
+   * Test {@link HiveStatement#executeAsync(String)} for a create table
+   * @throws Exception
+   */
+  @Test
+  public void testCreateTableExecAsync() throws Exception {
+    HiveStatement stmt = (HiveStatement) con.createStatement();
+    String tblName = "testCreateTableExecAsync";
+    boolean isResulSet = stmt.executeAsync("create table " + tblName + " (col1 int , col2 string)");
+    assertFalse(isResulSet);
+    // HiveStatement#getUpdateCount blocks until the async query is complete
+    stmt.getUpdateCount();
+    DatabaseMetaData metadata = con.getMetaData();
+    ResultSet tablesMetadata = metadata.getTables(null, null, "%", null);
+    boolean tblFound = false;
+    while (tablesMetadata.next()) {
+      String tableName = tablesMetadata.getString(3);
+      if (tableName.equalsIgnoreCase(tblName)) {
+        tblFound = true;
+      }
+    }
+    if (!tblFound) {
+      fail("Unable to create table using executeAsync");
+    }
+    stmt.execute("drop table " + tblName);
+    stmt.close();
+  }
+
+  /**
+   * Test {@link HiveStatement#executeAsync(String)} for an insert overwrite into a table
+   * @throws Exception
+   */
+  @Test
+  public void testInsertOverwriteExecAsync() throws Exception {
+    HiveStatement stmt = (HiveStatement) con.createStatement();
+    String tblName = "testInsertOverwriteExecAsync";
+    int rowCount = 0;
+    stmt.execute("create table " + tblName + " (col1 int , col2 string)");
+    boolean isResulSet =
+        stmt.executeAsync("insert overwrite table " + tblName + " select * from " + tableName);
+    assertFalse(isResulSet);
+    // HiveStatement#getUpdateCount blocks until the async query is complete
+    stmt.getUpdateCount();
+    // Read from the new table
+    ResultSet rs = stmt.executeQuery("select * from " + tblName);
+    assertNotNull(rs);
+    while (rs.next()) {
+      String value = rs.getString(2);
+      rowCount++;
+      assertNotNull(value);
+    }
+    assertEquals(rowCount, dataFileRowCount);
+    stmt.execute("drop table " + tblName);
+    stmt.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java b/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java
index 8f67209..92fdbca 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java
@@ -347,6 +347,15 @@ public class HiveQueryResultSet extends HiveBaseResultSet {
       return false;
     }
 
+    /**
+     * Poll on the operation status, till the operation is complete.
+     * We need to wait only for HiveStatement to complete.
+     * HiveDatabaseMetaData which also uses this ResultSet returns only after the RPC is complete.
+     */
+    if ((statement != null) && (statement instanceof HiveStatement)) {
+      ((HiveStatement) statement).waitForOperationToComplete();
+    }
+
     try {
       TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT;
       if (fetchFirst) {

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
index 0bbd0e3..f5b9672 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
@@ -62,6 +62,7 @@ public class HiveStatement implements java.sql.Statement {
   Map<String,String> sessConf = new HashMap<String,String>();
   private int fetchSize = DEFAULT_FETCH_SIZE;
   private boolean isScrollableResultset = false;
+  private boolean isOperationComplete = false;
   /**
    * We need to keep a reference to the result set to support the following:
    * <code>
@@ -222,7 +223,10 @@ public class HiveStatement implements java.sql.Statement {
     }
     closeClientOperation();
     client = null;
-    resultSet = null;
+    if (resultSet != null) {
+      resultSet.close();
+      resultSet = null;
+    }
     isClosed = true;
   }
 
@@ -239,6 +243,48 @@ public class HiveStatement implements java.sql.Statement {
 
   @Override
   public boolean execute(String sql) throws SQLException {
+    runAsyncOnServer(sql);
+    waitForOperationToComplete();
+
+    // The query should be completed by now
+    if (!stmtHandle.isHasResultSet()) {
+      return false;
+    }
+    resultSet =  new HiveQueryResultSet.Builder(this).setClient(client).setSessionHandle(sessHandle)
+        .setStmtHandle(stmtHandle).setMaxRows(maxRows).setFetchSize(fetchSize)
+        .setScrollable(isScrollableResultset)
+        .build();
+    return true;
+  }
+
+  /**
+   * Starts the query execution asynchronously on the server, and immediately returns to the client.
+   * The client subsequently blocks on ResultSet#next or Statement#getUpdateCount, depending on the
+   * query type. Users should call ResultSet.next or Statement#getUpdateCount (depending on whether
+   * query returns results) to ensure that query completes successfully. Calling another execute*
+   * method, or close before query completion would result in the async query getting killed if it
+   * is not already finished.
+   * Note: This method is an API for limited usage outside of Hive by applications like Apache Ambari,
+   * although it is not part of the interface java.sql.Statement.
+   *
+   * @param sql
+   * @return true if the first result is a ResultSet object; false if it is an update count or there
+   *         are no results
+   * @throws SQLException
+   */
+  public boolean executeAsync(String sql) throws SQLException {
+    runAsyncOnServer(sql);
+    if (!stmtHandle.isHasResultSet()) {
+      return false;
+    }
+    resultSet =
+        new HiveQueryResultSet.Builder(this).setClient(client).setSessionHandle(sessHandle)
+            .setStmtHandle(stmtHandle).setMaxRows(maxRows).setFetchSize(fetchSize)
+            .setScrollable(isScrollableResultset).build();
+    return true;
+  }
+
+  private void runAsyncOnServer(String sql) throws SQLException {
     checkConnection("execute");
 
     closeClientOperation();
@@ -266,13 +312,14 @@ public class HiveStatement implements java.sql.Statement {
       isExecuteStatementFailed = true;
       throw new SQLException(ex.toString(), "08S01", ex);
     }
+  }
 
+  void waitForOperationToComplete() throws SQLException {
     TGetOperationStatusReq statusReq = new TGetOperationStatusReq(stmtHandle);
-    boolean operationComplete = false;
     TGetOperationStatusResp statusResp;
 
     // Poll on the operation status, till the operation is complete
-    while (!operationComplete) {
+    while (!isOperationComplete) {
       try {
         /**
          * For an async SQLOperation, GetOperationStatus will use the long polling approach
@@ -284,7 +331,8 @@ public class HiveStatement implements java.sql.Statement {
           switch (statusResp.getOperationState()) {
           case CLOSED_STATE:
           case FINISHED_STATE:
-            operationComplete = true;
+            isOperationComplete = true;
+            isLogBeingGenerated = false;
             break;
           case CANCELED_STATE:
             // 01000 -> warning
@@ -309,17 +357,6 @@ public class HiveStatement implements java.sql.Statement {
         throw new SQLException(e.toString(), "08S01", e);
       }
     }
-    isLogBeingGenerated = false;
-
-    // The query should be completed by now
-    if (!stmtHandle.isHasResultSet()) {
-      return false;
-    }
-    resultSet =  new HiveQueryResultSet.Builder(this).setClient(client).setSessionHandle(sessHandle)
-        .setStmtHandle(stmtHandle).setMaxRows(maxRows).setFetchSize(fetchSize)
-        .setScrollable(isScrollableResultset)
-        .build();
-    return true;
   }
 
   private void checkConnection(String action) throws SQLException {
@@ -333,6 +370,7 @@ public class HiveStatement implements java.sql.Statement {
     isQueryClosed = false;
     isLogBeingGenerated = true;
     isExecuteStatementFailed = false;
+    isOperationComplete = false;
   }
 
   /*
@@ -593,10 +631,15 @@ public class HiveStatement implements java.sql.Statement {
    *
    * @see java.sql.Statement#getUpdateCount()
    */
-
   @Override
   public int getUpdateCount() throws SQLException {
     checkConnection("getUpdateCount");
+    /**
+     * Poll on the operation status, till the operation is complete. We want to ensure that since a
+     * client might end up using executeAsync and then call this to check if the query run is
+     * finished.
+     */
+    waitForOperationToComplete();
     return -1;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
index 8868ec1..2eeee47 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
@@ -18,6 +18,9 @@
 
 package org.apache.hive.service.cli.operation;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
@@ -71,7 +74,7 @@ public class GetCatalogsOperation extends MetadataOperation {
    */
   @Override
   public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     validateDefaultFetchOrientation(orientation);
     if (orientation.equals(FetchOrientation.FETCH_FIRST)) {
       rowSet.setStartOffset(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
index 35b6c52..574a757 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
@@ -20,6 +20,7 @@ package org.apache.hive.service.cli.operation;
 
 import java.sql.DatabaseMetaData;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -221,7 +222,7 @@ public class GetColumnsOperation extends MetadataOperation {
    */
   @Override
   public TableSchema getResultSetSchema() throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     return RESULT_SET_SCHEMA;
   }
 
@@ -230,7 +231,7 @@ public class GetColumnsOperation extends MetadataOperation {
    */
   @Override
   public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     validateDefaultFetchOrientation(orientation);
     if (orientation.equals(FetchOrientation.FETCH_FIRST)) {
       rowSet.setStartOffset(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
index 8db2e62..d774f4f95 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
@@ -19,6 +19,8 @@
 package org.apache.hive.service.cli.operation;
 
 import java.sql.DatabaseMetaData;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 
@@ -128,7 +130,7 @@ public class GetFunctionsOperation extends MetadataOperation {
    */
   @Override
   public TableSchema getResultSetSchema() throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     return RESULT_SET_SCHEMA;
   }
 
@@ -137,7 +139,7 @@ public class GetFunctionsOperation extends MetadataOperation {
    */
   @Override
   public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     validateDefaultFetchOrientation(orientation);
     if (orientation.equals(FetchOrientation.FETCH_FIRST)) {
       rowSet.setStartOffset(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
index d6f6280..dc0a3dd 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
@@ -18,6 +18,9 @@
 
 package org.apache.hive.service.cli.operation;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hive.service.cli.FetchOrientation;
@@ -77,7 +80,7 @@ public class GetSchemasOperation extends MetadataOperation {
    */
   @Override
   public TableSchema getResultSetSchema() throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     return RESULT_SET_SCHEMA;
   }
 
@@ -86,7 +89,7 @@ public class GetSchemasOperation extends MetadataOperation {
    */
   @Override
   public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     validateDefaultFetchOrientation(orientation);
     if (orientation.equals(FetchOrientation.FETCH_FIRST)) {
       rowSet.setStartOffset(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
index a09b39a..13d5b37 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
@@ -18,6 +18,9 @@
 
 package org.apache.hive.service.cli.operation;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
@@ -73,7 +76,7 @@ public class GetTableTypesOperation extends MetadataOperation {
    */
   @Override
   public TableSchema getResultSetSchema() throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     return RESULT_SET_SCHEMA;
   }
 
@@ -82,7 +85,7 @@ public class GetTableTypesOperation extends MetadataOperation {
    */
   @Override
   public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     validateDefaultFetchOrientation(orientation);
     if (orientation.equals(FetchOrientation.FETCH_FIRST)) {
       rowSet.setStartOffset(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
index 740b851..aac3692 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
@@ -60,9 +60,9 @@ public class GetTablesOperation extends MetadataOperation {
   .addStringColumn("TYPE_CAT", "The types catalog.")
   .addStringColumn("TYPE_SCHEM", "The types schema.")
   .addStringColumn("TYPE_NAME", "Type name.")
-  .addStringColumn("SELF_REFERENCING_COL_NAME", 
+  .addStringColumn("SELF_REFERENCING_COL_NAME",
       "Name of the designated \"identifier\" column of a typed table.")
-  .addStringColumn("REF_GENERATION", 
+  .addStringColumn("REF_GENERATION",
       "Specifies how values in SELF_REFERENCING_COL_NAME are created.");
 
   protected GetTablesOperation(HiveSession parentSession,
@@ -102,7 +102,7 @@ public class GetTablesOperation extends MetadataOperation {
 
       String tablePattern = convertIdentifierPattern(tableName, true);
 
-      for (TableMeta tableMeta : 
+      for (TableMeta tableMeta :
           metastoreClient.getTableMeta(schemaPattern, tablePattern, tableTypeList)) {
         rowSet.addRow(new Object[] {
               DEFAULT_HIVE_CATALOG,
@@ -125,7 +125,7 @@ public class GetTablesOperation extends MetadataOperation {
    */
   @Override
   public TableSchema getResultSetSchema() throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     return RESULT_SET_SCHEMA;
   }
 
@@ -134,7 +134,7 @@ public class GetTablesOperation extends MetadataOperation {
    */
   @Override
   public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     validateDefaultFetchOrientation(orientation);
     if (orientation.equals(FetchOrientation.FETCH_FIRST)) {
       rowSet.setStartOffset(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
index 2a0fec2..53660af 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
@@ -18,6 +18,9 @@
 
 package org.apache.hive.service.cli.operation;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
@@ -123,7 +126,7 @@ public class GetTypeInfoOperation extends MetadataOperation {
    */
   @Override
   public TableSchema getResultSetSchema() throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     return RESULT_SET_SCHEMA;
   }
 
@@ -132,7 +135,7 @@ public class GetTypeInfoOperation extends MetadataOperation {
    */
   @Override
   public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
     validateDefaultFetchOrientation(orientation);
     if (orientation.equals(FetchOrientation.FETCH_FIRST)) {
       rowSet.setStartOffset(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/Operation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
index b7d6549..4f3e9c2 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
@@ -22,6 +22,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Future;
@@ -199,9 +200,10 @@ public abstract class Operation {
     this.operationException = operationException;
   }
 
-  protected final void assertState(OperationState state) throws HiveSQLException {
-    if (this.state != state) {
-      throw new HiveSQLException("Expected state " + state + ", but found " + this.state);
+  protected final void assertState(List<OperationState> states) throws HiveSQLException {
+    if (!states.contains(state)) {
+      throw new HiveSQLException("Expected states: " + states.toString() + ", but found "
+          + this.state);
     }
     this.lastAccessTime = System.currentTimeMillis();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/40e0c380/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 9ce6055..ce06c1c 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -343,7 +343,10 @@ public class SQLOperation extends ExecuteStatementOperation {
 
   @Override
   public TableSchema getResultSetSchema() throws HiveSQLException {
-    assertState(OperationState.FINISHED);
+    // Since compilation is always a blocking RPC call, and schema is ready after compilation,
+    // we can return when are in the RUNNING state.
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.RUNNING,
+        OperationState.FINISHED)));
     if (resultSchema == null) {
       resultSchema = new TableSchema(driver.getSchema());
     }
@@ -355,7 +358,7 @@ public class SQLOperation extends ExecuteStatementOperation {
   @Override
   public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException {
     validateDefaultFetchOrientation(orientation);
-    assertState(OperationState.FINISHED);
+    assertState(new ArrayList<OperationState>(Arrays.asList(OperationState.FINISHED)));
 
     RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion());
 


[14/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 10eaf4a..f008788 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -394,6 +394,396 @@ class FieldSchema:
   def __ne__(self, other):
     return not (self == other)
 
+class SQLPrimaryKey:
+  """
+  Attributes:
+   - table_db
+   - table_name
+   - column_name
+   - key_seq
+   - pk_name
+   - enable_cstr
+   - validate_cstr
+   - rely_cstr
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'table_db', None, None, ), # 1
+    (2, TType.STRING, 'table_name', None, None, ), # 2
+    (3, TType.STRING, 'column_name', None, None, ), # 3
+    (4, TType.I32, 'key_seq', None, None, ), # 4
+    (5, TType.STRING, 'pk_name', None, None, ), # 5
+    (6, TType.BOOL, 'enable_cstr', None, None, ), # 6
+    (7, TType.BOOL, 'validate_cstr', None, None, ), # 7
+    (8, TType.BOOL, 'rely_cstr', None, None, ), # 8
+  )
+
+  def __init__(self, table_db=None, table_name=None, column_name=None, key_seq=None, pk_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+    self.table_db = table_db
+    self.table_name = table_name
+    self.column_name = column_name
+    self.key_seq = key_seq
+    self.pk_name = pk_name
+    self.enable_cstr = enable_cstr
+    self.validate_cstr = validate_cstr
+    self.rely_cstr = rely_cstr
+
+  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.table_db = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.table_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.column_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I32:
+          self.key_seq = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.pk_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.BOOL:
+          self.enable_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.BOOL:
+          self.validate_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.BOOL:
+          self.rely_cstr = 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('SQLPrimaryKey')
+    if self.table_db is not None:
+      oprot.writeFieldBegin('table_db', TType.STRING, 1)
+      oprot.writeString(self.table_db)
+      oprot.writeFieldEnd()
+    if self.table_name is not None:
+      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeString(self.table_name)
+      oprot.writeFieldEnd()
+    if self.column_name is not None:
+      oprot.writeFieldBegin('column_name', TType.STRING, 3)
+      oprot.writeString(self.column_name)
+      oprot.writeFieldEnd()
+    if self.key_seq is not None:
+      oprot.writeFieldBegin('key_seq', TType.I32, 4)
+      oprot.writeI32(self.key_seq)
+      oprot.writeFieldEnd()
+    if self.pk_name is not None:
+      oprot.writeFieldBegin('pk_name', TType.STRING, 5)
+      oprot.writeString(self.pk_name)
+      oprot.writeFieldEnd()
+    if self.enable_cstr is not None:
+      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 6)
+      oprot.writeBool(self.enable_cstr)
+      oprot.writeFieldEnd()
+    if self.validate_cstr is not None:
+      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 7)
+      oprot.writeBool(self.validate_cstr)
+      oprot.writeFieldEnd()
+    if self.rely_cstr is not None:
+      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 8)
+      oprot.writeBool(self.rely_cstr)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.table_db)
+    value = (value * 31) ^ hash(self.table_name)
+    value = (value * 31) ^ hash(self.column_name)
+    value = (value * 31) ^ hash(self.key_seq)
+    value = (value * 31) ^ hash(self.pk_name)
+    value = (value * 31) ^ hash(self.enable_cstr)
+    value = (value * 31) ^ hash(self.validate_cstr)
+    value = (value * 31) ^ hash(self.rely_cstr)
+    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 SQLForeignKey:
+  """
+  Attributes:
+   - pktable_db
+   - pktable_name
+   - pkcolumn_name
+   - fktable_db
+   - fktable_name
+   - fkcolumn_name
+   - key_seq
+   - update_rule
+   - delete_rule
+   - fk_name
+   - pk_name
+   - enable_cstr
+   - validate_cstr
+   - rely_cstr
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'pktable_db', None, None, ), # 1
+    (2, TType.STRING, 'pktable_name', None, None, ), # 2
+    (3, TType.STRING, 'pkcolumn_name', None, None, ), # 3
+    (4, TType.STRING, 'fktable_db', None, None, ), # 4
+    (5, TType.STRING, 'fktable_name', None, None, ), # 5
+    (6, TType.STRING, 'fkcolumn_name', None, None, ), # 6
+    (7, TType.I32, 'key_seq', None, None, ), # 7
+    (8, TType.I32, 'update_rule', None, None, ), # 8
+    (9, TType.I32, 'delete_rule', None, None, ), # 9
+    (10, TType.STRING, 'fk_name', None, None, ), # 10
+    (11, TType.STRING, 'pk_name', None, None, ), # 11
+    (12, TType.BOOL, 'enable_cstr', None, None, ), # 12
+    (13, TType.BOOL, 'validate_cstr', None, None, ), # 13
+    (14, TType.BOOL, 'rely_cstr', None, None, ), # 14
+  )
+
+  def __init__(self, pktable_db=None, pktable_name=None, pkcolumn_name=None, fktable_db=None, fktable_name=None, fkcolumn_name=None, key_seq=None, update_rule=None, delete_rule=None, fk_name=None, pk_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+    self.pktable_db = pktable_db
+    self.pktable_name = pktable_name
+    self.pkcolumn_name = pkcolumn_name
+    self.fktable_db = fktable_db
+    self.fktable_name = fktable_name
+    self.fkcolumn_name = fkcolumn_name
+    self.key_seq = key_seq
+    self.update_rule = update_rule
+    self.delete_rule = delete_rule
+    self.fk_name = fk_name
+    self.pk_name = pk_name
+    self.enable_cstr = enable_cstr
+    self.validate_cstr = validate_cstr
+    self.rely_cstr = rely_cstr
+
+  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.pktable_db = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.pktable_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.pkcolumn_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.fktable_db = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.fktable_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.fkcolumn_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.I32:
+          self.key_seq = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.I32:
+          self.update_rule = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.I32:
+          self.delete_rule = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 10:
+        if ftype == TType.STRING:
+          self.fk_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 11:
+        if ftype == TType.STRING:
+          self.pk_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 12:
+        if ftype == TType.BOOL:
+          self.enable_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 13:
+        if ftype == TType.BOOL:
+          self.validate_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 14:
+        if ftype == TType.BOOL:
+          self.rely_cstr = 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('SQLForeignKey')
+    if self.pktable_db is not None:
+      oprot.writeFieldBegin('pktable_db', TType.STRING, 1)
+      oprot.writeString(self.pktable_db)
+      oprot.writeFieldEnd()
+    if self.pktable_name is not None:
+      oprot.writeFieldBegin('pktable_name', TType.STRING, 2)
+      oprot.writeString(self.pktable_name)
+      oprot.writeFieldEnd()
+    if self.pkcolumn_name is not None:
+      oprot.writeFieldBegin('pkcolumn_name', TType.STRING, 3)
+      oprot.writeString(self.pkcolumn_name)
+      oprot.writeFieldEnd()
+    if self.fktable_db is not None:
+      oprot.writeFieldBegin('fktable_db', TType.STRING, 4)
+      oprot.writeString(self.fktable_db)
+      oprot.writeFieldEnd()
+    if self.fktable_name is not None:
+      oprot.writeFieldBegin('fktable_name', TType.STRING, 5)
+      oprot.writeString(self.fktable_name)
+      oprot.writeFieldEnd()
+    if self.fkcolumn_name is not None:
+      oprot.writeFieldBegin('fkcolumn_name', TType.STRING, 6)
+      oprot.writeString(self.fkcolumn_name)
+      oprot.writeFieldEnd()
+    if self.key_seq is not None:
+      oprot.writeFieldBegin('key_seq', TType.I32, 7)
+      oprot.writeI32(self.key_seq)
+      oprot.writeFieldEnd()
+    if self.update_rule is not None:
+      oprot.writeFieldBegin('update_rule', TType.I32, 8)
+      oprot.writeI32(self.update_rule)
+      oprot.writeFieldEnd()
+    if self.delete_rule is not None:
+      oprot.writeFieldBegin('delete_rule', TType.I32, 9)
+      oprot.writeI32(self.delete_rule)
+      oprot.writeFieldEnd()
+    if self.fk_name is not None:
+      oprot.writeFieldBegin('fk_name', TType.STRING, 10)
+      oprot.writeString(self.fk_name)
+      oprot.writeFieldEnd()
+    if self.pk_name is not None:
+      oprot.writeFieldBegin('pk_name', TType.STRING, 11)
+      oprot.writeString(self.pk_name)
+      oprot.writeFieldEnd()
+    if self.enable_cstr is not None:
+      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 12)
+      oprot.writeBool(self.enable_cstr)
+      oprot.writeFieldEnd()
+    if self.validate_cstr is not None:
+      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 13)
+      oprot.writeBool(self.validate_cstr)
+      oprot.writeFieldEnd()
+    if self.rely_cstr is not None:
+      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 14)
+      oprot.writeBool(self.rely_cstr)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.pktable_db)
+    value = (value * 31) ^ hash(self.pktable_name)
+    value = (value * 31) ^ hash(self.pkcolumn_name)
+    value = (value * 31) ^ hash(self.fktable_db)
+    value = (value * 31) ^ hash(self.fktable_name)
+    value = (value * 31) ^ hash(self.fkcolumn_name)
+    value = (value * 31) ^ hash(self.key_seq)
+    value = (value * 31) ^ hash(self.update_rule)
+    value = (value * 31) ^ hash(self.delete_rule)
+    value = (value * 31) ^ hash(self.fk_name)
+    value = (value * 31) ^ hash(self.pk_name)
+    value = (value * 31) ^ hash(self.enable_cstr)
+    value = (value * 31) ^ hash(self.validate_cstr)
+    value = (value * 31) ^ hash(self.rely_cstr)
+    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 Type:
   """
   Attributes:
@@ -5149,33 +5539,372 @@ class ColumnStatistics:
     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('ColumnStatistics')
-    if self.statsDesc is not None:
-      oprot.writeFieldBegin('statsDesc', TType.STRUCT, 1)
-      self.statsDesc.write(oprot)
-      oprot.writeFieldEnd()
-    if self.statsObj is not None:
-      oprot.writeFieldBegin('statsObj', TType.LIST, 2)
-      oprot.writeListBegin(TType.STRUCT, len(self.statsObj))
-      for iter245 in self.statsObj:
-        iter245.write(oprot)
-      oprot.writeListEnd()
+    oprot.writeStructBegin('ColumnStatistics')
+    if self.statsDesc is not None:
+      oprot.writeFieldBegin('statsDesc', TType.STRUCT, 1)
+      self.statsDesc.write(oprot)
+      oprot.writeFieldEnd()
+    if self.statsObj is not None:
+      oprot.writeFieldBegin('statsObj', TType.LIST, 2)
+      oprot.writeListBegin(TType.STRUCT, len(self.statsObj))
+      for iter245 in self.statsObj:
+        iter245.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.statsDesc is None:
+      raise TProtocol.TProtocolException(message='Required field statsDesc is unset!')
+    if self.statsObj is None:
+      raise TProtocol.TProtocolException(message='Required field statsObj is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.statsDesc)
+    value = (value * 31) ^ hash(self.statsObj)
+    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 AggrStats:
+  """
+  Attributes:
+   - colStats
+   - partsFound
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatisticsObj, ColumnStatisticsObj.thrift_spec)), None, ), # 1
+    (2, TType.I64, 'partsFound', None, None, ), # 2
+  )
+
+  def __init__(self, colStats=None, partsFound=None,):
+    self.colStats = colStats
+    self.partsFound = partsFound
+
+  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.colStats = []
+          (_etype249, _size246) = iprot.readListBegin()
+          for _i250 in xrange(_size246):
+            _elem251 = ColumnStatisticsObj()
+            _elem251.read(iprot)
+            self.colStats.append(_elem251)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I64:
+          self.partsFound = iprot.readI64()
+        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('AggrStats')
+    if self.colStats is not None:
+      oprot.writeFieldBegin('colStats', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.colStats))
+      for iter252 in self.colStats:
+        iter252.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.partsFound is not None:
+      oprot.writeFieldBegin('partsFound', TType.I64, 2)
+      oprot.writeI64(self.partsFound)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.colStats is None:
+      raise TProtocol.TProtocolException(message='Required field colStats is unset!')
+    if self.partsFound is None:
+      raise TProtocol.TProtocolException(message='Required field partsFound is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.colStats)
+    value = (value * 31) ^ hash(self.partsFound)
+    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 SetPartitionsStatsRequest:
+  """
+  Attributes:
+   - colStats
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatistics, ColumnStatistics.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, colStats=None,):
+    self.colStats = colStats
+
+  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.colStats = []
+          (_etype256, _size253) = iprot.readListBegin()
+          for _i257 in xrange(_size253):
+            _elem258 = ColumnStatistics()
+            _elem258.read(iprot)
+            self.colStats.append(_elem258)
+          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('SetPartitionsStatsRequest')
+    if self.colStats is not None:
+      oprot.writeFieldBegin('colStats', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.colStats))
+      for iter259 in self.colStats:
+        iter259.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.colStats is None:
+      raise TProtocol.TProtocolException(message='Required field colStats is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.colStats)
+    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 Schema:
+  """
+  Attributes:
+   - fieldSchemas
+   - properties
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'fieldSchemas', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 1
+    (2, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 2
+  )
+
+  def __init__(self, fieldSchemas=None, properties=None,):
+    self.fieldSchemas = fieldSchemas
+    self.properties = properties
+
+  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.fieldSchemas = []
+          (_etype263, _size260) = iprot.readListBegin()
+          for _i264 in xrange(_size260):
+            _elem265 = FieldSchema()
+            _elem265.read(iprot)
+            self.fieldSchemas.append(_elem265)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.properties = {}
+          (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin()
+          for _i270 in xrange(_size266):
+            _key271 = iprot.readString()
+            _val272 = iprot.readString()
+            self.properties[_key271] = _val272
+          iprot.readMapEnd()
+        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('Schema')
+    if self.fieldSchemas is not None:
+      oprot.writeFieldBegin('fieldSchemas', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.fieldSchemas))
+      for iter273 in self.fieldSchemas:
+        iter273.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.properties is not None:
+      oprot.writeFieldBegin('properties', TType.MAP, 2)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
+      for kiter274,viter275 in self.properties.items():
+        oprot.writeString(kiter274)
+        oprot.writeString(viter275)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.fieldSchemas)
+    value = (value * 31) ^ hash(self.properties)
+    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 EnvironmentContext:
+  """
+  Attributes:
+   - properties
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 1
+  )
+
+  def __init__(self, properties=None,):
+    self.properties = properties
+
+  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.properties = {}
+          (_ktype277, _vtype278, _size276 ) = iprot.readMapBegin()
+          for _i280 in xrange(_size276):
+            _key281 = iprot.readString()
+            _val282 = iprot.readString()
+            self.properties[_key281] = _val282
+          iprot.readMapEnd()
+        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('EnvironmentContext')
+    if self.properties is not None:
+      oprot.writeFieldBegin('properties', TType.MAP, 1)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
+      for kiter283,viter284 in self.properties.items():
+        oprot.writeString(kiter283)
+        oprot.writeString(viter284)
+      oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.statsDesc is None:
-      raise TProtocol.TProtocolException(message='Required field statsDesc is unset!')
-    if self.statsObj is None:
-      raise TProtocol.TProtocolException(message='Required field statsObj is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.statsDesc)
-    value = (value * 31) ^ hash(self.statsObj)
+    value = (value * 31) ^ hash(self.properties)
     return value
 
   def __repr__(self):
@@ -5189,22 +5918,22 @@ class ColumnStatistics:
   def __ne__(self, other):
     return not (self == other)
 
-class AggrStats:
+class PrimaryKeysRequest:
   """
   Attributes:
-   - colStats
-   - partsFound
+   - db_name
+   - tbl_name
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatisticsObj, ColumnStatisticsObj.thrift_spec)), None, ), # 1
-    (2, TType.I64, 'partsFound', None, None, ), # 2
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'tbl_name', None, None, ), # 2
   )
 
-  def __init__(self, colStats=None, partsFound=None,):
-    self.colStats = colStats
-    self.partsFound = partsFound
+  def __init__(self, db_name=None, tbl_name=None,):
+    self.db_name = db_name
+    self.tbl_name = tbl_name
 
   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:
@@ -5216,19 +5945,13 @@ class AggrStats:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.LIST:
-          self.colStats = []
-          (_etype249, _size246) = iprot.readListBegin()
-          for _i250 in xrange(_size246):
-            _elem251 = ColumnStatisticsObj()
-            _elem251.read(iprot)
-            self.colStats.append(_elem251)
-          iprot.readListEnd()
+        if ftype == TType.STRING:
+          self.db_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.I64:
-          self.partsFound = iprot.readI64()
+        if ftype == TType.STRING:
+          self.tbl_name = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -5240,33 +5963,30 @@ class AggrStats:
     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('AggrStats')
-    if self.colStats is not None:
-      oprot.writeFieldBegin('colStats', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.colStats))
-      for iter252 in self.colStats:
-        iter252.write(oprot)
-      oprot.writeListEnd()
+    oprot.writeStructBegin('PrimaryKeysRequest')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
       oprot.writeFieldEnd()
-    if self.partsFound is not None:
-      oprot.writeFieldBegin('partsFound', TType.I64, 2)
-      oprot.writeI64(self.partsFound)
+    if self.tbl_name is not None:
+      oprot.writeFieldBegin('tbl_name', TType.STRING, 2)
+      oprot.writeString(self.tbl_name)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.colStats is None:
-      raise TProtocol.TProtocolException(message='Required field colStats is unset!')
-    if self.partsFound is None:
-      raise TProtocol.TProtocolException(message='Required field partsFound is unset!')
+    if self.db_name is None:
+      raise TProtocol.TProtocolException(message='Required field db_name is unset!')
+    if self.tbl_name is None:
+      raise TProtocol.TProtocolException(message='Required field tbl_name is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.colStats)
-    value = (value * 31) ^ hash(self.partsFound)
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.tbl_name)
     return value
 
   def __repr__(self):
@@ -5280,19 +6000,19 @@ class AggrStats:
   def __ne__(self, other):
     return not (self == other)
 
-class SetPartitionsStatsRequest:
+class PrimaryKeysResponse:
   """
   Attributes:
-   - colStats
+   - primaryKeys
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatistics, ColumnStatistics.thrift_spec)), None, ), # 1
+    (1, TType.LIST, 'primaryKeys', (TType.STRUCT,(SQLPrimaryKey, SQLPrimaryKey.thrift_spec)), None, ), # 1
   )
 
-  def __init__(self, colStats=None,):
-    self.colStats = colStats
+  def __init__(self, primaryKeys=None,):
+    self.primaryKeys = primaryKeys
 
   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:
@@ -5305,12 +6025,12 @@ class SetPartitionsStatsRequest:
         break
       if fid == 1:
         if ftype == TType.LIST:
-          self.colStats = []
-          (_etype256, _size253) = iprot.readListBegin()
-          for _i257 in xrange(_size253):
-            _elem258 = ColumnStatistics()
-            _elem258.read(iprot)
-            self.colStats.append(_elem258)
+          self.primaryKeys = []
+          (_etype288, _size285) = iprot.readListBegin()
+          for _i289 in xrange(_size285):
+            _elem290 = SQLPrimaryKey()
+            _elem290.read(iprot)
+            self.primaryKeys.append(_elem290)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -5323,26 +6043,26 @@ class SetPartitionsStatsRequest:
     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('SetPartitionsStatsRequest')
-    if self.colStats is not None:
-      oprot.writeFieldBegin('colStats', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.colStats))
-      for iter259 in self.colStats:
-        iter259.write(oprot)
+    oprot.writeStructBegin('PrimaryKeysResponse')
+    if self.primaryKeys is not None:
+      oprot.writeFieldBegin('primaryKeys', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
+      for iter291 in self.primaryKeys:
+        iter291.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.colStats is None:
-      raise TProtocol.TProtocolException(message='Required field colStats is unset!')
+    if self.primaryKeys is None:
+      raise TProtocol.TProtocolException(message='Required field primaryKeys is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.colStats)
+    value = (value * 31) ^ hash(self.primaryKeys)
     return value
 
   def __repr__(self):
@@ -5356,22 +6076,28 @@ class SetPartitionsStatsRequest:
   def __ne__(self, other):
     return not (self == other)
 
-class Schema:
+class ForeignKeysRequest:
   """
   Attributes:
-   - fieldSchemas
-   - properties
+   - parent_db_name
+   - parent_tbl_name
+   - foreign_db_name
+   - foreign_tbl_name
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'fieldSchemas', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 1
-    (2, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 2
+    (1, TType.STRING, 'parent_db_name', None, None, ), # 1
+    (2, TType.STRING, 'parent_tbl_name', None, None, ), # 2
+    (3, TType.STRING, 'foreign_db_name', None, None, ), # 3
+    (4, TType.STRING, 'foreign_tbl_name', None, None, ), # 4
   )
 
-  def __init__(self, fieldSchemas=None, properties=None,):
-    self.fieldSchemas = fieldSchemas
-    self.properties = properties
+  def __init__(self, parent_db_name=None, parent_tbl_name=None, foreign_db_name=None, foreign_tbl_name=None,):
+    self.parent_db_name = parent_db_name
+    self.parent_tbl_name = parent_tbl_name
+    self.foreign_db_name = foreign_db_name
+    self.foreign_tbl_name = foreign_tbl_name
 
   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:
@@ -5383,25 +6109,23 @@ class Schema:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.LIST:
-          self.fieldSchemas = []
-          (_etype263, _size260) = iprot.readListBegin()
-          for _i264 in xrange(_size260):
-            _elem265 = FieldSchema()
-            _elem265.read(iprot)
-            self.fieldSchemas.append(_elem265)
-          iprot.readListEnd()
+        if ftype == TType.STRING:
+          self.parent_db_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.MAP:
-          self.properties = {}
-          (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin()
-          for _i270 in xrange(_size266):
-            _key271 = iprot.readString()
-            _val272 = iprot.readString()
-            self.properties[_key271] = _val272
-          iprot.readMapEnd()
+        if ftype == TType.STRING:
+          self.parent_tbl_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.foreign_db_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.foreign_tbl_name = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -5413,33 +6137,44 @@ class Schema:
     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('Schema')
-    if self.fieldSchemas is not None:
-      oprot.writeFieldBegin('fieldSchemas', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.fieldSchemas))
-      for iter273 in self.fieldSchemas:
-        iter273.write(oprot)
-      oprot.writeListEnd()
+    oprot.writeStructBegin('ForeignKeysRequest')
+    if self.parent_db_name is not None:
+      oprot.writeFieldBegin('parent_db_name', TType.STRING, 1)
+      oprot.writeString(self.parent_db_name)
       oprot.writeFieldEnd()
-    if self.properties is not None:
-      oprot.writeFieldBegin('properties', TType.MAP, 2)
-      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter274,viter275 in self.properties.items():
-        oprot.writeString(kiter274)
-        oprot.writeString(viter275)
-      oprot.writeMapEnd()
+    if self.parent_tbl_name is not None:
+      oprot.writeFieldBegin('parent_tbl_name', TType.STRING, 2)
+      oprot.writeString(self.parent_tbl_name)
+      oprot.writeFieldEnd()
+    if self.foreign_db_name is not None:
+      oprot.writeFieldBegin('foreign_db_name', TType.STRING, 3)
+      oprot.writeString(self.foreign_db_name)
+      oprot.writeFieldEnd()
+    if self.foreign_tbl_name is not None:
+      oprot.writeFieldBegin('foreign_tbl_name', TType.STRING, 4)
+      oprot.writeString(self.foreign_tbl_name)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
+    if self.parent_db_name is None:
+      raise TProtocol.TProtocolException(message='Required field parent_db_name is unset!')
+    if self.parent_tbl_name is None:
+      raise TProtocol.TProtocolException(message='Required field parent_tbl_name is unset!')
+    if self.foreign_db_name is None:
+      raise TProtocol.TProtocolException(message='Required field foreign_db_name is unset!')
+    if self.foreign_tbl_name is None:
+      raise TProtocol.TProtocolException(message='Required field foreign_tbl_name is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.fieldSchemas)
-    value = (value * 31) ^ hash(self.properties)
+    value = (value * 31) ^ hash(self.parent_db_name)
+    value = (value * 31) ^ hash(self.parent_tbl_name)
+    value = (value * 31) ^ hash(self.foreign_db_name)
+    value = (value * 31) ^ hash(self.foreign_tbl_name)
     return value
 
   def __repr__(self):
@@ -5453,19 +6188,19 @@ class Schema:
   def __ne__(self, other):
     return not (self == other)
 
-class EnvironmentContext:
+class ForeignKeysResponse:
   """
   Attributes:
-   - properties
+   - foreignKeys
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 1
+    (1, TType.LIST, 'foreignKeys', (TType.STRUCT,(SQLForeignKey, SQLForeignKey.thrift_spec)), None, ), # 1
   )
 
-  def __init__(self, properties=None,):
-    self.properties = properties
+  def __init__(self, foreignKeys=None,):
+    self.foreignKeys = foreignKeys
 
   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:
@@ -5477,14 +6212,14 @@ class EnvironmentContext:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.MAP:
-          self.properties = {}
-          (_ktype277, _vtype278, _size276 ) = iprot.readMapBegin()
-          for _i280 in xrange(_size276):
-            _key281 = iprot.readString()
-            _val282 = iprot.readString()
-            self.properties[_key281] = _val282
-          iprot.readMapEnd()
+        if ftype == TType.LIST:
+          self.foreignKeys = []
+          (_etype295, _size292) = iprot.readListBegin()
+          for _i296 in xrange(_size292):
+            _elem297 = SQLForeignKey()
+            _elem297.read(iprot)
+            self.foreignKeys.append(_elem297)
+          iprot.readListEnd()
         else:
           iprot.skip(ftype)
       else:
@@ -5496,25 +6231,26 @@ class EnvironmentContext:
     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('EnvironmentContext')
-    if self.properties is not None:
-      oprot.writeFieldBegin('properties', TType.MAP, 1)
-      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter283,viter284 in self.properties.items():
-        oprot.writeString(kiter283)
-        oprot.writeString(viter284)
-      oprot.writeMapEnd()
+    oprot.writeStructBegin('ForeignKeysResponse')
+    if self.foreignKeys is not None:
+      oprot.writeFieldBegin('foreignKeys', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
+      for iter298 in self.foreignKeys:
+        iter298.write(oprot)
+      oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
+    if self.foreignKeys is None:
+      raise TProtocol.TProtocolException(message='Required field foreignKeys is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.properties)
+    value = (value * 31) ^ hash(self.foreignKeys)
     return value
 
   def __repr__(self):
@@ -5557,11 +6293,11 @@ class PartitionsByExprResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype288, _size285) = iprot.readListBegin()
-          for _i289 in xrange(_size285):
-            _elem290 = Partition()
-            _elem290.read(iprot)
-            self.partitions.append(_elem290)
+          (_etype302, _size299) = iprot.readListBegin()
+          for _i303 in xrange(_size299):
+            _elem304 = Partition()
+            _elem304.read(iprot)
+            self.partitions.append(_elem304)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -5583,8 +6319,8 @@ class PartitionsByExprResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter291 in self.partitions:
-        iter291.write(oprot)
+      for iter305 in self.partitions:
+        iter305.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.hasUnknownPartitions is not None:
@@ -5768,11 +6504,11 @@ class TableStatsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tableStats = []
-          (_etype295, _size292) = iprot.readListBegin()
-          for _i296 in xrange(_size292):
-            _elem297 = ColumnStatisticsObj()
-            _elem297.read(iprot)
-            self.tableStats.append(_elem297)
+          (_etype309, _size306) = iprot.readListBegin()
+          for _i310 in xrange(_size306):
+            _elem311 = ColumnStatisticsObj()
+            _elem311.read(iprot)
+            self.tableStats.append(_elem311)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -5789,8 +6525,8 @@ class TableStatsResult:
     if self.tableStats is not None:
       oprot.writeFieldBegin('tableStats', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tableStats))
-      for iter298 in self.tableStats:
-        iter298.write(oprot)
+      for iter312 in self.tableStats:
+        iter312.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5844,17 +6580,17 @@ class PartitionsStatsResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.partStats = {}
-          (_ktype300, _vtype301, _size299 ) = iprot.readMapBegin()
-          for _i303 in xrange(_size299):
-            _key304 = iprot.readString()
-            _val305 = []
-            (_etype309, _size306) = iprot.readListBegin()
-            for _i310 in xrange(_size306):
-              _elem311 = ColumnStatisticsObj()
-              _elem311.read(iprot)
-              _val305.append(_elem311)
+          (_ktype314, _vtype315, _size313 ) = iprot.readMapBegin()
+          for _i317 in xrange(_size313):
+            _key318 = iprot.readString()
+            _val319 = []
+            (_etype323, _size320) = iprot.readListBegin()
+            for _i324 in xrange(_size320):
+              _elem325 = ColumnStatisticsObj()
+              _elem325.read(iprot)
+              _val319.append(_elem325)
             iprot.readListEnd()
-            self.partStats[_key304] = _val305
+            self.partStats[_key318] = _val319
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5871,11 +6607,11 @@ class PartitionsStatsResult:
     if self.partStats is not None:
       oprot.writeFieldBegin('partStats', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.LIST, len(self.partStats))
-      for kiter312,viter313 in self.partStats.items():
-        oprot.writeString(kiter312)
-        oprot.writeListBegin(TType.STRUCT, len(viter313))
-        for iter314 in viter313:
-          iter314.write(oprot)
+      for kiter326,viter327 in self.partStats.items():
+        oprot.writeString(kiter326)
+        oprot.writeListBegin(TType.STRUCT, len(viter327))
+        for iter328 in viter327:
+          iter328.write(oprot)
         oprot.writeListEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -5946,10 +6682,10 @@ class TableStatsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.colNames = []
-          (_etype318, _size315) = iprot.readListBegin()
-          for _i319 in xrange(_size315):
-            _elem320 = iprot.readString()
-            self.colNames.append(_elem320)
+          (_etype332, _size329) = iprot.readListBegin()
+          for _i333 in xrange(_size329):
+            _elem334 = iprot.readString()
+            self.colNames.append(_elem334)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -5974,8 +6710,8 @@ class TableStatsRequest:
     if self.colNames is not None:
       oprot.writeFieldBegin('colNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.colNames))
-      for iter321 in self.colNames:
-        oprot.writeString(iter321)
+      for iter335 in self.colNames:
+        oprot.writeString(iter335)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6054,20 +6790,20 @@ class PartitionsStatsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.colNames = []
-          (_etype325, _size322) = iprot.readListBegin()
-          for _i326 in xrange(_size322):
-            _elem327 = iprot.readString()
-            self.colNames.append(_elem327)
+          (_etype339, _size336) = iprot.readListBegin()
+          for _i340 in xrange(_size336):
+            _elem341 = iprot.readString()
+            self.colNames.append(_elem341)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype331, _size328) = iprot.readListBegin()
-          for _i332 in xrange(_size328):
-            _elem333 = iprot.readString()
-            self.partNames.append(_elem333)
+          (_etype345, _size342) = iprot.readListBegin()
+          for _i346 in xrange(_size342):
+            _elem347 = iprot.readString()
+            self.partNames.append(_elem347)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6092,15 +6828,15 @@ class PartitionsStatsRequest:
     if self.colNames is not None:
       oprot.writeFieldBegin('colNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.colNames))
-      for iter334 in self.colNames:
-        oprot.writeString(iter334)
+      for iter348 in self.colNames:
+        oprot.writeString(iter348)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter335 in self.partNames:
-        oprot.writeString(iter335)
+      for iter349 in self.partNames:
+        oprot.writeString(iter349)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6163,11 +6899,11 @@ class AddPartitionsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype339, _size336) = iprot.readListBegin()
-          for _i340 in xrange(_size336):
-            _elem341 = Partition()
-            _elem341.read(iprot)
-            self.partitions.append(_elem341)
+          (_etype353, _size350) = iprot.readListBegin()
+          for _i354 in xrange(_size350):
+            _elem355 = Partition()
+            _elem355.read(iprot)
+            self.partitions.append(_elem355)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6184,8 +6920,8 @@ class AddPartitionsResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter342 in self.partitions:
-        iter342.write(oprot)
+      for iter356 in self.partitions:
+        iter356.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6259,11 +6995,11 @@ class AddPartitionsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.parts = []
-          (_etype346, _size343) = iprot.readListBegin()
-          for _i347 in xrange(_size343):
-            _elem348 = Partition()
-            _elem348.read(iprot)
-            self.parts.append(_elem348)
+          (_etype360, _size357) = iprot.readListBegin()
+          for _i361 in xrange(_size357):
+            _elem362 = Partition()
+            _elem362.read(iprot)
+            self.parts.append(_elem362)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6298,8 +7034,8 @@ class AddPartitionsRequest:
     if self.parts is not None:
       oprot.writeFieldBegin('parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.parts))
-      for iter349 in self.parts:
-        iter349.write(oprot)
+      for iter363 in self.parts:
+        iter363.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ifNotExists is not None:
@@ -6371,11 +7107,11 @@ class DropPartitionsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype353, _size350) = iprot.readListBegin()
-          for _i354 in xrange(_size350):
-            _elem355 = Partition()
-            _elem355.read(iprot)
-            self.partitions.append(_elem355)
+          (_etype367, _size364) = iprot.readListBegin()
+          for _i368 in xrange(_size364):
+            _elem369 = Partition()
+            _elem369.read(iprot)
+            self.partitions.append(_elem369)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6392,8 +7128,8 @@ class DropPartitionsResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter356 in self.partitions:
-        iter356.write(oprot)
+      for iter370 in self.partitions:
+        iter370.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6528,21 +7264,21 @@ class RequestPartsSpec:
       if fid == 1:
         if ftype == TType.LIST:
           self.names = []
-          (_etype360, _size357) = iprot.readListBegin()
-          for _i361 in xrange(_size357):
-            _elem362 = iprot.readString()
-            self.names.append(_elem362)
+          (_etype374, _size371) = iprot.readListBegin()
+          for _i375 in xrange(_size371):
+            _elem376 = iprot.readString()
+            self.names.append(_elem376)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.exprs = []
-          (_etype366, _size363) = iprot.readListBegin()
-          for _i367 in xrange(_size363):
-            _elem368 = DropPartitionsExpr()
-            _elem368.read(iprot)
-            self.exprs.append(_elem368)
+          (_etype380, _size377) = iprot.readListBegin()
+          for _i381 in xrange(_size377):
+            _elem382 = DropPartitionsExpr()
+            _elem382.read(iprot)
+            self.exprs.append(_elem382)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6559,15 +7295,15 @@ class RequestPartsSpec:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter369 in self.names:
-        oprot.writeString(iter369)
+      for iter383 in self.names:
+        oprot.writeString(iter383)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.exprs is not None:
       oprot.writeFieldBegin('exprs', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.exprs))
-      for iter370 in self.exprs:
-        iter370.write(oprot)
+      for iter384 in self.exprs:
+        iter384.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6918,11 +7654,11 @@ class Function:
       elif fid == 8:
         if ftype == TType.LIST:
           self.resourceUris = []
-          (_etype374, _size371) = iprot.readListBegin()
-          for _i375 in xrange(_size371):
-            _elem376 = ResourceUri()
-            _elem376.read(iprot)
-            self.resourceUris.append(_elem376)
+          (_etype388, _size385) = iprot.readListBegin()
+          for _i389 in xrange(_size385):
+            _elem390 = ResourceUri()
+            _elem390.read(iprot)
+            self.resourceUris.append(_elem390)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6967,8 +7703,8 @@ class Function:
     if self.resourceUris is not None:
       oprot.writeFieldBegin('resourceUris', TType.LIST, 8)
       oprot.writeListBegin(TType.STRUCT, len(self.resourceUris))
-      for iter377 in self.resourceUris:
-        iter377.write(oprot)
+      for iter391 in self.resourceUris:
+        iter391.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7186,11 +7922,11 @@ class GetOpenTxnsInfoResponse:
       elif fid == 2:
         if ftype == TType.LIST:
           self.open_txns = []
-          (_etype381, _size378) = iprot.readListBegin()
-          for _i382 in xrange(_size378):
-            _elem383 = TxnInfo()
-            _elem383.read(iprot)
-            self.open_txns.append(_elem383)
+          (_etype395, _size392) = iprot.readListBegin()
+          for _i396 in xrange(_size392):
+            _elem397 = TxnInfo()
+            _elem397.read(iprot)
+            self.open_txns.append(_elem397)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7211,8 +7947,8 @@ class GetOpenTxnsInfoResponse:
     if self.open_txns is not None:
       oprot.writeFieldBegin('open_txns', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.open_txns))
-      for iter384 in self.open_txns:
-        iter384.write(oprot)
+      for iter398 in self.open_txns:
+        iter398.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7277,10 +8013,10 @@ class GetOpenTxnsResponse:
       elif fid == 2:
         if ftype == TType.SET:
           self.open_txns = set()
-          (_etype388, _size385) = iprot.readSetBegin()
-          for _i389 in xrange(_size385):
-            _elem390 = iprot.readI64()
-            self.open_txns.add(_elem390)
+          (_etype402, _size399) = iprot.readSetBegin()
+          for _i403 in xrange(_size399):
+            _elem404 = iprot.readI64()
+            self.open_txns.add(_elem404)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -7301,8 +8037,8 @@ class GetOpenTxnsResponse:
     if self.open_txns is not None:
       oprot.writeFieldBegin('open_txns', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.open_txns))
-      for iter391 in self.open_txns:
-        oprot.writeI64(iter391)
+      for iter405 in self.open_txns:
+        oprot.writeI64(iter405)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7469,10 +8205,10 @@ class OpenTxnsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txn_ids = []
-          (_etype395, _size392) = iprot.readListBegin()
-          for _i396 in xrange(_size392):
-            _elem397 = iprot.readI64()
-            self.txn_ids.append(_elem397)
+          (_etype409, _size406) = iprot.readListBegin()
+          for _i410 in xrange(_size406):
+            _elem411 = iprot.readI64()
+            self.txn_ids.append(_elem411)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7489,8 +8225,8 @@ class OpenTxnsResponse:
     if self.txn_ids is not None:
       oprot.writeFieldBegin('txn_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.txn_ids))
-      for iter398 in self.txn_ids:
-        oprot.writeI64(iter398)
+      for iter412 in self.txn_ids:
+        oprot.writeI64(iter412)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7813,11 +8549,11 @@ class LockRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.component = []
-          (_etype402, _size399) = iprot.readListBegin()
-          for _i403 in xrange(_size399):
-            _elem404 = LockComponent()
-            _elem404.read(iprot)
-            self.component.append(_elem404)
+          (_etype416, _size413) = iprot.readListBegin()
+          for _i417 in xrange(_size413):
+            _elem418 = LockComponent()
+            _elem418.read(iprot)
+            self.component.append(_elem418)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7854,8 +8590,8 @@ class LockRequest:
     if self.component is not None:
       oprot.writeFieldBegin('component', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.component))
-      for iter405 in self.component:
-        iter405.write(oprot)
+      for iter419 in self.component:
+        iter419.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txnid is not None:
@@ -8553,11 +9289,11 @@ class ShowLocksResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.locks = []
-          (_etype409, _size406) = iprot.readListBegin()
-          for _i410 in xrange(_size406):
-            _elem411 = ShowLocksResponseElement()
-            _elem411.read(iprot)
-            self.locks.append(_elem411)
+          (_etype423, _size420) = iprot.readListBegin()
+          for _i424 in xrange(_size420):
+            _elem425 = ShowLocksResponseElement()
+            _elem425.read(iprot)
+            self.locks.append(_elem425)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8574,8 +9310,8 @@ class ShowLocksResponse:
     if self.locks is not None:
       oprot.writeFieldBegin('locks', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.locks))
-      for iter412 in self.locks:
-        iter412.write(oprot)
+      for iter426 in self.locks:
+        iter426.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8790,20 +9526,20 @@ class HeartbeatTxnRangeResponse:
       if fid == 1:
         if ftype == TType.SET:
           self.aborted = set()
-          (_etype416, _size413) = iprot.readSetBegin()
-          for _i417 in xrange(_size413):
-            _elem418 = iprot.readI64()
-            self.aborted.add(_elem418)
+          (_etype430, _size427) = iprot.readSetBegin()
+          for _i431 in xrange(_size427):
+            _elem432 = iprot.readI64()
+            self.aborted.add(_elem432)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.SET:
           self.nosuch = set()
-          (_etype422, _size419) = iprot.readSetBegin()
-          for _i423 in xrange(_size419):
-            _elem424 = iprot.readI64()
-            self.nosuch.add(_elem424)
+          (_etype436, _size433) = iprot.readSetBegin()
+          for _i437 in xrange(_size433):
+            _elem438 = iprot.readI64()
+            self.nosuch.add(_elem438)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -8820,15 +9556,15 @@ class HeartbeatTxnRangeResponse:
     if self.aborted is not None:
       oprot.writeFieldBegin('aborted', TType.SET, 1)
       oprot.writeSetBegin(TType.I64, len(self.aborted))
-      for iter425 in self.aborted:
-        oprot.writeI64(iter425)
+      for iter439 in self.aborted:
+        oprot.writeI64(iter439)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.nosuch is not None:
       oprot.writeFieldBegin('nosuch', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.nosuch))
-      for iter426 in self.nosuch:
-        oprot.writeI64(iter426)
+      for iter440 in self.nosuch:
+        oprot.writeI64(iter440)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9270,11 +10006,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype430, _size427) = iprot.readListBegin()
-          for _i431 in xrange(_size427):
-            _elem432 = ShowCompactResponseElement()
-            _elem432.read(iprot)
-            self.compacts.append(_elem432)
+          (_etype444, _size441) = iprot.readListBegin()
+          for _i445 in xrange(_size441):
+            _elem446 = ShowCompactResponseElement()
+            _elem446.read(iprot)
+            self.compacts.append(_elem446)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9291,8 +10027,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter433 in self.compacts:
-        iter433.write(oprot)
+      for iter447 in self.compacts:
+        iter447.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9370,10 +10106,10 @@ class AddDynamicPartitions:
       elif fid == 4:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype437, _size434) = iprot.readListBegin()
-          for _i438 in xrange(_size434):
-            _elem439 = iprot.readString()
-            self.partitionnames.append(_elem439)
+          (_etype451, _size448) = iprot.readListBegin()
+          for _i452 in xrange(_size448):
+            _elem453 = iprot.readString()
+            self.partitionnames.append(_elem453)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9402,8 +10138,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter440 in self.partitionnames:
-        oprot.writeString(iter440)
+      for iter454 in self.partitionnames:
+        oprot.writeString(iter454)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9684,11 +10420,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype444, _size441) = iprot.readListBegin()
-          for _i445 in xrange(_size441):
-            _elem446 = NotificationEvent()
-            _elem446.read(iprot)
-            self.events.append(_elem446)
+          (_etype458, _size455) = iprot.readListBegin()
+          for _i459 in xrange(_size455):
+            _elem460 = NotificationEvent()
+            _elem460.read(iprot)
+            self.events.append(_elem460)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9705,8 +10441,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter447 in self.events:
-        iter447.write(oprot)
+      for iter461 in self.events:
+        iter461.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9827,10 +10563,10 @@ class InsertEventRequestData:
       if fid == 1:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype451, _size448) = iprot.readListBegin()
-          for _i452 in xrange(_size448):
-            _elem453 = iprot.readString()
-            self.filesAdded.append(_elem453)
+          (_etype465, _size462) = iprot.readListBegin()
+          for _i466 in xrange(_size462):
+            _elem467 = iprot.readString()
+            self.filesAdded.append(_elem467)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9847,8 +10583,8 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter454 in self.filesAdded:
-        oprot.writeString(iter454)
+      for iter468 in self.filesAdded:
+        oprot.writeString(iter468)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10001,10 +10737,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype458, _size455) = iprot.readListBegin()
-          for _i459 in xrange(_size455):
-            _elem460 = iprot.readString()
-            self.partitionVals.append(_elem460)
+          (_etype472, _size469) = iprot.readListBegin()
+          for _i473 in xrange(_size469):
+            _elem474 = iprot.readString()
+            self.partitionVals.append(_elem474)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10037,8 +10773,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter461 in self.partitionVals:
-        oprot.writeString(iter461)
+      for iter475 in self.partitionVals:
+        oprot.writeString(iter475)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10359,12 +11095,12 @@ class GetFileMetadataByExprResult:
       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
+          (_ktype477, _vtype478, _size476 ) = iprot.readMapBegin()
+          for _i480 in xrange(_size476):
+            _key481 = iprot.readI64()
+            _val482 = MetadataPpdResult()
+            _val482.read(iprot)
+            self.metadata[_key481] = _val482
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10386,9 +11122,9 @@ class GetFileMetadataByExprResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRUCT, len(self.metadata))
-      for kiter469,viter470 in self.metadata.items():
-        oprot.writeI64(kiter469)
-        viter470.write(oprot)
+      for kiter483,viter484 in self.metadata.items():
+        oprot.writeI64(kiter483)
+        viter484.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -10458,10 +11194,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype474, _size471) = iprot.readListBegin()
-          for _i475 in xrange(_size471):
-            _elem476 = iprot.readI64()
-            self.fileIds.append(_elem476)
+          (_etype488, _size485) = iprot.readListBegin()
+          for _i489 in xrange(_size485):
+            _elem490 = iprot.readI64()
+            self.fileIds.append(_elem490)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10493,8 +11229,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter477 in self.fileIds:
-        oprot.writeI64(iter477)
+      for iter491 in self.fileIds:
+        oprot.writeI64(iter491)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -10568,11 +11304,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype479, _vtype480, _size478 ) = iprot.readMapBegin()
-          for _i482 in xrange(_size478):
-            _key483 = iprot.readI64()
-            _val484 = iprot.readString()
-            self.metadata[_key483] = _val484
+          (_ktype493, _vtype494, _size492 ) = iprot.readMapBegin()
+          for _i496 in xrange(_size492):
+            _key497 = iprot.readI64()
+            _val498 = iprot.readString()
+            self.metadata[_key497] = _val498
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10594,9 +11330,9 @@ class GetFileMetadataResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRING, len(self.metadata))
-      for kiter485,viter486 in self.metadata.items():
-        oprot.writeI64(kiter485)
-        oprot.writeString(viter486)
+      for kiter499,viter500 in self.metadata.items():
+        oprot.writeI64(kiter499)
+        oprot.writeString(viter500)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -10657,10 +11393,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype490, _size487) = iprot.readListBegin()
-          for _i491 in xrange(_size487):
-            _elem492 = iprot.readI64()
-            self.fileIds.append(_elem492)
+          (_etype504, _size501) = iprot.readListBegin()
+          for _i505 in xrange(_size501):
+            _elem506 = iprot.readI64()
+            self.fileIds.append(_elem506)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10677,8 +11413,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter493 in self.fileIds:
-        oprot.writeI64(iter493)
+      for iter507 in self.fileIds:
+        oprot.writeI64(iter507)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10784,20 +11520,20 @@ class PutFileMetadataRequest:
       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)
+          (_etype511, _size508) = iprot.readListBegin()
+          for _i512 in xrange(_size508):
+            _elem513 = iprot.readI64()
+            self.fileIds.append(_elem513)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype503, _size500) = iprot.readListBegin()
-          for _i504 in xrange(_size500):
-            _elem505 = iprot.readString()
-            self.metadata.append(_elem505)
+          (_etype517, _size514) = iprot.readListBegin()
+          for _i518 in xrange(_size514):
+            _elem519 = iprot.readString()
+            self.metadata.append(_elem519)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10819,15 +11555,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter506 in self.fileIds:
-        oprot.writeI64(iter506)
+      for iter520 in self.fileIds:
+        oprot.writeI64(iter520)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter507 in self.metadata:
-        oprot.writeString(iter507)
+      for iter521 in self.metadata:
+        oprot.writeString(iter521)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -10935,10 +11671,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype511, _size508) = iprot.readListBegin()
-          for _i512 in xrange(_size508):
-            _elem513 = iprot.readI64()
-            self.fileIds.append(_elem513)
+          (_etype525, _size522) = iprot.readListBegin()
+          for _i526 in xrange(_size522):
+            _elem527 = iprot.readI64()
+            self.fileIds.append(_elem527)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10955,8 +11691,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter514 in self.fileIds:
-        oprot.writeI64(iter514)
+      for iter528 in self.fileIds:
+        oprot.writeI64(iter528)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11185,11 +11921,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype518, _size515) = iprot.readListBegin()
-          for _i519 in xrange(_size515):
-            _elem520 = Function()
-            _elem520.read(iprot)
-            self.functions.append(_elem520)
+          (_etype532, _size529) = iprot.readListBegin()
+          for _i533 in xrange(_size529):
+            _elem534 = Function()
+            _elem534.read(iprot)
+            self.functions.append(_elem534)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11206,8 +11942,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter521 in self.functions:
-        iter521.write(oprot)
+      for iter535 in self.functions:
+        iter535.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 1cf40ae..4a24a19 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -145,6 +145,78 @@ class FieldSchema
   ::Thrift::Struct.generate_accessors self
 end
 
+class SQLPrimaryKey
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  TABLE_DB = 1
+  TABLE_NAME = 2
+  COLUMN_NAME = 3
+  KEY_SEQ = 4
+  PK_NAME = 5
+  ENABLE_CSTR = 6
+  VALIDATE_CSTR = 7
+  RELY_CSTR = 8
+
+  FIELDS = {
+    TABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'table_db'},
+    TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'},
+    COLUMN_NAME => {:type => ::Thrift::Types::STRING, :name => 'column_name'},
+    KEY_SEQ => {:type => ::Thrift::Types::I32, :name => 'key_seq'},
+    PK_NAME => {:type => ::Thrift::Types::STRING, :name => 'pk_name'},
+    ENABLE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'enable_cstr'},
+    VALIDATE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'validate_cstr'},
+    RELY_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'rely_cstr'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class SQLForeignKey
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  PKTABLE_DB = 1
+  PKTABLE_NAME = 2
+  PKCOLUMN_NAME = 3
+  FKTABLE_DB = 4
+  FKTABLE_NAME = 5
+  FKCOLUMN_NAME = 6
+  KEY_SEQ = 7
+  UPDATE_RULE = 8
+  DELETE_RULE = 9
+  FK_NAME = 10
+  PK_NAME = 11
+  ENABLE_CSTR = 12
+  VALIDATE_CSTR = 13
+  RELY_CSTR = 14
+
+  FIELDS = {
+    PKTABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'pktable_db'},
+    PKTABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'pktable_name'},
+    PKCOLUMN_NAME => {:type => ::Thrift::Types::STRING, :name => 'pkcolumn_name'},
+    FKTABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'fktable_db'},
+    FKTABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'fktable_name'},
+    FKCOLUMN_NAME => {:type => ::Thrift::Types::STRING, :name => 'fkcolumn_name'},
+    KEY_SEQ => {:type => ::Thrift::Types::I32, :name => 'key_seq'},
+    UPDATE_RULE => {:type => ::Thrift::Types::I32, :name => 'update_rule'},
+    DELETE_RULE => {:type => ::Thrift::Types::I32, :name => 'delete_rule'},
+    FK_NAME => {:type => ::Thrift::Types::STRING, :name => 'fk_name'},
+    PK_NAME => {:type => ::Thrift::Types::STRING, :name => 'pk_name'},
+    ENABLE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'enable_cstr'},
+    VALIDATE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'validate_cstr'},
+    RELY_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'rely_cstr'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class Type
   include ::Thrift::Struct, ::Thrift::Struct_Union
   NAME = 1
@@ -1238,6 +1310,86 @@ class EnvironmentContext
   ::Thrift::Struct.generate_accessors self
 end
 
+class PrimaryKeysRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DB_NAME = 1
+  TBL_NAME = 2
+
+  FIELDS = {
+    DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
+    TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db_name is unset!') unless @db_name
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tbl_name is unset!') unless @tbl_name
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class PrimaryKeysResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  PRIMARYKEYS = 1
+
+  FIELDS = {
+    PRIMARYKEYS => {:type => ::Thrift::Types::LIST, :name => 'primaryKeys', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLPrimaryKey}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field primaryKeys is unset!') unless @primaryKeys
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class ForeignKeysRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  PARENT_DB_NAME = 1
+  PARENT_TBL_NAME = 2
+  FOREIGN_DB_NAME = 3
+  FOREIGN_TBL_NAME = 4
+
+  FIELDS = {
+    PARENT_DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'parent_db_name'},
+    PARENT_TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'parent_tbl_name'},
+    FOREIGN_DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'foreign_db_name'},
+    FOREIGN_TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'foreign_tbl_name'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field parent_db_name is unset!') unless @parent_db_name
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field parent_tbl_name is unset!') unless @parent_tbl_name
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field foreign_db_name is unset!') unless @foreign_db_name
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field foreign_tbl_name is unset!') unless @foreign_tbl_name
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class ForeignKeysResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  FOREIGNKEYS = 1
+
+  FIELDS = {
+    FOREIGNKEYS => {:type => ::Thrift::Types::LIST, :name => 'foreignKeys', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLForeignKey}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field foreignKeys is unset!') unless @foreignKeys
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class PartitionsByExprResult
   include ::Thrift::Struct, ::Thrift::Struct_Union
   PARTITIONS = 1


[27/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/llap
Commit: 55375ec13944eaa375d1244821a79b9a7f286371
Parents: 72b6bcd
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Tue Apr 19 11:41:52 2016 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Tue Apr 19 11:41:52 2016 -0700

----------------------------------------------------------------------
 metastore/if/hive_metastore.thrift              |   59 +
 .../upgrade/derby/hive-schema-2.1.0.derby.sql   |    6 +
 .../derby/upgrade-2.0.0-to-2.1.0.derby.sql      |    1 +
 .../upgrade/mssql/hive-schema-2.1.0.mssql.sql   |   18 +
 .../mssql/upgrade-2.0.0-to-2.1.0.mssql.sql      |    1 +
 .../upgrade/mysql/hive-schema-2.1.0.mysql.sql   |   17 +-
 .../mysql/upgrade-2.0.0-to-2.1.0.mysql.sql      |    1 +
 .../upgrade/oracle/hive-schema-2.1.0.oracle.sql |   19 +
 .../oracle/upgrade-2.0.0-to-2.1.0.oracle.sql    |    1 +
 .../postgres/hive-schema-2.1.0.postgres.sql     |   17 +
 .../upgrade-2.0.0-to-2.1.0.postgres.sql         |    1 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 4885 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  447 ++
 .../ThriftHiveMetastore_server.skeleton.cpp     |   15 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 6512 +++++++++-------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  404 +
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../metastore/api/AddPartitionsRequest.java     |   36 +-
 .../hive/metastore/api/AddPartitionsResult.java |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../metastore/api/DropPartitionsResult.java     |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../hive/metastore/api/ForeignKeysRequest.java  |  692 ++
 .../hive/metastore/api/ForeignKeysResponse.java |  443 ++
 .../hadoop/hive/metastore/api/Function.java     |   36 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../metastore/api/GetOpenTxnsInfoResponse.java  |   36 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |   32 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |   32 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |   32 +-
 .../metastore/api/PartitionsByExprResult.java   |   36 +-
 .../metastore/api/PartitionsStatsRequest.java   |   64 +-
 .../metastore/api/PartitionsStatsResult.java    |   76 +-
 .../hive/metastore/api/PrimaryKeysRequest.java  |  490 ++
 .../hive/metastore/api/PrimaryKeysResponse.java |  443 ++
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../hive/metastore/api/RequestPartsSpec.java    |   68 +-
 .../hive/metastore/api/SQLForeignKey.java       | 1715 +++++
 .../hive/metastore/api/SQLPrimaryKey.java       | 1103 +++
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/TableStatsRequest.java   |   32 +-
 .../hive/metastore/api/TableStatsResult.java    |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 7122 +++++++++++++-----
 .../gen-php/metastore/ThriftHiveMetastore.php   | 2260 ++++--
 .../src/gen/thrift/gen-php/metastore/Types.php  | 5564 ++++++++------
 .../hive_metastore/ThriftHiveMetastore-remote   |   21 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1890 +++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 1440 +++-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  152 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |  209 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   97 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   46 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   16 +
 .../hive/metastore/MetaStoreDirectSql.java      |  134 +
 .../hadoop/hive/metastore/ObjectStore.java      |  397 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   12 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   24 +
 .../hive/metastore/model/MConstraint.java       |  148 +
 metastore/src/model/package.jdo                 |   33 +
 .../DummyRawStoreControlledCommit.java          |   24 +
 .../DummyRawStoreForJdoConnection.java          |   24 +
 .../gen-py/hive_service/ThriftHive-remote       |   21 +
 70 files changed, 27516 insertions(+), 10556 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 6a55962..acebf7a 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -41,6 +41,34 @@ struct FieldSchema {
   3: string comment
 }
 
+struct SQLPrimaryKey {
+  1: string table_db,    // table schema
+  2: string table_name,  // table name
+  3: string column_name, // column name
+  4: i32 key_seq,        // sequence number within primary key
+  5: string pk_name,     // primary key name
+  6: bool enable_cstr,   // Enable/Disable
+  7: bool validate_cstr,  // Validate/No validate
+  8: bool rely_cstr      // Rely/No Rely
+}
+
+struct SQLForeignKey {
+  1: string pktable_db,    // primary key table schema
+  2: string pktable_name,  // primary key table name
+  3: string pkcolumn_name, // primary key column name
+  4: string fktable_db,    // foreign key table schema
+  5: string fktable_name,  // foreign key table name
+  6: string fkcolumn_name, // foreign key column name
+  7: i32 key_seq,          // sequence within foreign key
+  8: i32 update_rule,      // what happens to foreign key when parent key is updated
+  9: i32 delete_rule,      // what happens to foreign key when parent key is deleted
+  10: string fk_name,      // foreign key name
+  11: string pk_name,      // primary key name
+  12: bool enable_cstr,    // Enable/Disable
+  13: bool validate_cstr,  // Validate/No validate
+  14: bool rely_cstr       // Rely/No Rely
+}
+
 struct Type {
   1: string          name,             // one of the types in PrimitiveTypes or CollectionTypes or User defined types
   2: optional string type1,            // object type if the name is 'list' (LIST_TYPE), key type if the name is 'map' (MAP_TYPE)
@@ -439,6 +467,27 @@ struct EnvironmentContext {
   1: map<string, string> properties
 }
 
+struct PrimaryKeysRequest {
+  1: required string db_name,
+  2: required string tbl_name
+}
+
+struct PrimaryKeysResponse {
+  1: required list<SQLPrimaryKey> primaryKeys
+}
+
+struct ForeignKeysRequest {
+  1: required string parent_db_name,
+  2: required string parent_tbl_name,
+  3: required string foreign_db_name,
+  4: required string foreign_tbl_name
+}
+
+struct ForeignKeysResponse {
+  1: required list<SQLForeignKey> foreignKeys
+}
+
+
 // Return type for get_partitions_by_expr
 struct PartitionsByExprResult {
   1: required list<Partition> partitions,
@@ -940,6 +989,10 @@ service ThriftHiveMetastore extends fb303.FacebookService
       throws (1:AlreadyExistsException o1,
               2:InvalidObjectException o2, 3:MetaException o3,
               4:NoSuchObjectException o4)
+  void create_table_with_constraints(1:Table tbl, 2: list<SQLPrimaryKey> primaryKeys, 3: list<SQLForeignKey> foreignKeys)
+      throws (1:AlreadyExistsException o1,
+              2:InvalidObjectException o2, 3:MetaException o3,
+              4:NoSuchObjectException o4)
   // drops the table and all the partitions associated with it if the table has partitions
   // delete data (including partitions) if deleteData is set to true
   void drop_table(1:string dbname, 2:string name, 3:bool deleteData)
@@ -1179,6 +1232,12 @@ service ThriftHiveMetastore extends fb303.FacebookService
   list<string> get_index_names(1:string db_name, 2:string tbl_name, 3:i16 max_indexes=-1)
                        throws(1:MetaException o2)
 
+ //primary keys and foreign keys
+  PrimaryKeysResponse get_primary_keys(1:PrimaryKeysRequest request)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  ForeignKeysResponse get_foreign_keys(1:ForeignKeysRequest request)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
   // column statistics interfaces
 
   // update APIs persist the column statistics object(s) that are passed in. If statistics already

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql b/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
index 42f4eb6..2ef7223 100644
--- a/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
@@ -108,6 +108,12 @@ CREATE TABLE "APP"."NOTIFICATION_LOG" ("NL_ID" BIGINT NOT NULL, "DB_NAME" VARCHA
 
 CREATE TABLE "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID" BIGINT NOT NULL, "NEXT_EVENT_ID" BIGINT NOT NULL);
 
+CREATE TABLE "APP"."KEY_CONSTRAINTS" ("CHILD_CD_ID" BIGINT, "CHILD_TBL_ID" BIGINT, "PARENT_CD_ID" BIGINT NOT NULL, "PARENT_TBL_ID" BIGINT NOT NULL,  "POSITION" BIGINT NOT NULL, "CONSTRAINT_NAME" VARCHAR(400 NOT NULL, "CONSTRAINT_TYPE" SMALLINT NOT NULL, "UPDATE_RULE" SMALLINT, "DELETE_RULE" SMALLINT, "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL);
+
+ALTER TABLE "APP"."KEY_CONSTRAINTS" ADD CONSTRAINT "CONSTRAINTS_PK" PRIMARY KEY ("CONSTRAINT_NAME", "POSITION");
+
+CREATE INDEX "APP"."CONSTRAINTS_PARENT_TBL_ID_INDEX" ON "APP"."KEY_CONSTRAINTS"("PARENT_TBL_ID");
+
 -- ----------------------------------------------
 -- DDL Statements for indexes
 -- ----------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
index a0bac3c..dde8c45 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
@@ -1,4 +1,5 @@
 -- Upgrade MetaStore schema from 2.0.0 to 2.1.0
 RUN '033-HIVE-12892.derby.sql';
+RUN '034-HIVE-13076.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
index cf5a662..2d9cf76 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
@@ -993,6 +993,24 @@ CREATE TABLE AUX_TABLE (
 )
 );
 
+CREATE TABLE KEY_CONSTRAINTS
+(
+  CHILD_CD_ID BIGINT,
+  CHILD_TBL_ID BIGINT,
+  PARENT_CD_ID BIGINT NOT NULL,
+  PARENT_TBL_ID BIGINT NOT NULL,
+  POSITION INT NOT NULL,
+  CONSTRAINT_NAME VARCHAR(400) NOT NULL,
+  CONSTRAINT_TYPE SMALLINT NOT NULL,
+  UPDATE_RULE SMALLINT,
+  DELETE_RULE SMALLINT,
+  ENABLE_VALIDATE_RELY SMALLINT NOT NULL
+) ;
+
+ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAINT_NAME, POSITION);
+
+CREATE INDEX CONSTRAINTS_PARENT_TBL_ID__INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
+
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
index f25daf2..3e5cb30 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS MESSAGE;
 
 :r 018-HIVE-12892.mssql.sql;
+:r 019-HIVE-13076.mssql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
index 6fd3209..466e950 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
@@ -819,7 +819,22 @@ CREATE TABLE IF NOT EXISTS `NOTIFICATION_SEQUENCE`
     PRIMARY KEY (`NNI_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-
+CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
+(
+  `CHILD_CD_ID` BIGINT,
+  `CHILD_TBL_ID` BIGINT,
+  `PARENT_CD_ID` BIGINT NOT NULL,
+  `PARENT_TBL_ID` BIGINT NOT NULL,
+  `POSITION` BIGINT NOT NULL,
+  `CONSTRAINT_NAME` VARCHAR(400) NOT NULL,
+  `CONSTRAINT_TYPE` SMALLINT(6)  NOT NULL,
+  `UPDATE_RULE` SMALLINT(6),
+  `DELETE_RULE` SMALLINT(6),
+  `ENABLE_VALIDATE_RELY` SMALLINT(6) NOT NULL,
+  PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE INDEX `CONSTRAINTS_PARENT_TABLE_ID_INDEX` ON KEY_CONSTRAINTS (`PARENT_TBL_ID`) USING BTREE;
 
 -- ----------------------------
 -- Transaction and Lock Tables

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
index e790636..eb21f73 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS ' ';
 
 SOURCE 033-HIVE-12892.mysql.sql;
+SOURCE 034-HIVE-13076.mysql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
index 774f6be..f57e588 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
@@ -786,6 +786,25 @@ ALTER TABLE FUNC_RU ADD CONSTRAINT FUNC_RU_FK1 FOREIGN KEY (FUNC_ID) REFERENCES
 
 CREATE INDEX FUNC_RU_N49 ON FUNC_RU (FUNC_ID);
 
+CREATE TABLE KEY_CONSTRAINTS
+(
+  CHILD_CD_ID NUMBER,
+  CHILD_TBL_ID NUMBER,
+  PARENT_CD_ID NUMBER NOT NULL,
+  PARENT_TBL_ID NUMBER NOT NULL,
+  POSITION NUMBER NOT NULL,
+  CONSTRAINT_NAME VARCHAR(400) NOT NULL,
+  CONSTRAINT_TYPE NUMBER NOT NULL,
+  UPDATE_RULE NUMBER,
+  DELETE_RULE NUMBER,
+  ENABLE_VALIDATE_RELY NUMBER NOT NULL
+) ;
+
+ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAINT_NAME, POSITION);
+
+CREATE INDEX CONSTRAINTS_PARENT_TBL_ID_INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
+
+
 ------------------------------
 -- Transaction and lock tables
 ------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
index 8368d08..8c065a1 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS Status from dual;
 
 @033-HIVE-12892.oracle.sql;
+@034-HIVE-13076.oracle.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
index 7463a37..e209489 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
@@ -594,6 +594,23 @@ CREATE TABLE "NOTIFICATION_SEQUENCE"
     PRIMARY KEY ("NNI_ID")
 );
 
+CREATE TABLE "KEY_CONSTRAINTS"
+(
+  "CHILD_CD_ID" BIGINT,
+  "CHILD_TBL_ID" BIGINT,
+  "PARENT_CD_ID" BIGINT NOT NULL,
+  "PARENT_TBL_ID" BIGINT NOT NULL,
+  "POSITION" BIGINT NOT NULL,
+  "CONSTRAINT_NAME" VARCHAR(400) NOT NULL,
+  "CONSTRAINT_TYPE" SMALLINT NOT NULL,
+  "UPDATE_RULE" SMALLINT,
+  "DELETE_RULE"	SMALLINT,
+  "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL,
+  PRIMARY KEY ("CONSTRAINT_NAME", "POSITION")
+) ;
+
+CREATE INDEX "CONSTRAINTS_PARENT_TBLID_INDEX" ON "KEY_CONSTRAINTS" USING BTREE ("PARENT_TBL_ID");
+
 --
 -- Name: BUCKETING_COLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
 --

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
index 6172407..e96a6ec 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0';
 
 \i 032-HIVE-12892.postgres.sql;
+\i 033-HIVE-13076.postgres.sql;
 
 UPDATE "VERSION" SET "SCHEMA_VERSION"='2.1.0', "VERSION_COMMENT"='Hive release version 2.1.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0';


[10/30] hive git commit: HIVE-13505. Skip running TestDummy where possibe during precommit builds. (Siddharth Seth, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
HIVE-13505. Skip running TestDummy where possibe during precommit builds. (Siddharth Seth, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/llap
Commit: c3fee305ec7a039a072e20ea8c3e068802fd8b45
Parents: cc3544b
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 18 10:42:33 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Apr 18 10:42:33 2016 -0700

----------------------------------------------------------------------
 testutils/ptest2/src/main/resources/source-prep.vm | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c3fee305/testutils/ptest2/src/main/resources/source-prep.vm
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/resources/source-prep.vm b/testutils/ptest2/src/main/resources/source-prep.vm
index 97fb69c..9c83a14 100644
--- a/testutils/ptest2/src/main/resources/source-prep.vm
+++ b/testutils/ptest2/src/main/resources/source-prep.vm
@@ -97,10 +97,8 @@ cd $workingDir/
       done
     #end
     mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
-    mvn -B test -Dmaven.repo.local=$workingDir/maven -Dtest=TestDummy $mavenArgs $mavenTestArgs
     cd itests
     mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
-    mvn -B test -Dmaven.repo.local=$workingDir/maven -Dtest=TestDummy $mavenArgs $mavenTestArgs
   elif [[ "${buildTool}" == "ant" ]]
   then
     ant $antArgs -Divy.default.ivy.user.dir=$workingDir/ivy \


[22/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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
index 89eb819..1405203 100644
--- 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
@@ -444,16 +444,16 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
           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)
+                org.apache.thrift.protocol.TMap _map540 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map540.size);
+                long _key541;
+                MetadataPpdResult _val542;
+                for (int _i543 = 0; _i543 < _map540.size; ++_i543)
                 {
-                  _key525 = iprot.readI64();
-                  _val526 = new MetadataPpdResult();
-                  _val526.read(iprot);
-                  struct.metadata.put(_key525, _val526);
+                  _key541 = iprot.readI64();
+                  _val542 = new MetadataPpdResult();
+                  _val542.read(iprot);
+                  struct.metadata.put(_key541, _val542);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
         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> _iter528 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter544 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter528.getKey());
-            _iter528.getValue().write(oprot);
+            oprot.writeI64(_iter544.getKey());
+            _iter544.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter529 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter545 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter529.getKey());
-          _iter529.getValue().write(oprot);
+          oprot.writeI64(_iter545.getKey());
+          _iter545.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
     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 _map530 = 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*_map530.size);
-        long _key531;
-        MetadataPpdResult _val532;
-        for (int _i533 = 0; _i533 < _map530.size; ++_i533)
+        org.apache.thrift.protocol.TMap _map546 = 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*_map546.size);
+        long _key547;
+        MetadataPpdResult _val548;
+        for (int _i549 = 0; _i549 < _map546.size; ++_i549)
         {
-          _key531 = iprot.readI64();
-          _val532 = new MetadataPpdResult();
-          _val532.read(iprot);
-          struct.metadata.put(_key531, _val532);
+          _key547 = iprot.readI64();
+          _val548 = new MetadataPpdResult();
+          _val548.read(iprot);
+          struct.metadata.put(_key547, _val548);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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
index 2408ad1..ae867d0 100644
--- 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
@@ -351,13 +351,13 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list552 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list552.size);
-                long _elem553;
-                for (int _i554 = 0; _i554 < _list552.size; ++_i554)
+                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)
                 {
-                  _elem553 = iprot.readI64();
-                  struct.fileIds.add(_elem553);
+                  _elem569 = iprot.readI64();
+                  struct.fileIds.add(_elem569);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
         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 _iter555 : struct.fileIds)
+          for (long _iter571 : struct.fileIds)
           {
-            oprot.writeI64(_iter555);
+            oprot.writeI64(_iter571);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter556 : struct.fileIds)
+        for (long _iter572 : struct.fileIds)
         {
-          oprot.writeI64(_iter556);
+          oprot.writeI64(_iter572);
         }
       }
     }
@@ -421,13 +421,13 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
     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 _list557 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list557.size);
-        long _elem558;
-        for (int _i559 = 0; _i559 < _list557.size; ++_i559)
+        org.apache.thrift.protocol.TList _list573 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list573.size);
+        long _elem574;
+        for (int _i575 = 0; _i575 < _list573.size; ++_i575)
         {
-          _elem558 = iprot.readI64();
-          struct.fileIds.add(_elem558);
+          _elem574 = iprot.readI64();
+          struct.fileIds.add(_elem574);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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
index 8946635..21c9eea 100644
--- 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
@@ -433,15 +433,15 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map542 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map542.size);
-                long _key543;
-                ByteBuffer _val544;
-                for (int _i545 = 0; _i545 < _map542.size; ++_i545)
+                org.apache.thrift.protocol.TMap _map558 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map558.size);
+                long _key559;
+                ByteBuffer _val560;
+                for (int _i561 = 0; _i561 < _map558.size; ++_i561)
                 {
-                  _key543 = iprot.readI64();
-                  _val544 = iprot.readBinary();
-                  struct.metadata.put(_key543, _val544);
+                  _key559 = iprot.readI64();
+                  _val560 = iprot.readBinary();
+                  struct.metadata.put(_key559, _val560);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
         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> _iter546 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter562 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter546.getKey());
-            oprot.writeBinary(_iter546.getValue());
+            oprot.writeI64(_iter562.getKey());
+            oprot.writeBinary(_iter562.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter547 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter563 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter547.getKey());
-          oprot.writeBinary(_iter547.getValue());
+          oprot.writeI64(_iter563.getKey());
+          oprot.writeBinary(_iter563.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
     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 _map548 = 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*_map548.size);
-        long _key549;
-        ByteBuffer _val550;
-        for (int _i551 = 0; _i551 < _map548.size; ++_i551)
+        org.apache.thrift.protocol.TMap _map564 = 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*_map564.size);
+        long _key565;
+        ByteBuffer _val566;
+        for (int _i567 = 0; _i567 < _map564.size; ++_i567)
         {
-          _key549 = iprot.readI64();
-          _val550 = iprot.readBinary();
-          struct.metadata.put(_key549, _val550);
+          _key565 = iprot.readI64();
+          _val566 = iprot.readBinary();
+          struct.metadata.put(_key565, _val566);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 629c042..b2f98d6 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
@@ -447,14 +447,14 @@ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenT
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list428 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<TxnInfo>(_list428.size);
-                TxnInfo _elem429;
-                for (int _i430 = 0; _i430 < _list428.size; ++_i430)
+                org.apache.thrift.protocol.TList _list444 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<TxnInfo>(_list444.size);
+                TxnInfo _elem445;
+                for (int _i446 = 0; _i446 < _list444.size; ++_i446)
                 {
-                  _elem429 = new TxnInfo();
-                  _elem429.read(iprot);
-                  struct.open_txns.add(_elem429);
+                  _elem445 = new TxnInfo();
+                  _elem445.read(iprot);
+                  struct.open_txns.add(_elem445);
                 }
                 iprot.readListEnd();
               }
@@ -483,9 +483,9 @@ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenT
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.open_txns.size()));
-          for (TxnInfo _iter431 : struct.open_txns)
+          for (TxnInfo _iter447 : struct.open_txns)
           {
-            _iter431.write(oprot);
+            _iter447.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -511,9 +511,9 @@ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenT
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (TxnInfo _iter432 : struct.open_txns)
+        for (TxnInfo _iter448 : struct.open_txns)
         {
-          _iter432.write(oprot);
+          _iter448.write(oprot);
         }
       }
     }
@@ -524,14 +524,14 @@ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenT
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list433 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.open_txns = new ArrayList<TxnInfo>(_list433.size);
-        TxnInfo _elem434;
-        for (int _i435 = 0; _i435 < _list433.size; ++_i435)
+        org.apache.thrift.protocol.TList _list449 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.open_txns = new ArrayList<TxnInfo>(_list449.size);
+        TxnInfo _elem450;
+        for (int _i451 = 0; _i451 < _list449.size; ++_i451)
         {
-          _elem434 = new TxnInfo();
-          _elem434.read(iprot);
-          struct.open_txns.add(_elem434);
+          _elem450 = new TxnInfo();
+          _elem450.read(iprot);
+          struct.open_txns.add(_elem450);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 9f57a4a..ba99b89 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
@@ -444,13 +444,13 @@ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsR
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set436 = iprot.readSetBegin();
-                struct.open_txns = new HashSet<Long>(2*_set436.size);
-                long _elem437;
-                for (int _i438 = 0; _i438 < _set436.size; ++_i438)
+                org.apache.thrift.protocol.TSet _set452 = iprot.readSetBegin();
+                struct.open_txns = new HashSet<Long>(2*_set452.size);
+                long _elem453;
+                for (int _i454 = 0; _i454 < _set452.size; ++_i454)
                 {
-                  _elem437 = iprot.readI64();
-                  struct.open_txns.add(_elem437);
+                  _elem453 = iprot.readI64();
+                  struct.open_txns.add(_elem453);
                 }
                 iprot.readSetEnd();
               }
@@ -479,9 +479,9 @@ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsR
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.open_txns.size()));
-          for (long _iter439 : struct.open_txns)
+          for (long _iter455 : struct.open_txns)
           {
-            oprot.writeI64(_iter439);
+            oprot.writeI64(_iter455);
           }
           oprot.writeSetEnd();
         }
@@ -507,9 +507,9 @@ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsR
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (long _iter440 : struct.open_txns)
+        for (long _iter456 : struct.open_txns)
         {
-          oprot.writeI64(_iter440);
+          oprot.writeI64(_iter456);
         }
       }
     }
@@ -520,13 +520,13 @@ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsR
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set441 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.open_txns = new HashSet<Long>(2*_set441.size);
-        long _elem442;
-        for (int _i443 = 0; _i443 < _set441.size; ++_i443)
+        org.apache.thrift.protocol.TSet _set457 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.open_txns = new HashSet<Long>(2*_set457.size);
+        long _elem458;
+        for (int _i459 = 0; _i459 < _set457.size; ++_i459)
         {
-          _elem442 = iprot.readI64();
-          struct.open_txns.add(_elem442);
+          _elem458 = iprot.readI64();
+          struct.open_txns.add(_elem458);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 b00fb9c..5b399ae 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
@@ -453,13 +453,13 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set468 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set468.size);
-                long _elem469;
-                for (int _i470 = 0; _i470 < _set468.size; ++_i470)
+                org.apache.thrift.protocol.TSet _set484 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set484.size);
+                long _elem485;
+                for (int _i486 = 0; _i486 < _set484.size; ++_i486)
                 {
-                  _elem469 = iprot.readI64();
-                  struct.aborted.add(_elem469);
+                  _elem485 = iprot.readI64();
+                  struct.aborted.add(_elem485);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set471 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set471.size);
-                long _elem472;
-                for (int _i473 = 0; _i473 < _set471.size; ++_i473)
+                org.apache.thrift.protocol.TSet _set487 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set487.size);
+                long _elem488;
+                for (int _i489 = 0; _i489 < _set487.size; ++_i489)
                 {
-                  _elem472 = iprot.readI64();
-                  struct.nosuch.add(_elem472);
+                  _elem488 = iprot.readI64();
+                  struct.nosuch.add(_elem488);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter474 : struct.aborted)
+          for (long _iter490 : struct.aborted)
           {
-            oprot.writeI64(_iter474);
+            oprot.writeI64(_iter490);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter475 : struct.nosuch)
+          for (long _iter491 : struct.nosuch)
           {
-            oprot.writeI64(_iter475);
+            oprot.writeI64(_iter491);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter476 : struct.aborted)
+        for (long _iter492 : struct.aborted)
         {
-          oprot.writeI64(_iter476);
+          oprot.writeI64(_iter492);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter477 : struct.nosuch)
+        for (long _iter493 : struct.nosuch)
         {
-          oprot.writeI64(_iter477);
+          oprot.writeI64(_iter493);
         }
       }
     }
@@ -560,24 +560,24 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
     public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatTxnRangeResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set478 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set478.size);
-        long _elem479;
-        for (int _i480 = 0; _i480 < _set478.size; ++_i480)
+        org.apache.thrift.protocol.TSet _set494 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set494.size);
+        long _elem495;
+        for (int _i496 = 0; _i496 < _set494.size; ++_i496)
         {
-          _elem479 = iprot.readI64();
-          struct.aborted.add(_elem479);
+          _elem495 = iprot.readI64();
+          struct.aborted.add(_elem495);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set481 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set481.size);
-        long _elem482;
-        for (int _i483 = 0; _i483 < _set481.size; ++_i483)
+        org.apache.thrift.protocol.TSet _set497 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set497.size);
+        long _elem498;
+        for (int _i499 = 0; _i499 < _set497.size; ++_i499)
         {
-          _elem482 = iprot.readI64();
-          struct.nosuch.add(_elem482);
+          _elem498 = iprot.readI64();
+          struct.nosuch.add(_elem498);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 488d3a0..7511336 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
@@ -351,13 +351,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
           case 1: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list508 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list508.size);
-                String _elem509;
-                for (int _i510 = 0; _i510 < _list508.size; ++_i510)
+                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list524.size);
+                String _elem525;
+                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
                 {
-                  _elem509 = iprot.readString();
-                  struct.filesAdded.add(_elem509);
+                  _elem525 = iprot.readString();
+                  struct.filesAdded.add(_elem525);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter511 : struct.filesAdded)
+          for (String _iter527 : struct.filesAdded)
           {
-            oprot.writeString(_iter511);
+            oprot.writeString(_iter527);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter512 : struct.filesAdded)
+        for (String _iter528 : struct.filesAdded)
         {
-          oprot.writeString(_iter512);
+          oprot.writeString(_iter528);
         }
       }
     }
@@ -421,13 +421,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list513.size);
-        String _elem514;
-        for (int _i515 = 0; _i515 < _list513.size; ++_i515)
+        org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list529.size);
+        String _elem530;
+        for (int _i531 = 0; _i531 < _list529.size; ++_i531)
         {
-          _elem514 = iprot.readString();
-          struct.filesAdded.add(_elem514);
+          _elem530 = iprot.readString();
+          struct.filesAdded.add(_elem530);
         }
       }
       struct.setFilesAddedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 f39f582..bd10329 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
@@ -689,14 +689,14 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list452 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list452.size);
-                LockComponent _elem453;
-                for (int _i454 = 0; _i454 < _list452.size; ++_i454)
+                org.apache.thrift.protocol.TList _list468 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list468.size);
+                LockComponent _elem469;
+                for (int _i470 = 0; _i470 < _list468.size; ++_i470)
                 {
-                  _elem453 = new LockComponent();
-                  _elem453.read(iprot);
-                  struct.component.add(_elem453);
+                  _elem469 = new LockComponent();
+                  _elem469.read(iprot);
+                  struct.component.add(_elem469);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter455 : struct.component)
+          for (LockComponent _iter471 : struct.component)
           {
-            _iter455.write(oprot);
+            _iter471.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter456 : struct.component)
+        for (LockComponent _iter472 : struct.component)
         {
-          _iter456.write(oprot);
+          _iter472.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
     public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list457.size);
-        LockComponent _elem458;
-        for (int _i459 = 0; _i459 < _list457.size; ++_i459)
+        org.apache.thrift.protocol.TList _list473 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list473.size);
+        LockComponent _elem474;
+        for (int _i475 = 0; _i475 < _list473.size; ++_i475)
         {
-          _elem458 = new LockComponent();
-          _elem458.read(iprot);
-          struct.component.add(_elem458);
+          _elem474 = new LockComponent();
+          _elem474.read(iprot);
+          struct.component.add(_elem474);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 fcbbd18..8010cf5 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
@@ -354,14 +354,14 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list500.size);
-                NotificationEvent _elem501;
-                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
+                org.apache.thrift.protocol.TList _list516 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list516.size);
+                NotificationEvent _elem517;
+                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
                 {
-                  _elem501 = new NotificationEvent();
-                  _elem501.read(iprot);
-                  struct.events.add(_elem501);
+                  _elem517 = new NotificationEvent();
+                  _elem517.read(iprot);
+                  struct.events.add(_elem517);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter503 : struct.events)
+          for (NotificationEvent _iter519 : struct.events)
           {
-            _iter503.write(oprot);
+            _iter519.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter504 : struct.events)
+        for (NotificationEvent _iter520 : struct.events)
         {
-          _iter504.write(oprot);
+          _iter520.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list505.size);
-        NotificationEvent _elem506;
-        for (int _i507 = 0; _i507 < _list505.size; ++_i507)
+        org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list521.size);
+        NotificationEvent _elem522;
+        for (int _i523 = 0; _i523 < _list521.size; ++_i523)
         {
-          _elem506 = new NotificationEvent();
-          _elem506.read(iprot);
-          struct.events.add(_elem506);
+          _elem522 = new NotificationEvent();
+          _elem522.read(iprot);
+          struct.events.add(_elem522);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 e11a2b3..f745954 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
@@ -351,13 +351,13 @@ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsRespons
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list444 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list444.size);
-                long _elem445;
-                for (int _i446 = 0; _i446 < _list444.size; ++_i446)
+                org.apache.thrift.protocol.TList _list460 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list460.size);
+                long _elem461;
+                for (int _i462 = 0; _i462 < _list460.size; ++_i462)
                 {
-                  _elem445 = iprot.readI64();
-                  struct.txn_ids.add(_elem445);
+                  _elem461 = iprot.readI64();
+                  struct.txn_ids.add(_elem461);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsRespons
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter447 : struct.txn_ids)
+          for (long _iter463 : struct.txn_ids)
           {
-            oprot.writeI64(_iter447);
+            oprot.writeI64(_iter463);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsRespons
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter448 : struct.txn_ids)
+        for (long _iter464 : struct.txn_ids)
         {
-          oprot.writeI64(_iter448);
+          oprot.writeI64(_iter464);
         }
       }
     }
@@ -421,13 +421,13 @@ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsRespons
     public void read(org.apache.thrift.protocol.TProtocol prot, OpenTxnsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list449 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list449.size);
-        long _elem450;
-        for (int _i451 = 0; _i451 < _list449.size; ++_i451)
+        org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list465.size);
+        long _elem466;
+        for (int _i467 = 0; _i467 < _list465.size; ++_i467)
         {
-          _elem450 = iprot.readI64();
-          struct.txn_ids.add(_elem450);
+          _elem466 = iprot.readI64();
+          struct.txn_ids.add(_elem466);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 12ae66d..2d3c156 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
@@ -439,14 +439,14 @@ public class PartitionsByExprResult implements org.apache.thrift.TBase<Partition
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list322 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list322.size);
-                Partition _elem323;
-                for (int _i324 = 0; _i324 < _list322.size; ++_i324)
+                org.apache.thrift.protocol.TList _list338 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list338.size);
+                Partition _elem339;
+                for (int _i340 = 0; _i340 < _list338.size; ++_i340)
                 {
-                  _elem323 = new Partition();
-                  _elem323.read(iprot);
-                  struct.partitions.add(_elem323);
+                  _elem339 = new Partition();
+                  _elem339.read(iprot);
+                  struct.partitions.add(_elem339);
                 }
                 iprot.readListEnd();
               }
@@ -480,9 +480,9 @@ public class PartitionsByExprResult implements org.apache.thrift.TBase<Partition
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter325 : struct.partitions)
+          for (Partition _iter341 : struct.partitions)
           {
-            _iter325.write(oprot);
+            _iter341.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,9 +510,9 @@ public class PartitionsByExprResult implements org.apache.thrift.TBase<Partition
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitions.size());
-        for (Partition _iter326 : struct.partitions)
+        for (Partition _iter342 : struct.partitions)
         {
-          _iter326.write(oprot);
+          _iter342.write(oprot);
         }
       }
       oprot.writeBool(struct.hasUnknownPartitions);
@@ -522,14 +522,14 @@ public class PartitionsByExprResult implements org.apache.thrift.TBase<Partition
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitions = new ArrayList<Partition>(_list327.size);
-        Partition _elem328;
-        for (int _i329 = 0; _i329 < _list327.size; ++_i329)
+        org.apache.thrift.protocol.TList _list343 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitions = new ArrayList<Partition>(_list343.size);
+        Partition _elem344;
+        for (int _i345 = 0; _i345 < _list343.size; ++_i345)
         {
-          _elem328 = new Partition();
-          _elem328.read(iprot);
-          struct.partitions.add(_elem328);
+          _elem344 = new Partition();
+          _elem344.read(iprot);
+          struct.partitions.add(_elem344);
         }
       }
       struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 8416369..0ac7481 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
@@ -639,13 +639,13 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list364 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list364.size);
-                String _elem365;
-                for (int _i366 = 0; _i366 < _list364.size; ++_i366)
+                org.apache.thrift.protocol.TList _list380 = iprot.readListBegin();
+                struct.colNames = new ArrayList<String>(_list380.size);
+                String _elem381;
+                for (int _i382 = 0; _i382 < _list380.size; ++_i382)
                 {
-                  _elem365 = iprot.readString();
-                  struct.colNames.add(_elem365);
+                  _elem381 = iprot.readString();
+                  struct.colNames.add(_elem381);
                 }
                 iprot.readListEnd();
               }
@@ -657,13 +657,13 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
           case 4: // PART_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list367 = iprot.readListBegin();
-                struct.partNames = new ArrayList<String>(_list367.size);
-                String _elem368;
-                for (int _i369 = 0; _i369 < _list367.size; ++_i369)
+                org.apache.thrift.protocol.TList _list383 = iprot.readListBegin();
+                struct.partNames = new ArrayList<String>(_list383.size);
+                String _elem384;
+                for (int _i385 = 0; _i385 < _list383.size; ++_i385)
                 {
-                  _elem368 = iprot.readString();
-                  struct.partNames.add(_elem368);
+                  _elem384 = iprot.readString();
+                  struct.partNames.add(_elem384);
                 }
                 iprot.readListEnd();
               }
@@ -699,9 +699,9 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
         oprot.writeFieldBegin(COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.colNames.size()));
-          for (String _iter370 : struct.colNames)
+          for (String _iter386 : struct.colNames)
           {
-            oprot.writeString(_iter370);
+            oprot.writeString(_iter386);
           }
           oprot.writeListEnd();
         }
@@ -711,9 +711,9 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
         oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-          for (String _iter371 : struct.partNames)
+          for (String _iter387 : struct.partNames)
           {
-            oprot.writeString(_iter371);
+            oprot.writeString(_iter387);
           }
           oprot.writeListEnd();
         }
@@ -740,16 +740,16 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter372 : struct.colNames)
+        for (String _iter388 : struct.colNames)
         {
-          oprot.writeString(_iter372);
+          oprot.writeString(_iter388);
         }
       }
       {
         oprot.writeI32(struct.partNames.size());
-        for (String _iter373 : struct.partNames)
+        for (String _iter389 : struct.partNames)
         {
-          oprot.writeString(_iter373);
+          oprot.writeString(_iter389);
         }
       }
     }
@@ -762,24 +762,24 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list374 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list374.size);
-        String _elem375;
-        for (int _i376 = 0; _i376 < _list374.size; ++_i376)
+        org.apache.thrift.protocol.TList _list390 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.colNames = new ArrayList<String>(_list390.size);
+        String _elem391;
+        for (int _i392 = 0; _i392 < _list390.size; ++_i392)
         {
-          _elem375 = iprot.readString();
-          struct.colNames.add(_elem375);
+          _elem391 = iprot.readString();
+          struct.colNames.add(_elem391);
         }
       }
       struct.setColNamesIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list377 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partNames = new ArrayList<String>(_list377.size);
-        String _elem378;
-        for (int _i379 = 0; _i379 < _list377.size; ++_i379)
+        org.apache.thrift.protocol.TList _list393 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partNames = new ArrayList<String>(_list393.size);
+        String _elem394;
+        for (int _i395 = 0; _i395 < _list393.size; ++_i395)
         {
-          _elem378 = iprot.readString();
-          struct.partNames.add(_elem378);
+          _elem394 = iprot.readString();
+          struct.partNames.add(_elem394);
         }
       }
       struct.setPartNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 2e903f1..51e05ae 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
@@ -363,26 +363,26 @@ public class PartitionsStatsResult implements org.apache.thrift.TBase<Partitions
           case 1: // PART_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map338 = iprot.readMapBegin();
-                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map338.size);
-                String _key339;
-                List<ColumnStatisticsObj> _val340;
-                for (int _i341 = 0; _i341 < _map338.size; ++_i341)
+                org.apache.thrift.protocol.TMap _map354 = iprot.readMapBegin();
+                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map354.size);
+                String _key355;
+                List<ColumnStatisticsObj> _val356;
+                for (int _i357 = 0; _i357 < _map354.size; ++_i357)
                 {
-                  _key339 = iprot.readString();
+                  _key355 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TList _list342 = iprot.readListBegin();
-                    _val340 = new ArrayList<ColumnStatisticsObj>(_list342.size);
-                    ColumnStatisticsObj _elem343;
-                    for (int _i344 = 0; _i344 < _list342.size; ++_i344)
+                    org.apache.thrift.protocol.TList _list358 = iprot.readListBegin();
+                    _val356 = new ArrayList<ColumnStatisticsObj>(_list358.size);
+                    ColumnStatisticsObj _elem359;
+                    for (int _i360 = 0; _i360 < _list358.size; ++_i360)
                     {
-                      _elem343 = new ColumnStatisticsObj();
-                      _elem343.read(iprot);
-                      _val340.add(_elem343);
+                      _elem359 = new ColumnStatisticsObj();
+                      _elem359.read(iprot);
+                      _val356.add(_elem359);
                     }
                     iprot.readListEnd();
                   }
-                  struct.partStats.put(_key339, _val340);
+                  struct.partStats.put(_key355, _val356);
                 }
                 iprot.readMapEnd();
               }
@@ -408,14 +408,14 @@ public class PartitionsStatsResult implements org.apache.thrift.TBase<Partitions
         oprot.writeFieldBegin(PART_STATS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.partStats.size()));
-          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter345 : struct.partStats.entrySet())
+          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter361 : struct.partStats.entrySet())
           {
-            oprot.writeString(_iter345.getKey());
+            oprot.writeString(_iter361.getKey());
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter345.getValue().size()));
-              for (ColumnStatisticsObj _iter346 : _iter345.getValue())
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter361.getValue().size()));
+              for (ColumnStatisticsObj _iter362 : _iter361.getValue())
               {
-                _iter346.write(oprot);
+                _iter362.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -443,14 +443,14 @@ public class PartitionsStatsResult implements org.apache.thrift.TBase<Partitions
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partStats.size());
-        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter347 : struct.partStats.entrySet())
+        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter363 : struct.partStats.entrySet())
         {
-          oprot.writeString(_iter347.getKey());
+          oprot.writeString(_iter363.getKey());
           {
-            oprot.writeI32(_iter347.getValue().size());
-            for (ColumnStatisticsObj _iter348 : _iter347.getValue())
+            oprot.writeI32(_iter363.getValue().size());
+            for (ColumnStatisticsObj _iter364 : _iter363.getValue())
             {
-              _iter348.write(oprot);
+              _iter364.write(oprot);
             }
           }
         }
@@ -461,25 +461,25 @@ public class PartitionsStatsResult implements org.apache.thrift.TBase<Partitions
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map349 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-        struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map349.size);
-        String _key350;
-        List<ColumnStatisticsObj> _val351;
-        for (int _i352 = 0; _i352 < _map349.size; ++_i352)
+        org.apache.thrift.protocol.TMap _map365 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+        struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map365.size);
+        String _key366;
+        List<ColumnStatisticsObj> _val367;
+        for (int _i368 = 0; _i368 < _map365.size; ++_i368)
         {
-          _key350 = iprot.readString();
+          _key366 = iprot.readString();
           {
-            org.apache.thrift.protocol.TList _list353 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            _val351 = new ArrayList<ColumnStatisticsObj>(_list353.size);
-            ColumnStatisticsObj _elem354;
-            for (int _i355 = 0; _i355 < _list353.size; ++_i355)
+            org.apache.thrift.protocol.TList _list369 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            _val367 = new ArrayList<ColumnStatisticsObj>(_list369.size);
+            ColumnStatisticsObj _elem370;
+            for (int _i371 = 0; _i371 < _list369.size; ++_i371)
             {
-              _elem354 = new ColumnStatisticsObj();
-              _elem354.read(iprot);
-              _val351.add(_elem354);
+              _elem370 = new ColumnStatisticsObj();
+              _elem370.read(iprot);
+              _val367.add(_elem370);
             }
           }
-          struct.partStats.put(_key350, _val351);
+          struct.partStats.put(_key366, _val367);
         }
       }
       struct.setPartStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java
new file mode 100644
index 0000000..8763ec8
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java
@@ -0,0 +1,490 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * 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.3)")
+public class PrimaryKeysRequest implements org.apache.thrift.TBase<PrimaryKeysRequest, PrimaryKeysRequest._Fields>, java.io.Serializable, Cloneable, Comparable<PrimaryKeysRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PrimaryKeysRequest");
+
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", 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 PrimaryKeysRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new PrimaryKeysRequestTupleSchemeFactory());
+  }
+
+  private String db_name; // required
+  private String tbl_name; // 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 {
+    DB_NAME((short)1, "db_name"),
+    TBL_NAME((short)2, "tbl_name");
+
+    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: // DB_NAME
+          return DB_NAME;
+        case 2: // TBL_NAME
+          return TBL_NAME;
+        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.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PrimaryKeysRequest.class, metaDataMap);
+  }
+
+  public PrimaryKeysRequest() {
+  }
+
+  public PrimaryKeysRequest(
+    String db_name,
+    String tbl_name)
+  {
+    this();
+    this.db_name = db_name;
+    this.tbl_name = tbl_name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public PrimaryKeysRequest(PrimaryKeysRequest other) {
+    if (other.isSetDb_name()) {
+      this.db_name = other.db_name;
+    }
+    if (other.isSetTbl_name()) {
+      this.tbl_name = other.tbl_name;
+    }
+  }
+
+  public PrimaryKeysRequest deepCopy() {
+    return new PrimaryKeysRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.db_name = null;
+    this.tbl_name = null;
+  }
+
+  public String getDb_name() {
+    return this.db_name;
+  }
+
+  public void setDb_name(String db_name) {
+    this.db_name = db_name;
+  }
+
+  public void unsetDb_name() {
+    this.db_name = null;
+  }
+
+  /** Returns true if field db_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetDb_name() {
+    return this.db_name != null;
+  }
+
+  public void setDb_nameIsSet(boolean value) {
+    if (!value) {
+      this.db_name = null;
+    }
+  }
+
+  public String getTbl_name() {
+    return this.tbl_name;
+  }
+
+  public void setTbl_name(String tbl_name) {
+    this.tbl_name = tbl_name;
+  }
+
+  public void unsetTbl_name() {
+    this.tbl_name = null;
+  }
+
+  /** Returns true if field tbl_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetTbl_name() {
+    return this.tbl_name != null;
+  }
+
+  public void setTbl_nameIsSet(boolean value) {
+    if (!value) {
+      this.tbl_name = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DB_NAME:
+      if (value == null) {
+        unsetDb_name();
+      } else {
+        setDb_name((String)value);
+      }
+      break;
+
+    case TBL_NAME:
+      if (value == null) {
+        unsetTbl_name();
+      } else {
+        setTbl_name((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDb_name();
+
+    case TBL_NAME:
+      return getTbl_name();
+
+    }
+    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 DB_NAME:
+      return isSetDb_name();
+    case TBL_NAME:
+      return isSetTbl_name();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof PrimaryKeysRequest)
+      return this.equals((PrimaryKeysRequest)that);
+    return false;
+  }
+
+  public boolean equals(PrimaryKeysRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_db_name = true && this.isSetDb_name();
+    boolean that_present_db_name = true && that.isSetDb_name();
+    if (this_present_db_name || that_present_db_name) {
+      if (!(this_present_db_name && that_present_db_name))
+        return false;
+      if (!this.db_name.equals(that.db_name))
+        return false;
+    }
+
+    boolean this_present_tbl_name = true && this.isSetTbl_name();
+    boolean that_present_tbl_name = true && that.isSetTbl_name();
+    if (this_present_tbl_name || that_present_tbl_name) {
+      if (!(this_present_tbl_name && that_present_tbl_name))
+        return false;
+      if (!this.tbl_name.equals(that.tbl_name))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_db_name = true && (isSetDb_name());
+    list.add(present_db_name);
+    if (present_db_name)
+      list.add(db_name);
+
+    boolean present_tbl_name = true && (isSetTbl_name());
+    list.add(present_tbl_name);
+    if (present_tbl_name)
+      list.add(tbl_name);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(PrimaryKeysRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDb_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, other.db_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTbl_name()).compareTo(other.isSetTbl_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTbl_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_name, other.tbl_name);
+      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("PrimaryKeysRequest(");
+    boolean first = true;
+
+    sb.append("db_name:");
+    if (this.db_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.db_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tbl_name:");
+    if (this.tbl_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tbl_name);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDb_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'db_name' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTbl_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tbl_name' 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 PrimaryKeysRequestStandardSchemeFactory implements SchemeFactory {
+    public PrimaryKeysRequestStandardScheme getScheme() {
+      return new PrimaryKeysRequestStandardScheme();
+    }
+  }
+
+  private static class PrimaryKeysRequestStandardScheme extends StandardScheme<PrimaryKeysRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, PrimaryKeysRequest 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: // DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.db_name = iprot.readString();
+              struct.setDb_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TBL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tbl_name = iprot.readString();
+              struct.setTbl_nameIsSet(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, PrimaryKeysRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.db_name != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.db_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tbl_name != null) {
+        oprot.writeFieldBegin(TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.tbl_name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class PrimaryKeysRequestTupleSchemeFactory implements SchemeFactory {
+    public PrimaryKeysRequestTupleScheme getScheme() {
+      return new PrimaryKeysRequestTupleScheme();
+    }
+  }
+
+  private static class PrimaryKeysRequestTupleScheme extends TupleScheme<PrimaryKeysRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, PrimaryKeysRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.db_name);
+      oprot.writeString(struct.tbl_name);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, PrimaryKeysRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.db_name = iprot.readString();
+      struct.setDb_nameIsSet(true);
+      struct.tbl_name = iprot.readString();
+      struct.setTbl_nameIsSet(true);
+    }
+  }
+
+}
+


[26/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 6e5de20..690c895 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         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 _size749;
+            ::apache::thrift::protocol::TType _etype752;
+            xfer += iprot->readListBegin(_etype752, _size749);
+            this->success.resize(_size749);
+            uint32_t _i753;
+            for (_i753 = 0; _i753 < _size749; ++_i753)
             {
-              xfer += iprot->readString(this->success[_i729]);
+              xfer += iprot->readString(this->success[_i753]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,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 _iter730;
-      for (_iter730 = this->success.begin(); _iter730 != this->success.end(); ++_iter730)
+      std::vector<std::string> ::const_iterator _iter754;
+      for (_iter754 = this->success.begin(); _iter754 != this->success.end(); ++_iter754)
       {
-        xfer += oprot->writeString((*_iter730));
+        xfer += oprot->writeString((*_iter754));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size731;
-            ::apache::thrift::protocol::TType _etype734;
-            xfer += iprot->readListBegin(_etype734, _size731);
-            (*(this->success)).resize(_size731);
-            uint32_t _i735;
-            for (_i735 = 0; _i735 < _size731; ++_i735)
+            uint32_t _size755;
+            ::apache::thrift::protocol::TType _etype758;
+            xfer += iprot->readListBegin(_etype758, _size755);
+            (*(this->success)).resize(_size755);
+            uint32_t _i759;
+            for (_i759 = 0; _i759 < _size755; ++_i759)
             {
-              xfer += iprot->readString((*(this->success))[_i735]);
+              xfer += iprot->readString((*(this->success))[_i759]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         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 _size760;
+            ::apache::thrift::protocol::TType _etype763;
+            xfer += iprot->readListBegin(_etype763, _size760);
+            this->success.resize(_size760);
+            uint32_t _i764;
+            for (_i764 = 0; _i764 < _size760; ++_i764)
             {
-              xfer += iprot->readString(this->success[_i740]);
+              xfer += iprot->readString(this->success[_i764]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,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 _iter741;
-      for (_iter741 = this->success.begin(); _iter741 != this->success.end(); ++_iter741)
+      std::vector<std::string> ::const_iterator _iter765;
+      for (_iter765 = this->success.begin(); _iter765 != this->success.end(); ++_iter765)
       {
-        xfer += oprot->writeString((*_iter741));
+        xfer += oprot->writeString((*_iter765));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size742;
-            ::apache::thrift::protocol::TType _etype745;
-            xfer += iprot->readListBegin(_etype745, _size742);
-            (*(this->success)).resize(_size742);
-            uint32_t _i746;
-            for (_i746 = 0; _i746 < _size742; ++_i746)
+            uint32_t _size766;
+            ::apache::thrift::protocol::TType _etype769;
+            xfer += iprot->readListBegin(_etype769, _size766);
+            (*(this->success)).resize(_size766);
+            uint32_t _i770;
+            for (_i770 = 0; _i770 < _size766; ++_i770)
             {
-              xfer += iprot->readString((*(this->success))[_i746]);
+              xfer += iprot->readString((*(this->success))[_i770]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size747;
-            ::apache::thrift::protocol::TType _ktype748;
-            ::apache::thrift::protocol::TType _vtype749;
-            xfer += iprot->readMapBegin(_ktype748, _vtype749, _size747);
-            uint32_t _i751;
-            for (_i751 = 0; _i751 < _size747; ++_i751)
+            uint32_t _size771;
+            ::apache::thrift::protocol::TType _ktype772;
+            ::apache::thrift::protocol::TType _vtype773;
+            xfer += iprot->readMapBegin(_ktype772, _vtype773, _size771);
+            uint32_t _i775;
+            for (_i775 = 0; _i775 < _size771; ++_i775)
             {
-              std::string _key752;
-              xfer += iprot->readString(_key752);
-              Type& _val753 = this->success[_key752];
-              xfer += _val753.read(iprot);
+              std::string _key776;
+              xfer += iprot->readString(_key776);
+              Type& _val777 = this->success[_key776];
+              xfer += _val777.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,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 _iter754;
-      for (_iter754 = this->success.begin(); _iter754 != this->success.end(); ++_iter754)
+      std::map<std::string, Type> ::const_iterator _iter778;
+      for (_iter778 = this->success.begin(); _iter778 != this->success.end(); ++_iter778)
       {
-        xfer += oprot->writeString(_iter754->first);
-        xfer += _iter754->second.write(oprot);
+        xfer += oprot->writeString(_iter778->first);
+        xfer += _iter778->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size755;
-            ::apache::thrift::protocol::TType _ktype756;
-            ::apache::thrift::protocol::TType _vtype757;
-            xfer += iprot->readMapBegin(_ktype756, _vtype757, _size755);
-            uint32_t _i759;
-            for (_i759 = 0; _i759 < _size755; ++_i759)
+            uint32_t _size779;
+            ::apache::thrift::protocol::TType _ktype780;
+            ::apache::thrift::protocol::TType _vtype781;
+            xfer += iprot->readMapBegin(_ktype780, _vtype781, _size779);
+            uint32_t _i783;
+            for (_i783 = 0; _i783 < _size779; ++_i783)
             {
-              std::string _key760;
-              xfer += iprot->readString(_key760);
-              Type& _val761 = (*(this->success))[_key760];
-              xfer += _val761.read(iprot);
+              std::string _key784;
+              xfer += iprot->readString(_key784);
+              Type& _val785 = (*(this->success))[_key784];
+              xfer += _val785.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            uint32_t _size786;
+            ::apache::thrift::protocol::TType _etype789;
+            xfer += iprot->readListBegin(_etype789, _size786);
+            this->success.resize(_size786);
+            uint32_t _i790;
+            for (_i790 = 0; _i790 < _size786; ++_i790)
             {
-              xfer += this->success[_i766].read(iprot);
+              xfer += this->success[_i790].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,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 _iter767;
-      for (_iter767 = this->success.begin(); _iter767 != this->success.end(); ++_iter767)
+      std::vector<FieldSchema> ::const_iterator _iter791;
+      for (_iter791 = this->success.begin(); _iter791 != this->success.end(); ++_iter791)
       {
-        xfer += (*_iter767).write(oprot);
+        xfer += (*_iter791).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size768;
-            ::apache::thrift::protocol::TType _etype771;
-            xfer += iprot->readListBegin(_etype771, _size768);
-            (*(this->success)).resize(_size768);
-            uint32_t _i772;
-            for (_i772 = 0; _i772 < _size768; ++_i772)
+            uint32_t _size792;
+            ::apache::thrift::protocol::TType _etype795;
+            xfer += iprot->readListBegin(_etype795, _size792);
+            (*(this->success)).resize(_size792);
+            uint32_t _i796;
+            for (_i796 = 0; _i796 < _size792; ++_i796)
             {
-              xfer += (*(this->success))[_i772].read(iprot);
+              xfer += (*(this->success))[_i796].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            uint32_t _size797;
+            ::apache::thrift::protocol::TType _etype800;
+            xfer += iprot->readListBegin(_etype800, _size797);
+            this->success.resize(_size797);
+            uint32_t _i801;
+            for (_i801 = 0; _i801 < _size797; ++_i801)
             {
-              xfer += this->success[_i777].read(iprot);
+              xfer += this->success[_i801].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,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 _iter778;
-      for (_iter778 = this->success.begin(); _iter778 != this->success.end(); ++_iter778)
+      std::vector<FieldSchema> ::const_iterator _iter802;
+      for (_iter802 = this->success.begin(); _iter802 != this->success.end(); ++_iter802)
       {
-        xfer += (*_iter778).write(oprot);
+        xfer += (*_iter802).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size779;
-            ::apache::thrift::protocol::TType _etype782;
-            xfer += iprot->readListBegin(_etype782, _size779);
-            (*(this->success)).resize(_size779);
-            uint32_t _i783;
-            for (_i783 = 0; _i783 < _size779; ++_i783)
+            uint32_t _size803;
+            ::apache::thrift::protocol::TType _etype806;
+            xfer += iprot->readListBegin(_etype806, _size803);
+            (*(this->success)).resize(_size803);
+            uint32_t _i807;
+            for (_i807 = 0; _i807 < _size803; ++_i807)
             {
-              xfer += (*(this->success))[_i783].read(iprot);
+              xfer += (*(this->success))[_i807].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            uint32_t _size808;
+            ::apache::thrift::protocol::TType _etype811;
+            xfer += iprot->readListBegin(_etype811, _size808);
+            this->success.resize(_size808);
+            uint32_t _i812;
+            for (_i812 = 0; _i812 < _size808; ++_i812)
             {
-              xfer += this->success[_i788].read(iprot);
+              xfer += this->success[_i812].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,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 _iter789;
-      for (_iter789 = this->success.begin(); _iter789 != this->success.end(); ++_iter789)
+      std::vector<FieldSchema> ::const_iterator _iter813;
+      for (_iter813 = this->success.begin(); _iter813 != this->success.end(); ++_iter813)
       {
-        xfer += (*_iter789).write(oprot);
+        xfer += (*_iter813).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size790;
-            ::apache::thrift::protocol::TType _etype793;
-            xfer += iprot->readListBegin(_etype793, _size790);
-            (*(this->success)).resize(_size790);
-            uint32_t _i794;
-            for (_i794 = 0; _i794 < _size790; ++_i794)
+            uint32_t _size814;
+            ::apache::thrift::protocol::TType _etype817;
+            xfer += iprot->readListBegin(_etype817, _size814);
+            (*(this->success)).resize(_size814);
+            uint32_t _i818;
+            for (_i818 = 0; _i818 < _size814; ++_i818)
             {
-              xfer += (*(this->success))[_i794].read(iprot);
+              xfer += (*(this->success))[_i818].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            uint32_t _size819;
+            ::apache::thrift::protocol::TType _etype822;
+            xfer += iprot->readListBegin(_etype822, _size819);
+            this->success.resize(_size819);
+            uint32_t _i823;
+            for (_i823 = 0; _i823 < _size819; ++_i823)
             {
-              xfer += this->success[_i799].read(iprot);
+              xfer += this->success[_i823].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,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 _iter800;
-      for (_iter800 = this->success.begin(); _iter800 != this->success.end(); ++_iter800)
+      std::vector<FieldSchema> ::const_iterator _iter824;
+      for (_iter824 = this->success.begin(); _iter824 != this->success.end(); ++_iter824)
       {
-        xfer += (*_iter800).write(oprot);
+        xfer += (*_iter824).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size801;
-            ::apache::thrift::protocol::TType _etype804;
-            xfer += iprot->readListBegin(_etype804, _size801);
-            (*(this->success)).resize(_size801);
-            uint32_t _i805;
-            for (_i805 = 0; _i805 < _size801; ++_i805)
+            uint32_t _size825;
+            ::apache::thrift::protocol::TType _etype828;
+            xfer += iprot->readListBegin(_etype828, _size825);
+            (*(this->success)).resize(_size825);
+            uint32_t _i829;
+            for (_i829 = 0; _i829 < _size825; ++_i829)
             {
-              xfer += (*(this->success))[_i805].read(iprot);
+              xfer += (*(this->success))[_i829].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4481,11 +4481,11 @@ uint32_t ThriftHiveMetastore_create_table_with_environment_context_presult::read
 }
 
 
-ThriftHiveMetastore_drop_table_args::~ThriftHiveMetastore_drop_table_args() throw() {
+ThriftHiveMetastore_create_table_with_constraints_args::~ThriftHiveMetastore_create_table_with_constraints_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -4507,25 +4507,49 @@ uint32_t ThriftHiveMetastore_drop_table_args::read(::apache::thrift::protocol::T
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->dbname);
-          this->__isset.dbname = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->tbl.read(iprot);
+          this->__isset.tbl = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->name);
-          this->__isset.name = true;
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->primaryKeys.clear();
+            uint32_t _size830;
+            ::apache::thrift::protocol::TType _etype833;
+            xfer += iprot->readListBegin(_etype833, _size830);
+            this->primaryKeys.resize(_size830);
+            uint32_t _i834;
+            for (_i834 = 0; _i834 < _size830; ++_i834)
+            {
+              xfer += this->primaryKeys[_i834].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.primaryKeys = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->deleteData);
-          this->__isset.deleteData = true;
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->foreignKeys.clear();
+            uint32_t _size835;
+            ::apache::thrift::protocol::TType _etype838;
+            xfer += iprot->readListBegin(_etype838, _size835);
+            this->foreignKeys.resize(_size835);
+            uint32_t _i839;
+            for (_i839 = 0; _i839 < _size835; ++_i839)
+            {
+              xfer += this->foreignKeys[_i839].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.foreignKeys = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -4542,21 +4566,37 @@ uint32_t ThriftHiveMetastore_drop_table_args::read(::apache::thrift::protocol::T
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_args");
 
-  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->dbname);
+  xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->tbl.write(oprot);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
+    std::vector<SQLPrimaryKey> ::const_iterator _iter840;
+    for (_iter840 = this->primaryKeys.begin(); _iter840 != this->primaryKeys.end(); ++_iter840)
+    {
+      xfer += (*_iter840).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3);
-  xfer += oprot->writeBool(this->deleteData);
+  xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
+    std::vector<SQLForeignKey> ::const_iterator _iter841;
+    for (_iter841 = this->foreignKeys.begin(); _iter841 != this->foreignKeys.end(); ++_iter841)
+    {
+      xfer += (*_iter841).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -4565,25 +4605,41 @@ uint32_t ThriftHiveMetastore_drop_table_args::write(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_drop_table_pargs::~ThriftHiveMetastore_drop_table_pargs() throw() {
+ThriftHiveMetastore_create_table_with_constraints_pargs::~ThriftHiveMetastore_create_table_with_constraints_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_pargs");
 
-  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->dbname)));
+  xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->tbl)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->name)));
+  xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
+    std::vector<SQLPrimaryKey> ::const_iterator _iter842;
+    for (_iter842 = (*(this->primaryKeys)).begin(); _iter842 != (*(this->primaryKeys)).end(); ++_iter842)
+    {
+      xfer += (*_iter842).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3);
-  xfer += oprot->writeBool((*(this->deleteData)));
+  xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
+    std::vector<SQLForeignKey> ::const_iterator _iter843;
+    for (_iter843 = (*(this->foreignKeys)).begin(); _iter843 != (*(this->foreignKeys)).end(); ++_iter843)
+    {
+      xfer += (*_iter843).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -4592,11 +4648,11 @@ uint32_t ThriftHiveMetastore_drop_table_pargs::write(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_drop_table_result::~ThriftHiveMetastore_drop_table_result() throw() {
+ThriftHiveMetastore_create_table_with_constraints_result::~ThriftHiveMetastore_create_table_with_constraints_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_table_with_constraints_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -4627,12 +4683,28 @@ uint32_t ThriftHiveMetastore_drop_table_result::read(::apache::thrift::protocol:
         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;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o3.read(iprot);
           this->__isset.o3 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o4.read(iprot);
+          this->__isset.o4 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4645,20 +4717,28 @@ uint32_t ThriftHiveMetastore_drop_table_result::read(::apache::thrift::protocol:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_table_with_constraints_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_result");
 
   if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o2) {
+    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += this->o2.write(oprot);
+    xfer += oprot->writeFieldEnd();
   } else if (this->__isset.o3) {
-    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
     xfer += this->o3.write(oprot);
     xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o4) {
+    xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4);
+    xfer += this->o4.write(oprot);
+    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -4666,11 +4746,11 @@ uint32_t ThriftHiveMetastore_drop_table_result::write(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_drop_table_presult::~ThriftHiveMetastore_drop_table_presult() throw() {
+ThriftHiveMetastore_create_table_with_constraints_presult::~ThriftHiveMetastore_create_table_with_constraints_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_table_with_constraints_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -4701,12 +4781,28 @@ uint32_t ThriftHiveMetastore_drop_table_presult::read(::apache::thrift::protocol
         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;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o3.read(iprot);
           this->__isset.o3 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o4.read(iprot);
+          this->__isset.o4 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4720,11 +4816,11 @@ uint32_t ThriftHiveMetastore_drop_table_presult::read(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_drop_table_with_environment_context_args::~ThriftHiveMetastore_drop_table_with_environment_context_args() throw() {
+ThriftHiveMetastore_drop_table_args::~ThriftHiveMetastore_drop_table_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -4769,14 +4865,6 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::read(::ap
           xfer += iprot->skip(ftype);
         }
         break;
-      case 4:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->environment_context.read(iprot);
-          this->__isset.environment_context = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4789,10 +4877,10 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::read(::ap
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_args");
 
   xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->dbname);
@@ -4806,24 +4894,20 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::write(::a
   xfer += oprot->writeBool(this->deleteData);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4);
-  xfer += this->environment_context.write(oprot);
-  xfer += oprot->writeFieldEnd();
-
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_drop_table_with_environment_context_pargs::~ThriftHiveMetastore_drop_table_with_environment_context_pargs() throw() {
+ThriftHiveMetastore_drop_table_pargs::~ThriftHiveMetastore_drop_table_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_pargs");
 
   xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString((*(this->dbname)));
@@ -4837,21 +4921,17 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_pargs::write(::
   xfer += oprot->writeBool((*(this->deleteData)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4);
-  xfer += (*(this->environment_context)).write(oprot);
-  xfer += oprot->writeFieldEnd();
-
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_drop_table_with_environment_context_result::~ThriftHiveMetastore_drop_table_with_environment_context_result() throw() {
+ThriftHiveMetastore_drop_table_result::~ThriftHiveMetastore_drop_table_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -4900,11 +4980,11 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::read(::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_result");
 
   if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
@@ -4921,11 +5001,11 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::write(:
 }
 
 
-ThriftHiveMetastore_drop_table_with_environment_context_presult::~ThriftHiveMetastore_drop_table_with_environment_context_presult() throw() {
+ThriftHiveMetastore_drop_table_presult::~ThriftHiveMetastore_drop_table_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -4975,11 +5055,11 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_presult::read(:
 }
 
 
-ThriftHiveMetastore_get_tables_args::~ThriftHiveMetastore_get_tables_args() throw() {
+ThriftHiveMetastore_drop_table_with_environment_context_args::~ThriftHiveMetastore_drop_table_with_environment_context_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5002,16 +5082,32 @@ uint32_t ThriftHiveMetastore_get_tables_args::read(::apache::thrift::protocol::T
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->db_name);
-          this->__isset.db_name = true;
+          xfer += iprot->readString(this->dbname);
+          this->__isset.dbname = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->pattern);
-          this->__isset.pattern = true;
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->deleteData);
+          this->__isset.deleteData = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->environment_context.read(iprot);
+          this->__isset.environment_context = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -5028,17 +5124,25 @@ uint32_t ThriftHiveMetastore_get_tables_args::read(::apache::thrift::protocol::T
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_tables_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_args");
 
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->db_name);
+  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbname);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->pattern);
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3);
+  xfer += oprot->writeBool(this->deleteData);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4);
+  xfer += this->environment_context.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -5047,21 +5151,29 @@ uint32_t ThriftHiveMetastore_get_tables_args::write(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_get_tables_pargs::~ThriftHiveMetastore_get_tables_pargs() throw() {
+ThriftHiveMetastore_drop_table_with_environment_context_pargs::~ThriftHiveMetastore_drop_table_with_environment_context_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_pargs");
 
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->db_name)));
+  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->dbname)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->pattern)));
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->name)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3);
+  xfer += oprot->writeBool((*(this->deleteData)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4);
+  xfer += (*(this->environment_context)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -5070,11 +5182,234 @@ uint32_t ThriftHiveMetastore_get_tables_pargs::write(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_get_tables_result::~ThriftHiveMetastore_get_tables_result() throw() {
+ThriftHiveMetastore_drop_table_with_environment_context_result::~ThriftHiveMetastore_drop_table_with_environment_context_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*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_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->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_result");
+
+  if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o3) {
+    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += this->o3.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_drop_table_with_environment_context_presult::~ThriftHiveMetastore_drop_table_with_environment_context_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*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_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->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_args::~ThriftHiveMetastore_get_tables_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*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->pattern);
+          this->__isset.pattern = 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_tables_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_args");
+
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->db_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->pattern);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_pargs::~ThriftHiveMetastore_get_tables_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_pargs");
+
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->db_name)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->pattern)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_result::~ThriftHiveMetastore_get_tables_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5099,14 +5434,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            uint32_t _size844;
+            ::apache::thrift::protocol::TType _etype847;
+            xfer += iprot->readListBegin(_etype847, _size844);
+            this->success.resize(_size844);
+            uint32_t _i848;
+            for (_i848 = 0; _i848 < _size844; ++_i848)
             {
-              xfer += iprot->readString(this->success[_i810]);
+              xfer += iprot->readString(this->success[_i848]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5145,10 +5480,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 _iter811;
-      for (_iter811 = this->success.begin(); _iter811 != this->success.end(); ++_iter811)
+      std::vector<std::string> ::const_iterator _iter849;
+      for (_iter849 = this->success.begin(); _iter849 != this->success.end(); ++_iter849)
       {
-        xfer += oprot->writeString((*_iter811));
+        xfer += oprot->writeString((*_iter849));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5193,14 +5528,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size812;
-            ::apache::thrift::protocol::TType _etype815;
-            xfer += iprot->readListBegin(_etype815, _size812);
-            (*(this->success)).resize(_size812);
-            uint32_t _i816;
-            for (_i816 = 0; _i816 < _size812; ++_i816)
+            uint32_t _size850;
+            ::apache::thrift::protocol::TType _etype853;
+            xfer += iprot->readListBegin(_etype853, _size850);
+            (*(this->success)).resize(_size850);
+            uint32_t _i854;
+            for (_i854 = 0; _i854 < _size850; ++_i854)
             {
-              xfer += iprot->readString((*(this->success))[_i816]);
+              xfer += iprot->readString((*(this->success))[_i854]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5275,14 +5610,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size817;
-            ::apache::thrift::protocol::TType _etype820;
-            xfer += iprot->readListBegin(_etype820, _size817);
-            this->tbl_types.resize(_size817);
-            uint32_t _i821;
-            for (_i821 = 0; _i821 < _size817; ++_i821)
+            uint32_t _size855;
+            ::apache::thrift::protocol::TType _etype858;
+            xfer += iprot->readListBegin(_etype858, _size855);
+            this->tbl_types.resize(_size855);
+            uint32_t _i859;
+            for (_i859 = 0; _i859 < _size855; ++_i859)
             {
-              xfer += iprot->readString(this->tbl_types[_i821]);
+              xfer += iprot->readString(this->tbl_types[_i859]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5319,10 +5654,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter822;
-    for (_iter822 = this->tbl_types.begin(); _iter822 != this->tbl_types.end(); ++_iter822)
+    std::vector<std::string> ::const_iterator _iter860;
+    for (_iter860 = this->tbl_types.begin(); _iter860 != this->tbl_types.end(); ++_iter860)
     {
-      xfer += oprot->writeString((*_iter822));
+      xfer += oprot->writeString((*_iter860));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5354,10 +5689,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter823;
-    for (_iter823 = (*(this->tbl_types)).begin(); _iter823 != (*(this->tbl_types)).end(); ++_iter823)
+    std::vector<std::string> ::const_iterator _iter861;
+    for (_iter861 = (*(this->tbl_types)).begin(); _iter861 != (*(this->tbl_types)).end(); ++_iter861)
     {
-      xfer += oprot->writeString((*_iter823));
+      xfer += oprot->writeString((*_iter861));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5398,14 +5733,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size824;
-            ::apache::thrift::protocol::TType _etype827;
-            xfer += iprot->readListBegin(_etype827, _size824);
-            this->success.resize(_size824);
-            uint32_t _i828;
-            for (_i828 = 0; _i828 < _size824; ++_i828)
+            uint32_t _size862;
+            ::apache::thrift::protocol::TType _etype865;
+            xfer += iprot->readListBegin(_etype865, _size862);
+            this->success.resize(_size862);
+            uint32_t _i866;
+            for (_i866 = 0; _i866 < _size862; ++_i866)
             {
-              xfer += this->success[_i828].read(iprot);
+              xfer += this->success[_i866].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5444,10 +5779,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_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<TableMeta> ::const_iterator _iter829;
-      for (_iter829 = this->success.begin(); _iter829 != this->success.end(); ++_iter829)
+      std::vector<TableMeta> ::const_iterator _iter867;
+      for (_iter867 = this->success.begin(); _iter867 != this->success.end(); ++_iter867)
       {
-        xfer += (*_iter829).write(oprot);
+        xfer += (*_iter867).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5492,14 +5827,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size830;
-            ::apache::thrift::protocol::TType _etype833;
-            xfer += iprot->readListBegin(_etype833, _size830);
-            (*(this->success)).resize(_size830);
-            uint32_t _i834;
-            for (_i834 = 0; _i834 < _size830; ++_i834)
+            uint32_t _size868;
+            ::apache::thrift::protocol::TType _etype871;
+            xfer += iprot->readListBegin(_etype871, _size868);
+            (*(this->success)).resize(_size868);
+            uint32_t _i872;
+            for (_i872 = 0; _i872 < _size868; ++_i872)
             {
-              xfer += (*(this->success))[_i834].read(iprot);
+              xfer += (*(this->success))[_i872].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5637,14 +5972,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            uint32_t _size873;
+            ::apache::thrift::protocol::TType _etype876;
+            xfer += iprot->readListBegin(_etype876, _size873);
+            this->success.resize(_size873);
+            uint32_t _i877;
+            for (_i877 = 0; _i877 < _size873; ++_i877)
             {
-              xfer += iprot->readString(this->success[_i839]);
+              xfer += iprot->readString(this->success[_i877]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5683,10 +6018,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 _iter840;
-      for (_iter840 = this->success.begin(); _iter840 != this->success.end(); ++_iter840)
+      std::vector<std::string> ::const_iterator _iter878;
+      for (_iter878 = this->success.begin(); _iter878 != this->success.end(); ++_iter878)
       {
-        xfer += oprot->writeString((*_iter840));
+        xfer += oprot->writeString((*_iter878));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5731,14 +6066,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size841;
-            ::apache::thrift::protocol::TType _etype844;
-            xfer += iprot->readListBegin(_etype844, _size841);
-            (*(this->success)).resize(_size841);
-            uint32_t _i845;
-            for (_i845 = 0; _i845 < _size841; ++_i845)
+            uint32_t _size879;
+            ::apache::thrift::protocol::TType _etype882;
+            xfer += iprot->readListBegin(_etype882, _size879);
+            (*(this->success)).resize(_size879);
+            uint32_t _i883;
+            for (_i883 = 0; _i883 < _size879; ++_i883)
             {
-              xfer += iprot->readString((*(this->success))[_i845]);
+              xfer += iprot->readString((*(this->success))[_i883]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6048,14 +6383,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 _size846;
-            ::apache::thrift::protocol::TType _etype849;
-            xfer += iprot->readListBegin(_etype849, _size846);
-            this->tbl_names.resize(_size846);
-            uint32_t _i850;
-            for (_i850 = 0; _i850 < _size846; ++_i850)
+            uint32_t _size884;
+            ::apache::thrift::protocol::TType _etype887;
+            xfer += iprot->readListBegin(_etype887, _size884);
+            this->tbl_names.resize(_size884);
+            uint32_t _i888;
+            for (_i888 = 0; _i888 < _size884; ++_i888)
             {
-              xfer += iprot->readString(this->tbl_names[_i850]);
+              xfer += iprot->readString(this->tbl_names[_i888]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6088,10 +6423,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 _iter851;
-    for (_iter851 = this->tbl_names.begin(); _iter851 != this->tbl_names.end(); ++_iter851)
+    std::vector<std::string> ::const_iterator _iter889;
+    for (_iter889 = this->tbl_names.begin(); _iter889 != this->tbl_names.end(); ++_iter889)
     {
-      xfer += oprot->writeString((*_iter851));
+      xfer += oprot->writeString((*_iter889));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6119,10 +6454,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 _iter852;
-    for (_iter852 = (*(this->tbl_names)).begin(); _iter852 != (*(this->tbl_names)).end(); ++_iter852)
+    std::vector<std::string> ::const_iterator _iter890;
+    for (_iter890 = (*(this->tbl_names)).begin(); _iter890 != (*(this->tbl_names)).end(); ++_iter890)
     {
-      xfer += oprot->writeString((*_iter852));
+      xfer += oprot->writeString((*_iter890));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6163,14 +6498,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 _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 _size891;
+            ::apache::thrift::protocol::TType _etype894;
+            xfer += iprot->readListBegin(_etype894, _size891);
+            this->success.resize(_size891);
+            uint32_t _i895;
+            for (_i895 = 0; _i895 < _size891; ++_i895)
             {
-              xfer += this->success[_i857].read(iprot);
+              xfer += this->success[_i895].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6225,10 +6560,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 _iter858;
-      for (_iter858 = this->success.begin(); _iter858 != this->success.end(); ++_iter858)
+      std::vector<Table> ::const_iterator _iter896;
+      for (_iter896 = this->success.begin(); _iter896 != this->success.end(); ++_iter896)
       {
-        xfer += (*_iter858).write(oprot);
+        xfer += (*_iter896).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6281,14 +6616,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 _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 _size897;
+            ::apache::thrift::protocol::TType _etype900;
+            xfer += iprot->readListBegin(_etype900, _size897);
+            (*(this->success)).resize(_size897);
+            uint32_t _i901;
+            for (_i901 = 0; _i901 < _size897; ++_i901)
             {
-              xfer += (*(this->success))[_i863].read(iprot);
+              xfer += (*(this->success))[_i901].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6474,14 +6809,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 _size864;
-            ::apache::thrift::protocol::TType _etype867;
-            xfer += iprot->readListBegin(_etype867, _size864);
-            this->success.resize(_size864);
-            uint32_t _i868;
-            for (_i868 = 0; _i868 < _size864; ++_i868)
+            uint32_t _size902;
+            ::apache::thrift::protocol::TType _etype905;
+            xfer += iprot->readListBegin(_etype905, _size902);
+            this->success.resize(_size902);
+            uint32_t _i906;
+            for (_i906 = 0; _i906 < _size902; ++_i906)
             {
-              xfer += iprot->readString(this->success[_i868]);
+              xfer += iprot->readString(this->success[_i906]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6536,10 +6871,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 _iter869;
-      for (_iter869 = this->success.begin(); _iter869 != this->success.end(); ++_iter869)
+      std::vector<std::string> ::const_iterator _iter907;
+      for (_iter907 = this->success.begin(); _iter907 != this->success.end(); ++_iter907)
       {
-        xfer += oprot->writeString((*_iter869));
+        xfer += oprot->writeString((*_iter907));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6592,14 +6927,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 _size870;
-            ::apache::thrift::protocol::TType _etype873;
-            xfer += iprot->readListBegin(_etype873, _size870);
-            (*(this->success)).resize(_size870);
-            uint32_t _i874;
-            for (_i874 = 0; _i874 < _size870; ++_i874)
+            uint32_t _size908;
+            ::apache::thrift::protocol::TType _etype911;
+            xfer += iprot->readListBegin(_etype911, _size908);
+            (*(this->success)).resize(_size908);
+            uint32_t _i912;
+            for (_i912 = 0; _i912 < _size908; ++_i912)
             {
-              xfer += iprot->readString((*(this->success))[_i874]);
+              xfer += iprot->readString((*(this->success))[_i912]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7933,14 +8268,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size875;
-            ::apache::thrift::protocol::TType _etype878;
-            xfer += iprot->readListBegin(_etype878, _size875);
-            this->new_parts.resize(_size875);
-            uint32_t _i879;
-            for (_i879 = 0; _i879 < _size875; ++_i879)
+            uint32_t _size913;
+            ::apache::thrift::protocol::TType _etype916;
+            xfer += iprot->readListBegin(_etype916, _size913);
+            this->new_parts.resize(_size913);
+            uint32_t _i917;
+            for (_i917 = 0; _i917 < _size913; ++_i917)
             {
-              xfer += this->new_parts[_i879].read(iprot);
+              xfer += this->new_parts[_i917].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7969,10 +8304,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 _iter880;
-    for (_iter880 = this->new_parts.begin(); _iter880 != this->new_parts.end(); ++_iter880)
+    std::vector<Partition> ::const_iterator _iter918;
+    for (_iter918 = this->new_parts.begin(); _iter918 != this->new_parts.end(); ++_iter918)
     {
-      xfer += (*_iter880).write(oprot);
+      xfer += (*_iter918).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7996,10 +8331,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 _iter881;
-    for (_iter881 = (*(this->new_parts)).begin(); _iter881 != (*(this->new_parts)).end(); ++_iter881)
+    std::vector<Partition> ::const_iterator _iter919;
+    for (_iter919 = (*(this->new_parts)).begin(); _iter919 != (*(this->new_parts)).end(); ++_iter919)
     {
-      xfer += (*_iter881).write(oprot);
+      xfer += (*_iter919).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8208,14 +8543,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 _size882;
-            ::apache::thrift::protocol::TType _etype885;
-            xfer += iprot->readListBegin(_etype885, _size882);
-            this->new_parts.resize(_size882);
-            uint32_t _i886;
-            for (_i886 = 0; _i886 < _size882; ++_i886)
+            uint32_t _size920;
+            ::apache::thrift::protocol::TType _etype923;
+            xfer += iprot->readListBegin(_etype923, _size920);
+            this->new_parts.resize(_size920);
+            uint32_t _i924;
+            for (_i924 = 0; _i924 < _size920; ++_i924)
             {
-              xfer += this->new_parts[_i886].read(iprot);
+              xfer += this->new_parts[_i924].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8244,10 +8579,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 _iter887;
-    for (_iter887 = this->new_parts.begin(); _iter887 != this->new_parts.end(); ++_iter887)
+    std::vector<PartitionSpec> ::const_iterator _iter925;
+    for (_iter925 = this->new_parts.begin(); _iter925 != this->new_parts.end(); ++_iter925)
     {
-      xfer += (*_iter887).write(oprot);
+      xfer += (*_iter925).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8271,10 +8606,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 _iter888;
-    for (_iter888 = (*(this->new_parts)).begin(); _iter888 != (*(this->new_parts)).end(); ++_iter888)
+    std::vector<PartitionSpec> ::const_iterator _iter926;
+    for (_iter926 = (*(this->new_parts)).begin(); _iter926 != (*(this->new_parts)).end(); ++_iter926)
     {
-      xfer += (*_iter888).write(oprot);
+      xfer += (*_iter926).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8499,14 +8834,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size889;
-            ::apache::thrift::protocol::TType _etype892;
-            xfer += iprot->readListBegin(_etype892, _size889);
-            this->part_vals.resize(_size889);
-            uint32_t _i893;
-            for (_i893 = 0; _i893 < _size889; ++_i893)
+            uint32_t _size927;
+            ::apache::thrift::protocol::TType _etype930;
+            xfer += iprot->readListBegin(_etype930, _size927);
+            this->part_vals.resize(_size927);
+            uint32_t _i931;
+            for (_i931 = 0; _i931 < _size927; ++_i931)
             {
-              xfer += iprot->readString(this->part_vals[_i893]);
+              xfer += iprot->readString(this->part_vals[_i931]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8543,10 +8878,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 _iter894;
-    for (_iter894 = this->part_vals.begin(); _iter894 != this->part_vals.end(); ++_iter894)
+    std::vector<std::string> ::const_iterator _iter932;
+    for (_iter932 = this->part_vals.begin(); _iter932 != this->part_vals.end(); ++_iter932)
     {
-      xfer += oprot->writeString((*_iter894));
+      xfer += oprot->writeString((*_iter932));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8578,10 +8913,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 _iter895;
-    for (_iter895 = (*(this->part_vals)).begin(); _iter895 != (*(this->part_vals)).end(); ++_iter895)
+    std::vector<std::string> ::const_iterator _iter933;
+    for (_iter933 = (*(this->part_vals)).begin(); _iter933 != (*(this->part_vals)).end(); ++_iter933)
     {
-      xfer += oprot->writeString((*_iter895));
+      xfer += oprot->writeString((*_iter933));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9053,14 +9388,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size896;
-            ::apache::thrift::protocol::TType _etype899;
-            xfer += iprot->readListBegin(_etype899, _size896);
-            this->part_vals.resize(_size896);
-            uint32_t _i900;
-            for (_i900 = 0; _i900 < _size896; ++_i900)
+            uint32_t _size934;
+            ::apache::thrift::protocol::TType _etype937;
+            xfer += iprot->readListBegin(_etype937, _size934);
+            this->part_vals.resize(_size934);
+            uint32_t _i938;
+            for (_i938 = 0; _i938 < _size934; ++_i938)
             {
-              xfer += iprot->readString(this->part_vals[_i900]);
+              xfer += iprot->readString(this->part_vals[_i938]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9105,10 +9440,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 _iter901;
-    for (_iter901 = this->part_vals.begin(); _iter901 != this->part_vals.end(); ++_iter901)
+    std::vector<std::string> ::const_iterator _iter939;
+    for (_iter939 = this->part_vals.begin(); _iter939 != this->part_vals.end(); ++_iter939)
     {
-      xfer += oprot->writeString((*_iter901));
+      xfer += oprot->writeString((*_iter939));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9144,10 +9479,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 _iter902;
-    for (_iter902 = (*(this->part_vals)).begin(); _iter902 != (*(this->part_vals)).end(); ++_iter902)
+    std::vector<std::string> ::const_iterator _iter940;
+    for (_iter940 = (*(this->part_vals)).begin(); _iter940 != (*(this->part_vals)).end(); ++_iter940)
     {
-      xfer += oprot->writeString((*_iter902));
+      xfer += oprot->writeString((*_iter940));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9950,14 +10285,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size903;
-            ::apache::thrift::protocol::TType _etype906;
-            xfer += iprot->readListBegin(_etype906, _size903);
-            this->part_vals.resize(_size903);
-            uint32_t _i907;
-            for (_i907 = 0; _i907 < _size903; ++_i907)
+            uint32_t _size941;
+            ::apache::thrift::protocol::TType _etype944;
+            xfer += iprot->readListBegin(_etype944, _size941);
+            this->part_vals.resize(_size941);
+            uint32_t _i945;
+            for (_i945 = 0; _i945 < _size941; ++_i945)
             {
-              xfer += iprot->readString(this->part_vals[_i907]);
+              xfer += iprot->readString(this->part_vals[_i945]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10002,10 +10337,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 _iter908;
-    for (_iter908 = this->part_vals.begin(); _iter908 != this->part_vals.end(); ++_iter908)
+    std::vector<std::string> ::const_iterator _iter946;
+    for (_iter946 = this->part_vals.begin(); _iter946 != this->part_vals.end(); ++_iter946)
     {
-      xfer += oprot->writeString((*_iter908));
+      xfer += oprot->writeString((*_iter946));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10041,10 +10376,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 _iter909;
-    for (_iter909 = (*(this->part_vals)).begin(); _iter909 != (*(this->part_vals)).end(); ++_iter909)
+    std::vector<std::string> ::const_iterator _iter947;
+    for (_iter947 = (*(this->part_vals)).begin(); _iter947 != (*(this->part_vals)).end(); ++_iter947)
     {
-      xfer += oprot->writeString((*_iter909));
+      xfer += oprot->writeString((*_iter947));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10253,14 +10588,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size910;
-            ::apache::thrift::protocol::TType _etype913;
-            xfer += iprot->readListBegin(_etype913, _size910);
-            this->part_vals.resize(_size910);
-            uint32_t _i914;
-            for (_i914 = 0; _i914 < _size910; ++_i914)
+            uint32_t _size948;
+            ::apache::thrift::protocol::TType _etype951;
+            xfer += iprot->readListBegin(_etype951, _size948);
+            this->part_vals.resize(_size948);
+            uint32_t _i952;
+            for (_i952 = 0; _i952 < _size948; ++_i952)
             {
-              xfer += iprot->readString(this->part_vals[_i914]);
+              xfer += iprot->readString(this->part_vals[_i952]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10313,10 +10648,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 _iter915;
-    for (_iter915 = this->part_vals.begin(); _iter915 != this->part_vals.end(); ++_iter915)
+    std::vector<std::string> ::const_iterator _iter953;
+    for (_iter953 = this->part_vals.begin(); _iter953 != this->part_vals.end(); ++_iter953)
     {
-      xfer += oprot->writeString((*_iter915));
+      xfer += oprot->writeString((*_iter953));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10356,10 +10691,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 _iter916;
-    for (_iter916 = (*(this->part_vals)).begin(); _iter916 != (*(this->part_vals)).end(); ++_iter916)
+    std::vector<std::string> ::const_iterator _iter954;
+    for (_iter954 = (*(this->part_vals)).begin(); _iter954 != (*(this->part_vals)).end(); ++_iter954)
     {
-      xfer += oprot->writeString((*_iter916));
+      xfer += oprot->writeString((*_iter954));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11365,14 +11700,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size917;
-            ::apache::thrift::protocol::TType _etype920;
-            xfer += iprot->readListBegin(_etype920, _size917);
-            this->part_vals.resize(_size917);
-            uint32_t _i921;
-            for (_i921 = 0; _i921 < _size917; ++_i921)
+            uint32_t _size955;
+            ::apache::thrift::protocol::TType _etype958;
+            xfer += iprot->readListBegin(_etype958, _size955);
+            this->part_vals.resize(_size955);
+            uint32_t _i959;
+            for (_i959 = 0; _i959 < _size955; ++_i959)
             {
-              xfer += iprot->readString(this->part_vals[_i921]);
+              xfer += iprot->readString(this->part_vals[_i959]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11409,10 +11744,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 _iter922;
-    for (_iter922 = this->part_vals.begin(); _iter922 != this->part_vals.end(); ++_iter922)
+    std::vector<std::string> ::const_iterator _iter960;
+    for (_iter960 = this->part_vals.begin(); _iter960 != this->part_vals.end(); ++_iter960)
     {
-      xfer += oprot->writeString((*_iter922));
+      xfer += oprot->writeString((*_iter960));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11444,10 +11779,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 _iter923;
-    for (_iter923 = (*(this->part_vals)).begin(); _iter923 != (*(this->part_vals)).end(); ++_iter923)
+    std::vector<std::string> ::const_iterator _iter961;
+    for (_iter961 = (*(this->part_vals)).begin(); _iter961 != (*(this->part_vals)).end(); ++_iter961)
     {
-      xfer += oprot->writeString((*_iter923));
+      xfer += oprot->writeString((*_iter961));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11636,17 +11971,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size924;
-            ::apache::thrift::protocol::TType _ktype925;
-            ::apache::thrift::protocol::TType _vtype926;
-            xfer += iprot->readMapBegin(_ktype925, _vtype926, _size924);
-            uint32_t _i928;
-            for (_i928 = 0; _i928 < _size924; ++_i928)
+            uint32_t _size962;
+            ::apache::thrift::protocol::TType _ktype963;
+            ::apache::thrift::protocol::TType _vtype964;
+            xfer += iprot->readMapBegin(_ktype963, _vtype964, _size962);
+            uint32_t _i966;
+            for (_i966 = 0; _i966 < _size962; ++_i966)
             {
-              std::string _key929;
-              xfer += iprot->readString(_key929);
-              std::string& _val930 = this->partitionSpecs[_key929];
-              xfer += iprot->readString(_val930);
+              std::string _key967;
+              xfer += iprot->readString(_key967);
+              std::string& _val968 = this->partitionSpecs[_key967];
+              xfer += iprot->readString(_val968);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11707,11 +12042,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

<TRUNCATED>

[18/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 05a0749..4f0c8fd 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -157,6 +157,16 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function create_table_with_environment_context(\metastore\Table $tbl, \metastore\EnvironmentContext $environment_context);
   /**
+   * @param \metastore\Table $tbl
+   * @param \metastore\SQLPrimaryKey[] $primaryKeys
+   * @param \metastore\SQLForeignKey[] $foreignKeys
+   * @throws \metastore\AlreadyExistsException
+   * @throws \metastore\InvalidObjectException
+   * @throws \metastore\MetaException
+   * @throws \metastore\NoSuchObjectException
+   */
+  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys);
+  /**
    * @param string $dbname
    * @param string $name
    * @param bool $deleteData
@@ -699,6 +709,20 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_index_names($db_name, $tbl_name, $max_indexes);
   /**
+   * @param \metastore\PrimaryKeysRequest $request
+   * @return \metastore\PrimaryKeysResponse
+   * @throws \metastore\MetaException
+   * @throws \metastore\NoSuchObjectException
+   */
+  public function get_primary_keys(\metastore\PrimaryKeysRequest $request);
+  /**
+   * @param \metastore\ForeignKeysRequest $request
+   * @return \metastore\ForeignKeysResponse
+   * @throws \metastore\MetaException
+   * @throws \metastore\NoSuchObjectException
+   */
+  public function get_foreign_keys(\metastore\ForeignKeysRequest $request);
+  /**
    * @param \metastore\ColumnStatistics $stats_obj
    * @return bool
    * @throws \metastore\NoSuchObjectException
@@ -2164,6 +2188,68 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys)
+  {
+    $this->send_create_table_with_constraints($tbl, $primaryKeys, $foreignKeys);
+    $this->recv_create_table_with_constraints();
+  }
+
+  public function send_create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys)
+  {
+    $args = new \metastore\ThriftHiveMetastore_create_table_with_constraints_args();
+    $args->tbl = $tbl;
+    $args->primaryKeys = $primaryKeys;
+    $args->foreignKeys = $foreignKeys;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'create_table_with_constraints', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('create_table_with_constraints', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_create_table_with_constraints()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_create_table_with_constraints_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_create_table_with_constraints_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    if ($result->o3 !== null) {
+      throw $result->o3;
+    }
+    if ($result->o4 !== null) {
+      throw $result->o4;
+    }
+    return;
+  }
+
   public function drop_table($dbname, $name, $deleteData)
   {
     $this->send_drop_table($dbname, $name, $deleteData);
@@ -5662,6 +5748,120 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_index_names failed: unknown result");
   }
 
+  public function get_primary_keys(\metastore\PrimaryKeysRequest $request)
+  {
+    $this->send_get_primary_keys($request);
+    return $this->recv_get_primary_keys();
+  }
+
+  public function send_get_primary_keys(\metastore\PrimaryKeysRequest $request)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_primary_keys_args();
+    $args->request = $request;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_primary_keys', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_primary_keys', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_primary_keys()
+  {
+    $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_primary_keys_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_primary_keys_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_primary_keys failed: unknown result");
+  }
+
+  public function get_foreign_keys(\metastore\ForeignKeysRequest $request)
+  {
+    $this->send_get_foreign_keys($request);
+    return $this->recv_get_foreign_keys();
+  }
+
+  public function send_get_foreign_keys(\metastore\ForeignKeysRequest $request)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_foreign_keys_args();
+    $args->request = $request;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_foreign_keys', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_foreign_keys', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_foreign_keys()
+  {
+    $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_foreign_keys_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_foreign_keys_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_foreign_keys failed: unknown result");
+  }
+
   public function update_table_column_statistics(\metastore\ColumnStatistics $stats_obj)
   {
     $this->send_update_table_column_statistics($stats_obj);
@@ -10412,14 +10612,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size525 = 0;
-            $_etype528 = 0;
-            $xfer += $input->readListBegin($_etype528, $_size525);
-            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
+            $_size539 = 0;
+            $_etype542 = 0;
+            $xfer += $input->readListBegin($_etype542, $_size539);
+            for ($_i543 = 0; $_i543 < $_size539; ++$_i543)
             {
-              $elem530 = null;
-              $xfer += $input->readString($elem530);
-              $this->success []= $elem530;
+              $elem544 = null;
+              $xfer += $input->readString($elem544);
+              $this->success []= $elem544;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10455,9 +10655,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter531)
+          foreach ($this->success as $iter545)
           {
-            $xfer += $output->writeString($iter531);
+            $xfer += $output->writeString($iter545);
           }
         }
         $output->writeListEnd();
@@ -10588,14 +10788,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size532 = 0;
-            $_etype535 = 0;
-            $xfer += $input->readListBegin($_etype535, $_size532);
-            for ($_i536 = 0; $_i536 < $_size532; ++$_i536)
+            $_size546 = 0;
+            $_etype549 = 0;
+            $xfer += $input->readListBegin($_etype549, $_size546);
+            for ($_i550 = 0; $_i550 < $_size546; ++$_i550)
             {
-              $elem537 = null;
-              $xfer += $input->readString($elem537);
-              $this->success []= $elem537;
+              $elem551 = null;
+              $xfer += $input->readString($elem551);
+              $this->success []= $elem551;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10631,9 +10831,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter538)
+          foreach ($this->success as $iter552)
           {
-            $xfer += $output->writeString($iter538);
+            $xfer += $output->writeString($iter552);
           }
         }
         $output->writeListEnd();
@@ -11634,18 +11834,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size539 = 0;
-            $_ktype540 = 0;
-            $_vtype541 = 0;
-            $xfer += $input->readMapBegin($_ktype540, $_vtype541, $_size539);
-            for ($_i543 = 0; $_i543 < $_size539; ++$_i543)
+            $_size553 = 0;
+            $_ktype554 = 0;
+            $_vtype555 = 0;
+            $xfer += $input->readMapBegin($_ktype554, $_vtype555, $_size553);
+            for ($_i557 = 0; $_i557 < $_size553; ++$_i557)
             {
-              $key544 = '';
-              $val545 = new \metastore\Type();
-              $xfer += $input->readString($key544);
-              $val545 = new \metastore\Type();
-              $xfer += $val545->read($input);
-              $this->success[$key544] = $val545;
+              $key558 = '';
+              $val559 = new \metastore\Type();
+              $xfer += $input->readString($key558);
+              $val559 = new \metastore\Type();
+              $xfer += $val559->read($input);
+              $this->success[$key558] = $val559;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -11681,10 +11881,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter546 => $viter547)
+          foreach ($this->success as $kiter560 => $viter561)
           {
-            $xfer += $output->writeString($kiter546);
-            $xfer += $viter547->write($output);
+            $xfer += $output->writeString($kiter560);
+            $xfer += $viter561->write($output);
           }
         }
         $output->writeMapEnd();
@@ -11888,15 +12088,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size548 = 0;
-            $_etype551 = 0;
-            $xfer += $input->readListBegin($_etype551, $_size548);
-            for ($_i552 = 0; $_i552 < $_size548; ++$_i552)
+            $_size562 = 0;
+            $_etype565 = 0;
+            $xfer += $input->readListBegin($_etype565, $_size562);
+            for ($_i566 = 0; $_i566 < $_size562; ++$_i566)
             {
-              $elem553 = null;
-              $elem553 = new \metastore\FieldSchema();
-              $xfer += $elem553->read($input);
-              $this->success []= $elem553;
+              $elem567 = null;
+              $elem567 = new \metastore\FieldSchema();
+              $xfer += $elem567->read($input);
+              $this->success []= $elem567;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11948,9 +12148,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter554)
+          foreach ($this->success as $iter568)
           {
-            $xfer += $iter554->write($output);
+            $xfer += $iter568->write($output);
           }
         }
         $output->writeListEnd();
@@ -12192,15 +12392,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size555 = 0;
-            $_etype558 = 0;
-            $xfer += $input->readListBegin($_etype558, $_size555);
-            for ($_i559 = 0; $_i559 < $_size555; ++$_i559)
+            $_size569 = 0;
+            $_etype572 = 0;
+            $xfer += $input->readListBegin($_etype572, $_size569);
+            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
             {
-              $elem560 = null;
-              $elem560 = new \metastore\FieldSchema();
-              $xfer += $elem560->read($input);
-              $this->success []= $elem560;
+              $elem574 = null;
+              $elem574 = new \metastore\FieldSchema();
+              $xfer += $elem574->read($input);
+              $this->success []= $elem574;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12252,9 +12452,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter561)
+          foreach ($this->success as $iter575)
           {
-            $xfer += $iter561->write($output);
+            $xfer += $iter575->write($output);
           }
         }
         $output->writeListEnd();
@@ -12468,15 +12668,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size562 = 0;
-            $_etype565 = 0;
-            $xfer += $input->readListBegin($_etype565, $_size562);
-            for ($_i566 = 0; $_i566 < $_size562; ++$_i566)
+            $_size576 = 0;
+            $_etype579 = 0;
+            $xfer += $input->readListBegin($_etype579, $_size576);
+            for ($_i580 = 0; $_i580 < $_size576; ++$_i580)
             {
-              $elem567 = null;
-              $elem567 = new \metastore\FieldSchema();
-              $xfer += $elem567->read($input);
-              $this->success []= $elem567;
+              $elem581 = null;
+              $elem581 = new \metastore\FieldSchema();
+              $xfer += $elem581->read($input);
+              $this->success []= $elem581;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12528,9 +12728,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter568)
+          foreach ($this->success as $iter582)
           {
-            $xfer += $iter568->write($output);
+            $xfer += $iter582->write($output);
           }
         }
         $output->writeListEnd();
@@ -12772,15 +12972,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size569 = 0;
-            $_etype572 = 0;
-            $xfer += $input->readListBegin($_etype572, $_size569);
-            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
+            $_size583 = 0;
+            $_etype586 = 0;
+            $xfer += $input->readListBegin($_etype586, $_size583);
+            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
             {
-              $elem574 = null;
-              $elem574 = new \metastore\FieldSchema();
-              $xfer += $elem574->read($input);
-              $this->success []= $elem574;
+              $elem588 = null;
+              $elem588 = new \metastore\FieldSchema();
+              $xfer += $elem588->read($input);
+              $this->success []= $elem588;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12832,9 +13032,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter575)
+          foreach ($this->success as $iter589)
           {
-            $xfer += $iter575->write($output);
+            $xfer += $iter589->write($output);
           }
         }
         $output->writeListEnd();
@@ -13355,54 +13555,65 @@ class ThriftHiveMetastore_create_table_with_environment_context_result {
 
 }
 
-class ThriftHiveMetastore_drop_table_args {
+class ThriftHiveMetastore_create_table_with_constraints_args {
   static $_TSPEC;
 
   /**
-   * @var string
+   * @var \metastore\Table
    */
-  public $dbname = null;
+  public $tbl = null;
   /**
-   * @var string
+   * @var \metastore\SQLPrimaryKey[]
    */
-  public $name = null;
+  public $primaryKeys = null;
   /**
-   * @var bool
+   * @var \metastore\SQLForeignKey[]
    */
-  public $deleteData = null;
+  public $foreignKeys = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'dbname',
-          'type' => TType::STRING,
+          'var' => 'tbl',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Table',
           ),
         2 => array(
-          'var' => 'name',
-          'type' => TType::STRING,
+          'var' => 'primaryKeys',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\SQLPrimaryKey',
+            ),
           ),
         3 => array(
-          'var' => 'deleteData',
-          'type' => TType::BOOL,
+          'var' => 'foreignKeys',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\SQLForeignKey',
+            ),
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['dbname'])) {
-        $this->dbname = $vals['dbname'];
+      if (isset($vals['tbl'])) {
+        $this->tbl = $vals['tbl'];
       }
-      if (isset($vals['name'])) {
-        $this->name = $vals['name'];
+      if (isset($vals['primaryKeys'])) {
+        $this->primaryKeys = $vals['primaryKeys'];
       }
-      if (isset($vals['deleteData'])) {
-        $this->deleteData = $vals['deleteData'];
+      if (isset($vals['foreignKeys'])) {
+        $this->foreignKeys = $vals['foreignKeys'];
       }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_drop_table_args';
+    return 'ThriftHiveMetastore_create_table_with_constraints_args';
   }
 
   public function read($input)
@@ -13421,22 +13632,45 @@ class ThriftHiveMetastore_drop_table_args {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbname);
+          if ($ftype == TType::STRUCT) {
+            $this->tbl = new \metastore\Table();
+            $xfer += $this->tbl->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->name);
+          if ($ftype == TType::LST) {
+            $this->primaryKeys = array();
+            $_size590 = 0;
+            $_etype593 = 0;
+            $xfer += $input->readListBegin($_etype593, $_size590);
+            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
+            {
+              $elem595 = null;
+              $elem595 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem595->read($input);
+              $this->primaryKeys []= $elem595;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->deleteData);
+          if ($ftype == TType::LST) {
+            $this->foreignKeys = array();
+            $_size596 = 0;
+            $_etype599 = 0;
+            $xfer += $input->readListBegin($_etype599, $_size596);
+            for ($_i600 = 0; $_i600 < $_size596; ++$_i600)
+            {
+              $elem601 = null;
+              $elem601 = new \metastore\SQLForeignKey();
+              $xfer += $elem601->read($input);
+              $this->foreignKeys []= $elem601;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -13453,20 +13687,47 @@ class ThriftHiveMetastore_drop_table_args {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_args');
-    if ($this->dbname !== null) {
-      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1);
-      $xfer += $output->writeString($this->dbname);
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_constraints_args');
+    if ($this->tbl !== null) {
+      if (!is_object($this->tbl)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1);
+      $xfer += $this->tbl->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->name !== null) {
-      $xfer += $output->writeFieldBegin('name', TType::STRING, 2);
-      $xfer += $output->writeString($this->name);
+    if ($this->primaryKeys !== null) {
+      if (!is_array($this->primaryKeys)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('primaryKeys', TType::LST, 2);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
+        {
+          foreach ($this->primaryKeys as $iter602)
+          {
+            $xfer += $iter602->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->deleteData !== null) {
-      $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3);
-      $xfer += $output->writeBool($this->deleteData);
+    if ($this->foreignKeys !== null) {
+      if (!is_array($this->foreignKeys)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('foreignKeys', TType::LST, 3);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
+        {
+          foreach ($this->foreignKeys as $iter603)
+          {
+            $xfer += $iter603->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -13476,17 +13737,25 @@ class ThriftHiveMetastore_drop_table_args {
 
 }
 
-class ThriftHiveMetastore_drop_table_result {
+class ThriftHiveMetastore_create_table_with_constraints_result {
   static $_TSPEC;
 
   /**
-   * @var \metastore\NoSuchObjectException
+   * @var \metastore\AlreadyExistsException
    */
   public $o1 = null;
   /**
+   * @var \metastore\InvalidObjectException
+   */
+  public $o2 = null;
+  /**
    * @var \metastore\MetaException
    */
   public $o3 = null;
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o4 = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -13494,27 +13763,43 @@ class ThriftHiveMetastore_drop_table_result {
         1 => array(
           'var' => 'o1',
           'type' => TType::STRUCT,
-          'class' => '\metastore\NoSuchObjectException',
+          'class' => '\metastore\AlreadyExistsException',
           ),
         2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InvalidObjectException',
+          ),
+        3 => array(
           'var' => 'o3',
           'type' => TType::STRUCT,
           'class' => '\metastore\MetaException',
           ),
+        4 => array(
+          'var' => 'o4',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
         );
     }
     if (is_array($vals)) {
       if (isset($vals['o1'])) {
         $this->o1 = $vals['o1'];
       }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
       if (isset($vals['o3'])) {
         $this->o3 = $vals['o3'];
       }
+      if (isset($vals['o4'])) {
+        $this->o4 = $vals['o4'];
+      }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_drop_table_result';
+    return 'ThriftHiveMetastore_create_table_with_constraints_result';
   }
 
   public function read($input)
@@ -13534,7 +13819,7 @@ class ThriftHiveMetastore_drop_table_result {
       {
         case 1:
           if ($ftype == TType::STRUCT) {
-            $this->o1 = new \metastore\NoSuchObjectException();
+            $this->o1 = new \metastore\AlreadyExistsException();
             $xfer += $this->o1->read($input);
           } else {
             $xfer += $input->skip($ftype);
@@ -13542,12 +13827,28 @@ class ThriftHiveMetastore_drop_table_result {
           break;
         case 2:
           if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\InvalidObjectException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRUCT) {
             $this->o3 = new \metastore\MetaException();
             $xfer += $this->o3->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 4:
+          if ($ftype == TType::STRUCT) {
+            $this->o4 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o4->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -13560,17 +13861,27 @@ class ThriftHiveMetastore_drop_table_result {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_result');
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_constraints_result');
     if ($this->o1 !== null) {
       $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
       $xfer += $this->o1->write($output);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
     if ($this->o3 !== null) {
-      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2);
+      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3);
       $xfer += $this->o3->write($output);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->o4 !== null) {
+      $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4);
+      $xfer += $this->o4->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -13578,7 +13889,7 @@ class ThriftHiveMetastore_drop_table_result {
 
 }
 
-class ThriftHiveMetastore_drop_table_with_environment_context_args {
+class ThriftHiveMetastore_drop_table_args {
   static $_TSPEC;
 
   /**
@@ -13593,10 +13904,6 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args {
    * @var bool
    */
   public $deleteData = null;
-  /**
-   * @var \metastore\EnvironmentContext
-   */
-  public $environment_context = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -13613,11 +13920,6 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args {
           'var' => 'deleteData',
           'type' => TType::BOOL,
           ),
-        4 => array(
-          'var' => 'environment_context',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\EnvironmentContext',
-          ),
         );
     }
     if (is_array($vals)) {
@@ -13630,14 +13932,246 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args {
       if (isset($vals['deleteData'])) {
         $this->deleteData = $vals['deleteData'];
       }
-      if (isset($vals['environment_context'])) {
-        $this->environment_context = $vals['environment_context'];
-      }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_drop_table_with_environment_context_args';
+    return 'ThriftHiveMetastore_drop_table_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->dbname);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->deleteData);
+          } 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_drop_table_args');
+    if ($this->dbname !== null) {
+      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbname);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 2);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->deleteData !== null) {
+      $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3);
+      $xfer += $output->writeBool($this->deleteData);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_table_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o3 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        2 => array(
+          'var' => 'o3',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o3'])) {
+        $this->o3 = $vals['o3'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_table_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 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o3 = new \metastore\MetaException();
+            $xfer += $this->o3->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_drop_table_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o3 !== null) {
+      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2);
+      $xfer += $this->o3->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_table_with_environment_context_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbname = null;
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var bool
+   */
+  public $deleteData = null;
+  /**
+   * @var \metastore\EnvironmentContext
+   */
+  public $environment_context = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbname',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'deleteData',
+          'type' => TType::BOOL,
+          ),
+        4 => array(
+          'var' => 'environment_context',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\EnvironmentContext',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbname'])) {
+        $this->dbname = $vals['dbname'];
+      }
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['deleteData'])) {
+        $this->deleteData = $vals['deleteData'];
+      }
+      if (isset($vals['environment_context'])) {
+        $this->environment_context = $vals['environment_context'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_table_with_environment_context_args';
   }
 
   public function read($input)
@@ -13989,14 +14523,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size576 = 0;
-            $_etype579 = 0;
-            $xfer += $input->readListBegin($_etype579, $_size576);
-            for ($_i580 = 0; $_i580 < $_size576; ++$_i580)
+            $_size604 = 0;
+            $_etype607 = 0;
+            $xfer += $input->readListBegin($_etype607, $_size604);
+            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
             {
-              $elem581 = null;
-              $xfer += $input->readString($elem581);
-              $this->success []= $elem581;
+              $elem609 = null;
+              $xfer += $input->readString($elem609);
+              $this->success []= $elem609;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14032,9 +14566,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter582)
+          foreach ($this->success as $iter610)
           {
-            $xfer += $output->writeString($iter582);
+            $xfer += $output->writeString($iter610);
           }
         }
         $output->writeListEnd();
@@ -14139,14 +14673,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size583 = 0;
-            $_etype586 = 0;
-            $xfer += $input->readListBegin($_etype586, $_size583);
-            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
+            $_size611 = 0;
+            $_etype614 = 0;
+            $xfer += $input->readListBegin($_etype614, $_size611);
+            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
             {
-              $elem588 = null;
-              $xfer += $input->readString($elem588);
-              $this->tbl_types []= $elem588;
+              $elem616 = null;
+              $xfer += $input->readString($elem616);
+              $this->tbl_types []= $elem616;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14184,9 +14718,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter589)
+          foreach ($this->tbl_types as $iter617)
           {
-            $xfer += $output->writeString($iter589);
+            $xfer += $output->writeString($iter617);
           }
         }
         $output->writeListEnd();
@@ -14263,15 +14797,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size590 = 0;
-            $_etype593 = 0;
-            $xfer += $input->readListBegin($_etype593, $_size590);
-            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
+            $_size618 = 0;
+            $_etype621 = 0;
+            $xfer += $input->readListBegin($_etype621, $_size618);
+            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
             {
-              $elem595 = null;
-              $elem595 = new \metastore\TableMeta();
-              $xfer += $elem595->read($input);
-              $this->success []= $elem595;
+              $elem623 = null;
+              $elem623 = new \metastore\TableMeta();
+              $xfer += $elem623->read($input);
+              $this->success []= $elem623;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14307,9 +14841,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter596)
+          foreach ($this->success as $iter624)
           {
-            $xfer += $iter596->write($output);
+            $xfer += $iter624->write($output);
           }
         }
         $output->writeListEnd();
@@ -14465,14 +14999,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size597 = 0;
-            $_etype600 = 0;
-            $xfer += $input->readListBegin($_etype600, $_size597);
-            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
+            $_size625 = 0;
+            $_etype628 = 0;
+            $xfer += $input->readListBegin($_etype628, $_size625);
+            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
             {
-              $elem602 = null;
-              $xfer += $input->readString($elem602);
-              $this->success []= $elem602;
+              $elem630 = null;
+              $xfer += $input->readString($elem630);
+              $this->success []= $elem630;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14508,9 +15042,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter603)
+          foreach ($this->success as $iter631)
           {
-            $xfer += $output->writeString($iter603);
+            $xfer += $output->writeString($iter631);
           }
         }
         $output->writeListEnd();
@@ -14825,14 +15359,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size604 = 0;
-            $_etype607 = 0;
-            $xfer += $input->readListBegin($_etype607, $_size604);
-            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
+            $_size632 = 0;
+            $_etype635 = 0;
+            $xfer += $input->readListBegin($_etype635, $_size632);
+            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
             {
-              $elem609 = null;
-              $xfer += $input->readString($elem609);
-              $this->tbl_names []= $elem609;
+              $elem637 = null;
+              $xfer += $input->readString($elem637);
+              $this->tbl_names []= $elem637;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14865,9 +15399,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter610)
+          foreach ($this->tbl_names as $iter638)
           {
-            $xfer += $output->writeString($iter610);
+            $xfer += $output->writeString($iter638);
           }
         }
         $output->writeListEnd();
@@ -14968,15 +15502,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size611 = 0;
-            $_etype614 = 0;
-            $xfer += $input->readListBegin($_etype614, $_size611);
-            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
+            $_size639 = 0;
+            $_etype642 = 0;
+            $xfer += $input->readListBegin($_etype642, $_size639);
+            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
             {
-              $elem616 = null;
-              $elem616 = new \metastore\Table();
-              $xfer += $elem616->read($input);
-              $this->success []= $elem616;
+              $elem644 = null;
+              $elem644 = new \metastore\Table();
+              $xfer += $elem644->read($input);
+              $this->success []= $elem644;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15028,9 +15562,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter617)
+          foreach ($this->success as $iter645)
           {
-            $xfer += $iter617->write($output);
+            $xfer += $iter645->write($output);
           }
         }
         $output->writeListEnd();
@@ -15266,14 +15800,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size618 = 0;
-            $_etype621 = 0;
-            $xfer += $input->readListBegin($_etype621, $_size618);
-            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
+            $_size646 = 0;
+            $_etype649 = 0;
+            $xfer += $input->readListBegin($_etype649, $_size646);
+            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
             {
-              $elem623 = null;
-              $xfer += $input->readString($elem623);
-              $this->success []= $elem623;
+              $elem651 = null;
+              $xfer += $input->readString($elem651);
+              $this->success []= $elem651;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15325,9 +15859,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter624)
+          foreach ($this->success as $iter652)
           {
-            $xfer += $output->writeString($iter624);
+            $xfer += $output->writeString($iter652);
           }
         }
         $output->writeListEnd();
@@ -16640,15 +17174,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size625 = 0;
-            $_etype628 = 0;
-            $xfer += $input->readListBegin($_etype628, $_size625);
-            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
+            $_size653 = 0;
+            $_etype656 = 0;
+            $xfer += $input->readListBegin($_etype656, $_size653);
+            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
             {
-              $elem630 = null;
-              $elem630 = new \metastore\Partition();
-              $xfer += $elem630->read($input);
-              $this->new_parts []= $elem630;
+              $elem658 = null;
+              $elem658 = new \metastore\Partition();
+              $xfer += $elem658->read($input);
+              $this->new_parts []= $elem658;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16676,9 +17210,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter631)
+          foreach ($this->new_parts as $iter659)
           {
-            $xfer += $iter631->write($output);
+            $xfer += $iter659->write($output);
           }
         }
         $output->writeListEnd();
@@ -16893,15 +17427,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size632 = 0;
-            $_etype635 = 0;
-            $xfer += $input->readListBegin($_etype635, $_size632);
-            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
+            $_size660 = 0;
+            $_etype663 = 0;
+            $xfer += $input->readListBegin($_etype663, $_size660);
+            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
             {
-              $elem637 = null;
-              $elem637 = new \metastore\PartitionSpec();
-              $xfer += $elem637->read($input);
-              $this->new_parts []= $elem637;
+              $elem665 = null;
+              $elem665 = new \metastore\PartitionSpec();
+              $xfer += $elem665->read($input);
+              $this->new_parts []= $elem665;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16929,9 +17463,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter638)
+          foreach ($this->new_parts as $iter666)
           {
-            $xfer += $iter638->write($output);
+            $xfer += $iter666->write($output);
           }
         }
         $output->writeListEnd();
@@ -17181,14 +17715,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size639 = 0;
-            $_etype642 = 0;
-            $xfer += $input->readListBegin($_etype642, $_size639);
-            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
+            $_size667 = 0;
+            $_etype670 = 0;
+            $xfer += $input->readListBegin($_etype670, $_size667);
+            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
             {
-              $elem644 = null;
-              $xfer += $input->readString($elem644);
-              $this->part_vals []= $elem644;
+              $elem672 = null;
+              $xfer += $input->readString($elem672);
+              $this->part_vals []= $elem672;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17226,9 +17760,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter645)
+          foreach ($this->part_vals as $iter673)
           {
-            $xfer += $output->writeString($iter645);
+            $xfer += $output->writeString($iter673);
           }
         }
         $output->writeListEnd();
@@ -17730,14 +18264,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size646 = 0;
-            $_etype649 = 0;
-            $xfer += $input->readListBegin($_etype649, $_size646);
-            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
+            $_size674 = 0;
+            $_etype677 = 0;
+            $xfer += $input->readListBegin($_etype677, $_size674);
+            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
             {
-              $elem651 = null;
-              $xfer += $input->readString($elem651);
-              $this->part_vals []= $elem651;
+              $elem679 = null;
+              $xfer += $input->readString($elem679);
+              $this->part_vals []= $elem679;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17783,9 +18317,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter652)
+          foreach ($this->part_vals as $iter680)
           {
-            $xfer += $output->writeString($iter652);
+            $xfer += $output->writeString($iter680);
           }
         }
         $output->writeListEnd();
@@ -18639,14 +19173,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size653 = 0;
-            $_etype656 = 0;
-            $xfer += $input->readListBegin($_etype656, $_size653);
-            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
+            $_size681 = 0;
+            $_etype684 = 0;
+            $xfer += $input->readListBegin($_etype684, $_size681);
+            for ($_i685 = 0; $_i685 < $_size681; ++$_i685)
             {
-              $elem658 = null;
-              $xfer += $input->readString($elem658);
-              $this->part_vals []= $elem658;
+              $elem686 = null;
+              $xfer += $input->readString($elem686);
+              $this->part_vals []= $elem686;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18691,9 +19225,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter659)
+          foreach ($this->part_vals as $iter687)
           {
-            $xfer += $output->writeString($iter659);
+            $xfer += $output->writeString($iter687);
           }
         }
         $output->writeListEnd();
@@ -18946,14 +19480,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size660 = 0;
-            $_etype663 = 0;
-            $xfer += $input->readListBegin($_etype663, $_size660);
-            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
+            $_size688 = 0;
+            $_etype691 = 0;
+            $xfer += $input->readListBegin($_etype691, $_size688);
+            for ($_i692 = 0; $_i692 < $_size688; ++$_i692)
             {
-              $elem665 = null;
-              $xfer += $input->readString($elem665);
-              $this->part_vals []= $elem665;
+              $elem693 = null;
+              $xfer += $input->readString($elem693);
+              $this->part_vals []= $elem693;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19006,9 +19540,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter666)
+          foreach ($this->part_vals as $iter694)
           {
-            $xfer += $output->writeString($iter666);
+            $xfer += $output->writeString($iter694);
           }
         }
         $output->writeListEnd();
@@ -20022,14 +20556,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size667 = 0;
-            $_etype670 = 0;
-            $xfer += $input->readListBegin($_etype670, $_size667);
-            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
+            $_size695 = 0;
+            $_etype698 = 0;
+            $xfer += $input->readListBegin($_etype698, $_size695);
+            for ($_i699 = 0; $_i699 < $_size695; ++$_i699)
             {
-              $elem672 = null;
-              $xfer += $input->readString($elem672);
-              $this->part_vals []= $elem672;
+              $elem700 = null;
+              $xfer += $input->readString($elem700);
+              $this->part_vals []= $elem700;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20067,9 +20601,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter673)
+          foreach ($this->part_vals as $iter701)
           {
-            $xfer += $output->writeString($iter673);
+            $xfer += $output->writeString($iter701);
           }
         }
         $output->writeListEnd();
@@ -20311,17 +20845,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size674 = 0;
-            $_ktype675 = 0;
-            $_vtype676 = 0;
-            $xfer += $input->readMapBegin($_ktype675, $_vtype676, $_size674);
-            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
+            $_size702 = 0;
+            $_ktype703 = 0;
+            $_vtype704 = 0;
+            $xfer += $input->readMapBegin($_ktype703, $_vtype704, $_size702);
+            for ($_i706 = 0; $_i706 < $_size702; ++$_i706)
             {
-              $key679 = '';
-              $val680 = '';
-              $xfer += $input->readString($key679);
-              $xfer += $input->readString($val680);
-              $this->partitionSpecs[$key679] = $val680;
+              $key707 = '';
+              $val708 = '';
+              $xfer += $input->readString($key707);
+              $xfer += $input->readString($val708);
+              $this->partitionSpecs[$key707] = $val708;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -20377,10 +20911,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter681 => $viter682)
+          foreach ($this->partitionSpecs as $kiter709 => $viter710)
           {
-            $xfer += $output->writeString($kiter681);
-            $xfer += $output->writeString($viter682);
+            $xfer += $output->writeString($kiter709);
+            $xfer += $output->writeString($viter710);
           }
         }
         $output->writeMapEnd();
@@ -20692,17 +21226,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size683 = 0;
-            $_ktype684 = 0;
-            $_vtype685 = 0;
-            $xfer += $input->readMapBegin($_ktype684, $_vtype685, $_size683);
-            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
+            $_size711 = 0;
+            $_ktype712 = 0;
+            $_vtype713 = 0;
+            $xfer += $input->readMapBegin($_ktype712, $_vtype713, $_size711);
+            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
             {
-              $key688 = '';
-              $val689 = '';
-              $xfer += $input->readString($key688);
-              $xfer += $input->readString($val689);
-              $this->partitionSpecs[$key688] = $val689;
+              $key716 = '';
+              $val717 = '';
+              $xfer += $input->readString($key716);
+              $xfer += $input->readString($val717);
+              $this->partitionSpecs[$key716] = $val717;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -20758,10 +21292,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter690 => $viter691)
+          foreach ($this->partitionSpecs as $kiter718 => $viter719)
           {
-            $xfer += $output->writeString($kiter690);
-            $xfer += $output->writeString($viter691);
+            $xfer += $output->writeString($kiter718);
+            $xfer += $output->writeString($viter719);
           }
         }
         $output->writeMapEnd();
@@ -20894,15 +21428,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size692 = 0;
-            $_etype695 = 0;
-            $xfer += $input->readListBegin($_etype695, $_size692);
-            for ($_i696 = 0; $_i696 < $_size692; ++$_i696)
+            $_size720 = 0;
+            $_etype723 = 0;
+            $xfer += $input->readListBegin($_etype723, $_size720);
+            for ($_i724 = 0; $_i724 < $_size720; ++$_i724)
             {
-              $elem697 = null;
-              $elem697 = new \metastore\Partition();
-              $xfer += $elem697->read($input);
-              $this->success []= $elem697;
+              $elem725 = null;
+              $elem725 = new \metastore\Partition();
+              $xfer += $elem725->read($input);
+              $this->success []= $elem725;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20962,9 +21496,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter698)
+          foreach ($this->success as $iter726)
           {
-            $xfer += $iter698->write($output);
+            $xfer += $iter726->write($output);
           }
         }
         $output->writeListEnd();
@@ -21110,14 +21644,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size699 = 0;
-            $_etype702 = 0;
-            $xfer += $input->readListBegin($_etype702, $_size699);
-            for ($_i703 = 0; $_i703 < $_size699; ++$_i703)
+            $_size727 = 0;
+            $_etype730 = 0;
+            $xfer += $input->readListBegin($_etype730, $_size727);
+            for ($_i731 = 0; $_i731 < $_size727; ++$_i731)
             {
-              $elem704 = null;
-              $xfer += $input->readString($elem704);
-              $this->part_vals []= $elem704;
+              $elem732 = null;
+              $xfer += $input->readString($elem732);
+              $this->part_vals []= $elem732;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21134,14 +21668,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size705 = 0;
-            $_etype708 = 0;
-            $xfer += $input->readListBegin($_etype708, $_size705);
-            for ($_i709 = 0; $_i709 < $_size705; ++$_i709)
+            $_size733 = 0;
+            $_etype736 = 0;
+            $xfer += $input->readListBegin($_etype736, $_size733);
+            for ($_i737 = 0; $_i737 < $_size733; ++$_i737)
             {
-              $elem710 = null;
-              $xfer += $input->readString($elem710);
-              $this->group_names []= $elem710;
+              $elem738 = null;
+              $xfer += $input->readString($elem738);
+              $this->group_names []= $elem738;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21179,9 +21713,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter711)
+          foreach ($this->part_vals as $iter739)
           {
-            $xfer += $output->writeString($iter711);
+            $xfer += $output->writeString($iter739);
           }
         }
         $output->writeListEnd();
@@ -21201,9 +21735,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter712)
+          foreach ($this->group_names as $iter740)
           {
-            $xfer += $output->writeString($iter712);
+            $xfer += $output->writeString($iter740);
           }
         }
         $output->writeListEnd();
@@ -21794,15 +22328,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size713 = 0;
-            $_etype716 = 0;
-            $xfer += $input->readListBegin($_etype716, $_size713);
-            for ($_i717 = 0; $_i717 < $_size713; ++$_i717)
+            $_size741 = 0;
+            $_etype744 = 0;
+            $xfer += $input->readListBegin($_etype744, $_size741);
+            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
             {
-              $elem718 = null;
-              $elem718 = new \metastore\Partition();
-              $xfer += $elem718->read($input);
-              $this->success []= $elem718;
+              $elem746 = null;
+              $elem746 = new \metastore\Partition();
+              $xfer += $elem746->read($input);
+              $this->success []= $elem746;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21846,9 +22380,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter719)
+          foreach ($this->success as $iter747)
           {
-            $xfer += $iter719->write($output);
+            $xfer += $iter747->write($output);
           }
         }
         $output->writeListEnd();
@@ -21994,14 +22528,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size720 = 0;
-            $_etype723 = 0;
-            $xfer += $input->readListBegin($_etype723, $_size720);
-            for ($_i724 = 0; $_i724 < $_size720; ++$_i724)
+            $_size748 = 0;
+            $_etype751 = 0;
+            $xfer += $input->readListBegin($_etype751, $_size748);
+            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
             {
-              $elem725 = null;
-              $xfer += $input->readString($elem725);
-              $this->group_names []= $elem725;
+              $elem753 = null;
+              $xfer += $input->readString($elem753);
+              $this->group_names []= $elem753;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22049,9 +22583,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter726)
+          foreach ($this->group_names as $iter754)
           {
-            $xfer += $output->writeString($iter726);
+            $xfer += $output->writeString($iter754);
           }
         }
         $output->writeListEnd();
@@ -22140,15 +22674,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size727 = 0;
-            $_etype730 = 0;
-            $xfer += $input->readListBegin($_etype730, $_size727);
-            for ($_i731 = 0; $_i731 < $_size727; ++$_i731)
+            $_size755 = 0;
+            $_etype758 = 0;
+            $xfer += $input->readListBegin($_etype758, $_size755);
+            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
             {
-              $elem732 = null;
-              $elem732 = new \metastore\Partition();
-              $xfer += $elem732->read($input);
-              $this->success []= $elem732;
+              $elem760 = null;
+              $elem760 = new \metastore\Partition();
+              $xfer += $elem760->read($input);
+              $this->success []= $elem760;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22192,9 +22726,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter733)
+          foreach ($this->success as $iter761)
           {
-            $xfer += $iter733->write($output);
+            $xfer += $iter761->write($output);
           }
         }
         $output->writeListEnd();
@@ -22414,15 +22948,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size734 = 0;
-            $_etype737 = 0;
-            $xfer += $input->readListBegin($_etype737, $_size734);
-            for ($_i738 = 0; $_i738 < $_size734; ++$_i738)
+            $_size762 = 0;
+            $_etype765 = 0;
+            $xfer += $input->readListBegin($_etype765, $_size762);
+            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
             {
-              $elem739 = null;
-              $elem739 = new \metastore\PartitionSpec();
-              $xfer += $elem739->read($input);
-              $this->success []= $elem739;
+              $elem767 = null;
+              $elem767 = new \metastore\PartitionSpec();
+              $xfer += $elem767->read($input);
+              $this->success []= $elem767;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22466,9 +23000,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter740)
+          foreach ($this->success as $iter768)
           {
-            $xfer += $iter740->write($output);
+            $xfer += $iter768->write($output);
           }
         }
         $output->writeListEnd();
@@ -22675,14 +23209,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size741 = 0;
-            $_etype744 = 0;
-            $xfer += $input->readListBegin($_etype744, $_size741);
-            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
+            $_size769 = 0;
+            $_etype772 = 0;
+            $xfer += $input->readListBegin($_etype772, $_size769);
+            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
             {
-              $elem746 = null;
-              $xfer += $input->readString($elem746);
-              $this->success []= $elem746;
+              $elem774 = null;
+              $xfer += $input->readString($elem774);
+              $this->success []= $elem774;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22718,9 +23252,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter747)
+          foreach ($this->success as $iter775)
           {
-            $xfer += $output->writeString($iter747);
+            $xfer += $output->writeString($iter775);
           }
         }
         $output->writeListEnd();
@@ -22836,14 +23370,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size748 = 0;
-            $_etype751 = 0;
-            $xfer += $input->readListBegin($_etype751, $_size748);
-            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
+            $_size776 = 0;
+            $_etype779 = 0;
+            $xfer += $input->readListBegin($_etype779, $_size776);
+            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
             {
-              $elem753 = null;
-              $xfer += $input->readString($elem753);
-              $this->part_vals []= $elem753;
+              $elem781 = null;
+              $xfer += $input->readString($elem781);
+              $this->part_vals []= $elem781;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22888,9 +23422,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter754)
+          foreach ($this->part_vals as $iter782)
           {
-            $xfer += $output->writeString($iter754);
+            $xfer += $output->writeString($iter782);
           }
         }
         $output->writeListEnd();
@@ -22984,15 +23518,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size755 = 0;
-            $_etype758 = 0;
-            $xfer += $input->readListBegin($_etype758, $_size755);
-            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
+            $_size783 = 0;
+            $_etype786 = 0;
+            $xfer += $input->readListBegin($_etype786, $_size783);
+            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
             {
-              $elem760 = null;
-              $elem760 = new \metastore\Partition();
-              $xfer += $elem760->read($input);
-              $this->success []= $elem760;
+              $elem788 = null;
+              $elem788 = new \metastore\Partition();
+              $xfer += $elem788->read($input);
+              $this->success []= $elem788;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23036,9 +23570,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter761)
+          foreach ($this->success as $iter789)
           {
-            $xfer += $iter761->write($output);
+            $xfer += $iter789->write($output);
           }
         }
         $output->writeListEnd();
@@ -23185,14 +23719,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size762 = 0;
-            $_etype765 = 0;
-            $xfer += $input->readListBegin($_etype765, $_size762);
-            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
+            $_size790 = 0;
+            $_etype793 = 0;
+            $xfer += $input->readListBegin($_etype793, $_size790);
+            for ($_i794 = 0; $_i794 < $_size790; ++$_i794)
             {
-              $elem767 = null;
-              $xfer += $input->readString($elem767);
-              $this->part_vals []= $elem767;
+              $elem795 = null;
+              $xfer += $input->readString($elem795);
+              $this->part_vals []= $elem795;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23216,14 +23750,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size768 = 0;
-            $_etype771 = 0;
-            $xfer += $input->readListBegin($_etype771, $_size768);
-            for ($_i772 = 0; $_i772 < $_size768; ++$_i772)
+            $_size796 = 0;
+            $_etype799 = 0;
+            $xfer += $input->readListBegin($_etype799, $_size796);
+            for ($_i800 = 0; $_i800 < $_size796; ++$_i800)
             {
-              $elem773 = null;
-              $xfer += $input->readString($elem773);
-              $this->group_names []= $elem773;
+              $elem801 = null;
+              $xfer += $input->readString($elem801);
+              $this->group_names []= $elem801;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23261,9 +23795,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter774)
+          foreach ($this->part_vals as $iter802)
           {
-            $xfer += $output->writeString($iter774);
+            $xfer += $output->writeString($iter802);
           }
         }
         $output->writeListEnd();
@@ -23288,9 +23822,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter775)
+          foreach ($this->group_names as $iter803)
           {
-            $xfer += $output->writeString($iter775);
+            $xfer += $output->writeString($iter803);
           }
         }
         $output->writeListEnd();
@@ -23379,15 +23913,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size776 = 0;
-            $_etype779 = 0;
-            $xfer += $input->readListBegin($_etype779, $_size776);
-            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
+            $_size804 = 0;
+            $_etype807 = 0;
+            $xfer += $input->readListBegin($_etype807, $_size804);
+            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
             {
-              $elem781 = null;
-              $elem781 = new \metastore\Partition();
-              $xfer += $elem781->read($input);
-              $this->success []= $elem781;
+              $elem809 = null;
+              $elem809 = new \metastore\Partition();
+              $xfer += $elem809->read($input);
+              $this->success []= $elem809;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23431,9 +23965,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter782)
+          foreach ($this->success as $iter810)
           {
-            $xfer += $iter782->write($output);
+            $xfer += $iter810->write($output);
           }
         }
         $output->writeListEnd();
@@ -23554,14 +24088,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size783 = 0;
-            $_etype786 = 0;
-            $xfer += $input->readListBegin($_etype786, $_size783);
-            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
+            $_size811 = 0;
+            $_etype814 = 0;
+            $xfer += $input->readListBegin($_etype814, $_size811);
+            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
             {
-              $elem788 = null;
-              $xfer += $input->readString($elem788);
-              $this->part_vals []= $elem788;
+              $elem816 = null;
+              $xfer += $input->readString($elem816);
+              $this->part_vals []= $elem816;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23606,9 +24140,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter789)
+          foreach ($this->part_vals as $iter817)
           {
-            $xfer += $output->writeString($iter789);
+            $xfer += $output->writeString($iter817);
           }
         }
         $output->writeListEnd();
@@ -23701,14 +24235,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size790 = 0;
-            $_etype793 = 0;
-            $xfer += $input->readListBegin($_etype793, $_size790);
-            for ($_i794 = 0; $_i794 < $_size790; ++$_i794)
+            $_size818 = 0;
+            $_etype821 = 0;
+            $xfer += $input->readListBegin($_etype821, $_size818);
+            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
             {
-              $elem795 = null;
-              $xfer += $input->readString($elem795);
-              $this->success []= $elem795;
+              $elem823 = null;
+              $xfer += $input->readString($elem823);
+              $this->success []= $elem823;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23752,9 +24286,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter796)
+          foreach ($this->success as $iter824)
           {
-            $xfer += $output->writeString($iter796);
+            $xfer += $output->writeString($iter824);
           }
         }
         $output->writeListEnd();
@@ -23997,15 +24531,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_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)
+            $_size825 = 0;
+            $_etype828 = 0;
+            $xfer += $input->readListBegin($_etype828, $_size825);
+            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
             {
-              $elem802 = null;
-              $elem802 = new \metastore\Partition();
-              $xfer += $elem802->read($input);
-              $this->success []= $elem802;
+              $elem830 = null;
+              $elem830 = new \metastore\Partition();
+              $xfer += $elem830->read($input);
+              $this->success []= $elem830;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24049,9 +24583,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter803)
+          foreach ($this->success as $iter831)
           {
-            $xfer += $iter803->write($output);
+            $xfer += $iter831->write($output);
           }
         }
         $output->writeListEnd();
@@ -24294,15 +24828,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_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)
+            $_size832 = 0;
+            $_etype835 = 0;
+            $xfer += $input->readListBegin($_etype835, $_size832);
+            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
             {
-              $elem809 = null;
-              $elem809 = new \metastore\PartitionSpec();
-              $xfer += $elem809->read($input);
-              $this->success []= $elem809;
+              $elem837 = null;
+              $elem837 = new \metastore\PartitionSpec();
+              $xfer += $elem837->read($input);
+              $this->success []= $elem837;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24346,9 +24880,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter810)
+          foreach ($this->success as $iter838)
           {
-            $xfer += $iter810->write($output);
+            $xfer += $iter838->write($output);
           }
         }
         $output->writeListEnd();
@@ -24914,14 +25448,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size811 = 0;
-            $_etype814 = 0;
-            $xfer += $input->readListBegin($_etype814, $_size811);
-            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
+            $_size839 = 0;
+            $_etype842 = 0;
+            $xfer += $input->readListBegin($_etype842, $_size839);
+            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
             {
-              $elem816 = null;
-              $xfer += $input->readString($elem816);
-              $this->names []= $elem816;
+              $elem844 = null;
+              $xfer += $input->readString($elem844);
+              $this->names []= $elem844;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24959,9 +25493,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter817)
+          foreach ($this->names as $iter845)
           {
-            $xfer += $output->writeString($iter817);
+            $xfer += $output->writeString($iter845);
           }
         }
         $output->writeListEnd();
@@ -25050,15 +25584,15 @@ class ThriftHiveMetastore_get_partitions_by_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)
+            $_size846 = 0;
+            $_etype849 = 0;
+            $xfer += $input->readListBegin($_etype849, $_size846);
+            for ($_i850 = 0; $_i850 < $_size846; ++$_i850)
             {
-              $elem823 = null;
-              $elem823 = new \metastore\Partition();
-              $xfer += $elem823->read($input);
-              $this->success []= $elem823;
+              $elem851 = null;
+              $elem851 = new \metastore\Partition();
+              $xfer += $elem851->read($input);
+              $this->success []= $elem851;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25102,9 +25636,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter824)
+          foreach ($this->success as $iter852)
           {
-            $xfer += $iter824->write($output);
+            $xfer += $iter852->write($output);
           }
         }
         $output->writeListEnd();
@@ -25443,15 +25977,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size825 = 0;
-            $_etype828 = 0;
-            $xfer += $input->readListBegin($_etype828, $_size825);
-            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
+            $_size853 = 0;
+            $_etype856 = 0;
+            $xfer += $input->readListBegin($_etype856, $_size853);
+            for ($_i857 = 0; $_i857 < $_size853; ++$_i857)
             {
-              $elem830 = null;
-              $elem830 = new \metastore\Partition();
-              $xfer += $elem830->read($input);
-              $this->new_parts []= $elem830;
+              $elem858 = null;
+              $elem858 = new \metastore\Partition();
+              $xfer += $elem858->read($input);
+              $this->new_parts []= $elem858;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25489,9 +26023,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter831)
+          foreach ($this->new_parts as $iter859)
           {
-            $xfer += $iter831->write($output);
+            $xfer += $iter859->write($output);
           }
         }
         $output->writeListEnd();
@@ -25706,15 +26240,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size832 = 0;
-            $_etype835 = 0;
-            $xfer += $input->readListBegin($_etype835, $_size832);
-            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
+            $_size860 = 0;
+            $_etype863 

<TRUNCATED>

[05/30] hive git commit: Revert "HIVE-12159: Create vectorized readers for the complex types (Owen O'Malley, reviewed by Matt McCline)"

Posted by jd...@apache.org.
Revert "HIVE-12159: Create vectorized readers for the complex types (Owen O'Malley, reviewed by Matt McCline)"

This reverts commit 0dd4621f34f6043071474220a082268cda124b9d.


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

Branch: refs/heads/llap
Commit: d559b34755010b5ed3ecc31fa423d01788e5e875
Parents: 40e0c38
Author: Matt McCline <mm...@hortonworks.com>
Authored: Fri Apr 15 16:00:18 2016 -0700
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Fri Apr 15 16:00:18 2016 -0700

----------------------------------------------------------------------
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   45 +-
 orc/src/java/org/apache/orc/OrcUtils.java       |   75 -
 orc/src/java/org/apache/orc/Reader.java         |    6 -
 orc/src/java/org/apache/orc/RecordReader.java   |    8 +-
 .../java/org/apache/orc/TypeDescription.java    |   62 +-
 .../org/apache/orc/impl/BitFieldReader.java     |    5 +-
 .../java/org/apache/orc/impl/IntegerReader.java |   26 +-
 .../apache/orc/impl/RunLengthByteReader.java    |   36 +-
 .../apache/orc/impl/RunLengthIntegerReader.java |   31 +-
 .../orc/impl/RunLengthIntegerReaderV2.java      |   33 +-
 .../java/org/apache/orc/impl/WriterImpl.java    |   47 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |   13 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   43 +-
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |    3 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   12 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   50 +-
 .../hadoop/hive/ql/io/orc/SchemaEvolution.java  |  234 ++-
 .../hive/ql/io/orc/TreeReaderFactory.java       |  838 ++++-----
 .../ql/io/orc/VectorizedOrcInputFormat.java     |   32 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |    2 +
 .../hive/ql/io/orc/TestTypeDescription.java     |    4 +-
 .../hive/ql/io/orc/TestVectorOrcFile.java       | 1634 +++++++++---------
 .../hive/ql/io/orc/TestVectorizedORCReader.java |    7 +-
 .../hive/ql/exec/vector/BytesColumnVector.java  |   11 -
 .../ql/exec/vector/TimestampColumnVector.java   |    2 +-
 .../hive/ql/exec/vector/UnionColumnVector.java  |    2 +
 26 files changed, 1476 insertions(+), 1785 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
index baaa4d7..7ee263d 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hive.llap.io.decode;
 
 import java.io.IOException;
-import java.util.List;
 
 import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch;
 import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch.ColumnStreamData;
@@ -28,12 +27,7 @@ import org.apache.hadoop.hive.llap.io.api.impl.ColumnVectorBatch;
 import org.apache.hadoop.hive.llap.io.metadata.OrcFileMetadata;
 import org.apache.hadoop.hive.llap.io.metadata.OrcStripeMetadata;
 import org.apache.hadoop.hive.llap.metrics.LlapDaemonQueueMetrics;
-import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.orc.CompressionCodec;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer;
@@ -77,35 +71,6 @@ public class OrcEncodedDataConsumer
     stripes[m.getStripeIx()] = m;
   }
 
-  private static ColumnVector createColumn(OrcProto.Type type,
-                                           int batchSize) {
-    switch (type.getKind()) {
-      case BOOLEAN:
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-      case DATE:
-        return new LongColumnVector(batchSize);
-      case FLOAT:
-      case DOUBLE:
-        return new DoubleColumnVector(batchSize);
-      case BINARY:
-      case STRING:
-      case CHAR:
-      case VARCHAR:
-        return new BytesColumnVector(batchSize);
-      case TIMESTAMP:
-        return new TimestampColumnVector(batchSize);
-      case DECIMAL:
-        return new DecimalColumnVector(batchSize, type.getPrecision(),
-            type.getScale());
-      default:
-        throw new IllegalArgumentException("LLAP does not support " +
-            type.getKind());
-    }
-  }
-
   @Override
   protected void decodeBatch(OrcEncodedColumnBatch batch,
       Consumer<ColumnVectorBatch> downstreamConsumer) {
@@ -147,15 +112,9 @@ public class OrcEncodedDataConsumer
         ColumnVectorBatch cvb = cvbPool.take();
         assert cvb.cols.length == batch.getColumnIxs().length; // Must be constant per split.
         cvb.size = batchSize;
-        List<OrcProto.Type> types = fileMetadata.getTypes();
-        int[] columnMapping = batch.getColumnIxs();
+
         for (int idx = 0; idx < batch.getColumnIxs().length; idx++) {
-          if (cvb.cols[idx] == null) {
-            // skip over the top level struct, but otherwise assume no complex
-            // types
-            cvb.cols[idx] = createColumn(types.get(columnMapping[idx]), batchSize);
-          }
-          columnReaders[idx].nextVector(cvb.cols[idx], null, batchSize);
+          cvb.cols[idx] = (ColumnVector)columnReaders[idx].nextVector(cvb.cols[idx], batchSize);
         }
 
         // we are done reading a batch, send it to consumer for processing

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/OrcUtils.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/OrcUtils.java b/orc/src/java/org/apache/orc/OrcUtils.java
index 2ebe9a7..2e93254 100644
--- a/orc/src/java/org/apache/orc/OrcUtils.java
+++ b/orc/src/java/org/apache/orc/OrcUtils.java
@@ -449,79 +449,4 @@ public class OrcUtils {
     return columnId;
   }
 
-  /**
-   * Translate the given rootColumn from the list of types to a TypeDescription.
-   * @param types all of the types
-   * @param rootColumn translate this type
-   * @return a new TypeDescription that matches the given rootColumn
-   */
-  public static
-        TypeDescription convertTypeFromProtobuf(List<OrcProto.Type> types,
-                                                int rootColumn) {
-    OrcProto.Type type = types.get(rootColumn);
-    switch (type.getKind()) {
-      case BOOLEAN:
-        return TypeDescription.createBoolean();
-      case BYTE:
-        return TypeDescription.createByte();
-      case SHORT:
-        return TypeDescription.createShort();
-      case INT:
-        return TypeDescription.createInt();
-      case LONG:
-        return TypeDescription.createLong();
-      case FLOAT:
-        return TypeDescription.createFloat();
-      case DOUBLE:
-        return TypeDescription.createDouble();
-      case STRING:
-        return TypeDescription.createString();
-      case CHAR:
-        return TypeDescription.createChar()
-            .withMaxLength(type.getMaximumLength());
-      case VARCHAR:
-        return TypeDescription.createVarchar()
-            .withMaxLength(type.getMaximumLength());
-      case BINARY:
-        return TypeDescription.createBinary();
-      case TIMESTAMP:
-        return TypeDescription.createTimestamp();
-      case DATE:
-        return TypeDescription.createDate();
-      case DECIMAL: {
-        TypeDescription result = TypeDescription.createDecimal();
-        if (type.hasScale()) {
-          result.withScale(type.getScale());
-        }
-        if (type.hasPrecision()) {
-          result.withPrecision(type.getPrecision());
-        }
-        return result;
-      }
-      case LIST:
-        return TypeDescription.createList(
-            convertTypeFromProtobuf(types, type.getSubtypes(0)));
-      case MAP:
-        return TypeDescription.createMap(
-            convertTypeFromProtobuf(types, type.getSubtypes(0)),
-            convertTypeFromProtobuf(types, type.getSubtypes(1)));
-      case STRUCT: {
-        TypeDescription result = TypeDescription.createStruct();
-        for(int f=0; f < type.getSubtypesCount(); ++f) {
-          result.addField(type.getFieldNames(f),
-              convertTypeFromProtobuf(types, type.getSubtypes(f)));
-        }
-        return result;
-      }
-      case UNION: {
-        TypeDescription result = TypeDescription.createUnion();
-        for(int f=0; f < type.getSubtypesCount(); ++f) {
-          result.addUnionChild(
-              convertTypeFromProtobuf(types, type.getSubtypes(f)));
-        }
-        return result;
-      }
-    }
-    throw new IllegalArgumentException("Unknown ORC type " + type.getKind());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/Reader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/Reader.java b/orc/src/java/org/apache/orc/Reader.java
index 62a05e9..be722b5 100644
--- a/orc/src/java/org/apache/orc/Reader.java
+++ b/orc/src/java/org/apache/orc/Reader.java
@@ -116,15 +116,9 @@ public interface Reader {
   ColumnStatistics[] getStatistics();
 
   /**
-   * Get the type of rows in this ORC file.
-   */
-  TypeDescription getSchema();
-
-  /**
    * Get the list of types contained in the file. The root type is the first
    * type in the list.
    * @return the list of flattened types
-   * @deprecated use getSchema instead
    */
   List<OrcProto.Type> getTypes();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/RecordReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/RecordReader.java b/orc/src/java/org/apache/orc/RecordReader.java
index 09ba0f0..7229dda 100644
--- a/orc/src/java/org/apache/orc/RecordReader.java
+++ b/orc/src/java/org/apache/orc/RecordReader.java
@@ -30,11 +30,13 @@ public interface RecordReader {
    * controlled by the callers. Caller need to look at
    * VectorizedRowBatch.size of the retunred object to know the batch
    * size read.
-   * @param batch a row batch object to read into
-   * @return were more rows available to read?
+   * @param previousBatch a row batch object that can be reused by the reader
+   * @return the row batch that was read. The batch will have a non-zero row
+   *         count if the pointer isn't at the end of the file
    * @throws java.io.IOException
    */
-  boolean nextBatch(VectorizedRowBatch batch) throws IOException;
+  VectorizedRowBatch nextBatch(VectorizedRowBatch previousBatch
+			       ) throws IOException;
 
   /**
    * Get the row number of the row that will be returned by the following

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/TypeDescription.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/TypeDescription.java b/orc/src/java/org/apache/orc/TypeDescription.java
index b8e057e..bd900ac 100644
--- a/orc/src/java/org/apache/orc/TypeDescription.java
+++ b/orc/src/java/org/apache/orc/TypeDescription.java
@@ -61,7 +61,7 @@ public class TypeDescription {
     LIST("array", false),
     MAP("map", false),
     STRUCT("struct", false),
-    UNION("uniontype", false);
+    UNION("union", false);
 
     Category(String name, boolean isPrimitive) {
       this.name = name;
@@ -258,66 +258,6 @@ public class TypeDescription {
     return id;
   }
 
-  public TypeDescription clone() {
-    TypeDescription result = new TypeDescription(category);
-    result.maxLength = maxLength;
-    result.precision = precision;
-    result.scale = scale;
-    if (fieldNames != null) {
-      result.fieldNames.addAll(fieldNames);
-    }
-    if (children != null) {
-      for(TypeDescription child: children) {
-        TypeDescription clone = child.clone();
-        clone.parent = result;
-        result.children.add(clone);
-      }
-    }
-    return result;
-  }
-
-  @Override
-  public int hashCode() {
-    return getId();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == null || other.getClass() != TypeDescription.class) {
-      return false;
-    }
-    if (other == this) {
-      return true;
-    }
-    TypeDescription castOther = (TypeDescription) other;
-    if (category != castOther.category ||
-        getId() != castOther.getId() ||
-        getMaximumId() != castOther.getMaximumId() ||
-        maxLength != castOther.maxLength ||
-        scale != castOther.scale ||
-        precision != castOther.precision) {
-      return false;
-    }
-    if (children != null) {
-      if (children.size() != castOther.children.size()) {
-        return false;
-      }
-      for (int i = 0; i < children.size(); ++i) {
-        if (!children.get(i).equals(castOther.children.get(i))) {
-          return false;
-        }
-      }
-    }
-    if (category == Category.STRUCT) {
-      for(int i=0; i < fieldNames.size(); ++i) {
-        if (!fieldNames.get(i).equals(castOther.fieldNames.get(i))) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
   /**
    * Get the maximum id assigned to this type or its children.
    * The first call will cause all of the the ids in tree to be assigned, so

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/impl/BitFieldReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/BitFieldReader.java b/orc/src/java/org/apache/orc/impl/BitFieldReader.java
index dda7355..8d9d3cb 100644
--- a/orc/src/java/org/apache/orc/impl/BitFieldReader.java
+++ b/orc/src/java/org/apache/orc/impl/BitFieldReader.java
@@ -137,7 +137,7 @@ public class BitFieldReader {
                          long previousLen) throws IOException {
     previous.isRepeating = true;
     for (int i = 0; i < previousLen; i++) {
-      if (previous.noNulls || !previous.isNull[i]) {
+      if (!previous.isNull[i]) {
         previous.vector[i] = next();
       } else {
         // The default value of null for int types in vectorized
@@ -150,8 +150,7 @@ public class BitFieldReader {
       // when determining the isRepeating flag.
       if (previous.isRepeating
           && i > 0
-          && ((previous.vector[0] != previous.vector[i]) ||
-          (previous.isNull[0] != previous.isNull[i]))) {
+          && ((previous.vector[i - 1] != previous.vector[i]) || (previous.isNull[i - 1] != previous.isNull[i]))) {
         previous.isRepeating = false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/impl/IntegerReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/IntegerReader.java b/orc/src/java/org/apache/orc/impl/IntegerReader.java
index 8bef0f1..7dfd289 100644
--- a/orc/src/java/org/apache/orc/impl/IntegerReader.java
+++ b/orc/src/java/org/apache/orc/impl/IntegerReader.java
@@ -20,7 +20,7 @@ package org.apache.orc.impl;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 
 /**
  * Interface for reading integers.
@@ -57,25 +57,9 @@ public interface IntegerReader {
 
   /**
    * Return the next available vector for values.
-   * @param column the column being read
-   * @param data the vector to read into
-   * @param length the number of numbers to read
-   * @throws IOException
-   */
-   void nextVector(ColumnVector column,
-                   long[] data,
-                   int length
-                   ) throws IOException;
-
-  /**
-   * Return the next available vector for values. Does not change the
-   * value of column.isRepeating.
-   * @param column the column being read
-   * @param data the vector to read into
-   * @param length the number of numbers to read
+   * @return
    * @throws IOException
    */
-  void nextVector(ColumnVector column,
-                  int[] data,
-                  int length
-                  ) throws IOException;}
+   void nextVector(LongColumnVector previous, final int previousLen)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/impl/RunLengthByteReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/RunLengthByteReader.java b/orc/src/java/org/apache/orc/impl/RunLengthByteReader.java
index 24bd051..380f3391 100644
--- a/orc/src/java/org/apache/orc/impl/RunLengthByteReader.java
+++ b/orc/src/java/org/apache/orc/impl/RunLengthByteReader.java
@@ -20,7 +20,7 @@ package org.apache.orc.impl;
 import java.io.EOFException;
 import java.io.IOException;
 
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 
 /**
  * A reader that reads a sequence of bytes. A control byte is read before
@@ -92,16 +92,16 @@ public class RunLengthByteReader {
     return result;
   }
 
-  public void nextVector(ColumnVector previous, long[] data, long size)
+  public void nextVector(LongColumnVector previous, long previousLen)
       throws IOException {
     previous.isRepeating = true;
-    for (int i = 0; i < size; i++) {
+    for (int i = 0; i < previousLen; i++) {
       if (!previous.isNull[i]) {
-        data[i] = next();
+        previous.vector[i] = next();
       } else {
         // The default value of null for int types in vectorized
         // processing is 1, so set that if the value is null
-        data[i] = 1;
+        previous.vector[i] = 1;
       }
 
       // The default value for nulls in Vectorization for int types is 1
@@ -109,36 +109,12 @@ public class RunLengthByteReader {
       // when determining the isRepeating flag.
       if (previous.isRepeating
           && i > 0
-          && ((data[0] != data[i]) ||
-              (previous.isNull[0] != previous.isNull[i]))) {
+          && ((previous.vector[i - 1] != previous.vector[i]) || (previous.isNull[i - 1] != previous.isNull[i]))) {
         previous.isRepeating = false;
       }
     }
   }
 
-  /**
-   * Read the next size bytes into the data array, skipping over any slots
-   * where isNull is true.
-   * @param isNull if non-null, skip any rows where isNull[r] is true
-   * @param data the array to read into
-   * @param size the number of elements to read
-   * @throws IOException
-   */
-  public void nextVector(boolean[] isNull, int[] data,
-                         long size) throws IOException {
-    if (isNull == null) {
-      for(int i=0; i < size; ++i) {
-        data[i] = next();
-      }
-    } else {
-      for(int i=0; i < size; ++i) {
-        if (!isNull[i]) {
-          data[i] = next();
-        }
-      }
-    }
-  }
-
   public void seek(PositionProvider index) throws IOException {
     input.seek(index);
     int consumed = (int) index.getNext();

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java b/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java
index b91a263..0c90cde 100644
--- a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java
+++ b/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java
@@ -20,7 +20,7 @@ package org.apache.orc.impl;
 import java.io.EOFException;
 import java.io.IOException;
 
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 
 /**
  * A reader that reads a sequence of integers.
@@ -99,17 +99,15 @@ public class RunLengthIntegerReader implements IntegerReader {
   }
 
   @Override
-  public void nextVector(ColumnVector previous,
-                         long[] data,
-                         int previousLen) throws IOException {
+  public void nextVector(LongColumnVector previous, final int previousLen) throws IOException {
     previous.isRepeating = true;
     for (int i = 0; i < previousLen; i++) {
       if (!previous.isNull[i]) {
-        data[i] = next();
+        previous.vector[i] = next();
       } else {
         // The default value of null for int type in vectorized
         // processing is 1, so set that if the value is null
-        data[i] = 1;
+        previous.vector[i] = 1;
       }
 
       // The default value for nulls in Vectorization for int types is 1
@@ -117,32 +115,13 @@ public class RunLengthIntegerReader implements IntegerReader {
       // when determining the isRepeating flag.
       if (previous.isRepeating
           && i > 0
-          && (data[0] != data[i] || previous.isNull[0] != previous.isNull[i])) {
+          && (previous.vector[i - 1] != previous.vector[i] || previous.isNull[i - 1] != previous.isNull[i])) {
         previous.isRepeating = false;
       }
     }
   }
 
   @Override
-  public void nextVector(ColumnVector vector,
-                         int[] data,
-                         int size) throws IOException {
-    if (vector.noNulls) {
-      for(int r=0; r < data.length && r < size; ++r) {
-        data[r] = (int) next();
-      }
-    } else if (!(vector.isRepeating && vector.isNull[0])) {
-      for(int r=0; r < data.length && r < size; ++r) {
-        if (!vector.isNull[r]) {
-          data[r] = (int) next();
-        } else {
-          data[r] = 1;
-        }
-      }
-    }
-  }
-
-  @Override
   public void seek(PositionProvider index) throws IOException {
     input.seek(index);
     int consumed = (int) index.getNext();

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java b/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java
index 610d9b5..c6d685a 100644
--- a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java
+++ b/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java
@@ -21,9 +21,9 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 
 /**
  * A reader that reads a sequence of light weight compressed integers. Refer
@@ -360,17 +360,15 @@ public class RunLengthIntegerReaderV2 implements IntegerReader {
   }
 
   @Override
-  public void nextVector(ColumnVector previous,
-                         long[] data,
-                         int previousLen) throws IOException {
+  public void nextVector(LongColumnVector previous, final int previousLen) throws IOException {
     previous.isRepeating = true;
     for (int i = 0; i < previousLen; i++) {
       if (!previous.isNull[i]) {
-        data[i] = next();
+        previous.vector[i] = next();
       } else {
         // The default value of null for int type in vectorized
         // processing is 1, so set that if the value is null
-        data[i] = 1;
+        previous.vector[i] = 1;
       }
 
       // The default value for nulls in Vectorization for int types is 1
@@ -378,29 +376,10 @@ public class RunLengthIntegerReaderV2 implements IntegerReader {
       // when determining the isRepeating flag.
       if (previous.isRepeating
           && i > 0
-          && (data[0] != data[i] ||
-          previous.isNull[0] != previous.isNull[i])) {
+          && (previous.vector[i - 1] != previous.vector[i] ||
+          previous.isNull[i - 1] != previous.isNull[i])) {
         previous.isRepeating = false;
       }
     }
   }
-
-  @Override
-  public void nextVector(ColumnVector vector,
-                         int[] data,
-                         int size) throws IOException {
-    if (vector.noNulls) {
-      for(int r=0; r < data.length && r < size; ++r) {
-        data[r] = (int) next();
-      }
-    } else if (!(vector.isRepeating && vector.isNull[0])) {
-      for(int r=0; r < data.length && r < size; ++r) {
-        if (!vector.isNull[r]) {
-          data[r] = (int) next();
-        } else {
-          data[r] = 1;
-        }
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/orc/src/java/org/apache/orc/impl/WriterImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/WriterImpl.java b/orc/src/java/org/apache/orc/impl/WriterImpl.java
index b2966e0..f8afe06 100644
--- a/orc/src/java/org/apache/orc/impl/WriterImpl.java
+++ b/orc/src/java/org/apache/orc/impl/WriterImpl.java
@@ -1693,10 +1693,9 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     }
   }
 
-  public static long MILLIS_PER_DAY = 24 * 60 * 60 * 1000;
-  public static long NANOS_PER_MILLI = 1000000;
   public static final int MILLIS_PER_SECOND = 1000;
   static final int NANOS_PER_SECOND = 1000000000;
+  static final int MILLIS_PER_NANO  = 1000000;
   public static final String BASE_TIMESTAMP_STRING = "2015-01-01 00:00:00";
 
   private static class TimestampTreeWriter extends TreeWriter {
@@ -2262,36 +2261,32 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
         }
       } else {
         // write the records in runs of the same tag
-        int[] currentStart = new int[vec.fields.length];
-        int[] currentLength = new int[vec.fields.length];
+        byte prevTag = 0;
+        int currentRun = 0;
+        boolean started = false;
         for(int i=0; i < length; ++i) {
-          // only need to deal with the non-nulls, since the nulls were dealt
-          // with in the super method.
-          if (vec.noNulls || !vec.isNull[i + offset]) {
+          if (!vec.isNull[i + offset]) {
             byte tag = (byte) vec.tags[offset + i];
             tags.write(tag);
-            if (currentLength[tag] == 0) {
-              // start a new sequence
-              currentStart[tag] = i + offset;
-              currentLength[tag] = 1;
-            } else if (currentStart[tag] + currentLength[tag] == i + offset) {
-              // ok, we are extending the current run for that tag.
-              currentLength[tag] += 1;
-            } else {
-              // otherwise, we need to close off the old run and start a new one
-              childrenWriters[tag].writeBatch(vec.fields[tag],
-                  currentStart[tag], currentLength[tag]);
-              currentStart[tag] = i + offset;
-              currentLength[tag] = 1;
+            if (!started) {
+              started = true;
+              currentRun = i;
+              prevTag = tag;
+            } else if (tag != prevTag) {
+              childrenWriters[prevTag].writeBatch(vec.fields[prevTag],
+                  offset + currentRun, i - currentRun);
+              currentRun = i;
+              prevTag = tag;
             }
+          } else if (started) {
+            started = false;
+            childrenWriters[prevTag].writeBatch(vec.fields[prevTag],
+                offset + currentRun, i - currentRun);
           }
         }
-        // write out any left over sequences
-        for(int tag=0; tag < currentStart.length; ++tag) {
-          if (currentLength[tag] != 0) {
-            childrenWriters[tag].writeBatch(vec.fields[tag], currentStart[tag],
-                currentLength[tag]);
-          }
+        if (started) {
+          childrenWriters[prevTag].writeBatch(vec.fields[prevTag],
+              offset + currentRun, length - currentRun);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
index 82a97e0..0724191 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
@@ -215,9 +215,12 @@ public class VectorizedRowBatchCtx {
     LOG.info("createVectorizedRowBatch columnsToIncludeTruncated " + Arrays.toString(columnsToIncludeTruncated));
     int totalColumnCount = rowColumnTypeInfos.length + scratchColumnTypeNames.length;
     VectorizedRowBatch result = new VectorizedRowBatch(totalColumnCount);
-    for (int i = 0; i < dataColumnCount; i++) {
-      TypeInfo typeInfo = rowColumnTypeInfos[i];
-      result.cols[i] = VectorizedBatchUtil.createColumnVector(typeInfo);
+
+    for (int i = 0; i < columnsToIncludeTruncated.length; i++) {
+      if (columnsToIncludeTruncated[i]) {
+        TypeInfo typeInfo = rowColumnTypeInfos[i];
+        result.cols[i] = VectorizedBatchUtil.createColumnVector(typeInfo);
+      }
     }
 
     for (int i = dataColumnCount; i < dataColumnCount + partitionColumnCount; i++) {
@@ -473,8 +476,8 @@ public class VectorizedRowBatchCtx {
             bcv.isNull[0] = true;
             bcv.isRepeating = true;
           } else {
-            bcv.setVal(0, sVal.getBytes());
-            bcv.isRepeating = true;
+            bcv.fill(sVal.getBytes());
+            bcv.isNull[0] = false;
           }
         }
         break;

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index fcb8ca4..fe0be7b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -301,7 +301,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     /**
      * Do we have schema on read in the configuration variables?
      */
-    TypeDescription schema = getDesiredRowTypeDescr(conf, false, Integer.MAX_VALUE);
+    TypeDescription schema = getDesiredRowTypeDescr(conf, /* isAcidRead */ false);
 
     Reader.Options options = new Reader.Options().range(offset, length);
     options.schema(schema);
@@ -1743,7 +1743,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     /**
      * Do we have schema on read in the configuration variables?
      */
-    TypeDescription schema = getDesiredRowTypeDescr(conf, true, Integer.MAX_VALUE);
+    TypeDescription schema = getDesiredRowTypeDescr(conf, /* isAcidRead */ true);
 
     final Reader reader;
     final int bucket;
@@ -1994,13 +1994,10 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
   /**
    * Convert a Hive type property string that contains separated type names into a list of
    * TypeDescription objects.
-   * @param hiveTypeProperty the desired types from hive
-   * @param maxColumns the maximum number of desired columns
    * @return the list of TypeDescription objects.
    */
-  public static ArrayList<TypeDescription>
-      typeDescriptionsFromHiveTypeProperty(String hiveTypeProperty,
-                                           int maxColumns) {
+  public static ArrayList<TypeDescription> typeDescriptionsFromHiveTypeProperty(
+      String hiveTypeProperty) {
 
     // CONSDIER: We need a type name parser for TypeDescription.
 
@@ -2008,9 +2005,6 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     ArrayList<TypeDescription> typeDescrList =new ArrayList<TypeDescription>(typeInfoList.size());
     for (TypeInfo typeInfo : typeInfoList) {
       typeDescrList.add(convertTypeInfo(typeInfo));
-      if (typeDescrList.size() >= maxColumns) {
-        break;
-      }
     }
     return typeDescrList;
   }
@@ -2097,18 +2091,8 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     }
   }
 
-  /**
-   * Generate the desired schema for reading the file.
-   * @param conf the configuration
-   * @param isAcidRead is this an acid format?
-   * @param dataColumns the desired number of data columns for vectorized read
-   * @return the desired schema or null if schema evolution isn't enabled
-   * @throws IOException
-   */
-  public static TypeDescription getDesiredRowTypeDescr(Configuration conf,
-                                                       boolean isAcidRead,
-                                                       int dataColumns
-                                                       ) throws IOException {
+  public static TypeDescription getDesiredRowTypeDescr(Configuration conf, boolean isAcidRead)
+      throws IOException {
 
     String columnNameProperty = null;
     String columnTypeProperty = null;
@@ -2131,10 +2115,8 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
           haveSchemaEvolutionProperties = false;
         } else {
           schemaEvolutionTypeDescrs =
-              typeDescriptionsFromHiveTypeProperty(columnTypeProperty,
-                  dataColumns);
-          if (schemaEvolutionTypeDescrs.size() !=
-              Math.min(dataColumns, schemaEvolutionColumnNames.size())) {
+              typeDescriptionsFromHiveTypeProperty(columnTypeProperty);
+          if (schemaEvolutionTypeDescrs.size() != schemaEvolutionColumnNames.size()) {
             haveSchemaEvolutionProperties = false;
           }
         }
@@ -2165,9 +2147,8 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
         return null;
       }
       schemaEvolutionTypeDescrs =
-          typeDescriptionsFromHiveTypeProperty(columnTypeProperty, dataColumns);
-      if (schemaEvolutionTypeDescrs.size() !=
-          Math.min(dataColumns, schemaEvolutionColumnNames.size())) {
+          typeDescriptionsFromHiveTypeProperty(columnTypeProperty);
+      if (schemaEvolutionTypeDescrs.size() != schemaEvolutionColumnNames.size()) {
         return null;
       }
 
@@ -2181,7 +2162,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
         }
         columnNum++;
       }
-      if (virtualColumnClipNum != -1 && virtualColumnClipNum < dataColumns) {
+      if (virtualColumnClipNum != -1) {
         schemaEvolutionColumnNames =
             Lists.newArrayList(schemaEvolutionColumnNames.subList(0, virtualColumnClipNum));
         schemaEvolutionTypeDescrs = Lists.newArrayList(schemaEvolutionTypeDescrs.subList(0, virtualColumnClipNum));
@@ -2198,7 +2179,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
 
     // Desired schema does not include virtual columns or partition columns.
     TypeDescription result = TypeDescription.createStruct();
-    for (int i = 0; i < schemaEvolutionTypeDescrs.size(); i++) {
+    for (int i = 0; i < schemaEvolutionColumnNames.size(); i++) {
       result.addField(schemaEvolutionColumnNames.get(i), schemaEvolutionTypeDescrs.get(i));
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
index 0dd58b7..1fce282 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
@@ -447,8 +447,7 @@ public class OrcRawRecordMerger implements AcidInputFormat.RawReader<OrcStruct>{
     this.length = options.getLength();
     this.validTxnList = validTxnList;
 
-    TypeDescription typeDescr =
-        OrcInputFormat.getDesiredRowTypeDescr(conf, true, Integer.MAX_VALUE);
+    TypeDescription typeDescr = OrcInputFormat.getDesiredRowTypeDescr(conf, /* isAcidRead */ true);
 
     objectInspector = OrcRecordUpdater.createEventSchema
         (OrcStruct.createObjectInspector(0, OrcUtils.getOrcTypes(typeDescr)));

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
index 0bcf9e3..a031a92 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
@@ -26,8 +26,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.orc.OrcUtils;
-import org.apache.orc.TypeDescription;
 import org.apache.orc.impl.BufferChunk;
 import org.apache.orc.ColumnStatistics;
 import org.apache.orc.impl.ColumnStatisticsImpl;
@@ -73,7 +71,6 @@ public class ReaderImpl implements Reader {
   private final List<OrcProto.StripeStatistics> stripeStats;
   private final int metadataSize;
   protected final List<OrcProto.Type> types;
-  private final TypeDescription schema;
   private final List<OrcProto.UserMetadataItem> userMetadata;
   private final List<OrcProto.ColumnStatistics> fileStats;
   private final List<StripeInformation> stripes;
@@ -246,11 +243,6 @@ public class ReaderImpl implements Reader {
     return result;
   }
 
-  @Override
-  public TypeDescription getSchema() {
-    return schema;
-  }
-
   /**
    * Ensure this is an ORC file to prevent users from trying to read text
    * files or RC files as ORC files.
@@ -394,9 +386,7 @@ public class ReaderImpl implements Reader {
       this.writerVersion = footerMetaData.writerVersion;
       this.stripes = convertProtoStripesToStripes(rInfo.footer.getStripesList());
     }
-    this.schema = OrcUtils.convertTypeFromProtobuf(this.types, 0);
   }
-
   /**
    * Get the WriterVersion based on the ORC file postscript.
    * @param writerVersion the integer writer version
@@ -678,7 +668,7 @@ public class ReaderImpl implements Reader {
       options.include(include);
     }
     return new RecordReaderImpl(this.getStripes(), fileSystem, path,
-        options, schema, types, codec, bufferSize, rowIndexStride, conf);
+        options, types, codec, bufferSize, rowIndexStride, conf);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index c214658..3975409 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -27,8 +27,9 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.orc.BooleanColumnStatistics;
-import org.apache.orc.TypeDescription;
+import org.apache.orc.OrcUtils;
 import org.apache.orc.impl.BufferChunk;
 import org.apache.orc.ColumnStatistics;
 import org.apache.orc.impl.ColumnStatisticsImpl;
@@ -57,6 +58,7 @@ import org.apache.hadoop.hive.common.io.DiskRange;
 import org.apache.hadoop.hive.common.io.DiskRangeList;
 import org.apache.hadoop.hive.common.io.DiskRangeList.CreateHelper;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.orc.BloomFilterIO;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
@@ -96,6 +98,7 @@ public class RecordReaderImpl implements RecordReader {
   private final SargApplier sargApp;
   // an array about which row groups aren't skipped
   private boolean[] includedRowGroups = null;
+  private final Configuration conf;
   private final MetadataReader metadata;
   private final DataReader dataReader;
 
@@ -142,33 +145,33 @@ public class RecordReaderImpl implements RecordReader {
                              FileSystem fileSystem,
                              Path path,
                              Reader.Options options,
-                             TypeDescription fileSchema,
                              List<OrcProto.Type> types,
                              CompressionCodec codec,
                              int bufferSize,
                              long strideRate,
                              Configuration conf
                              ) throws IOException {
-    SchemaEvolution treeReaderSchema;
-    this.included = options.getInclude();
-    included[0] = true;
+
+    TreeReaderFactory.TreeReaderSchema treeReaderSchema;
     if (options.getSchema() == null) {
       if (LOG.isInfoEnabled()) {
         LOG.info("Schema on read not provided -- using file schema " + types.toString());
       }
-      treeReaderSchema = new SchemaEvolution(fileSchema, included);
+      treeReaderSchema = new TreeReaderFactory.TreeReaderSchema().fileTypes(types).schemaTypes(types);
     } else {
 
       // Now that we are creating a record reader for a file, validate that the schema to read
       // is compatible with the file schema.
       //
-      treeReaderSchema = new SchemaEvolution(fileSchema, options.getSchema(),
-          included);
+      List<OrcProto.Type> schemaTypes = OrcUtils.getOrcTypes(options.getSchema());
+      treeReaderSchema = SchemaEvolution.validateAndCreate(types, schemaTypes);
     }
     this.path = path;
     this.codec = codec;
     this.types = types;
     this.bufferSize = bufferSize;
+    this.included = options.getInclude();
+    this.conf = conf;
     this.rowIndexStride = strideRate;
     this.metadata = new MetadataReaderImpl(fileSystem, path, codec, bufferSize, types.size());
     SearchArgument sarg = options.getSearchArgument();
@@ -207,8 +210,7 @@ public class RecordReaderImpl implements RecordReader {
       skipCorrupt = OrcConf.SKIP_CORRUPT_DATA.getBoolean(conf);
     }
 
-    reader = TreeReaderFactory.createTreeReader(treeReaderSchema.getReaderSchema(),
-        treeReaderSchema, included, skipCorrupt);
+    reader = TreeReaderFactory.createTreeReader(0, treeReaderSchema, included, skipCorrupt);
     indexes = new OrcProto.RowIndex[types.size()];
     bloomFilterIndices = new OrcProto.BloomFilterIndex[types.size()];
     advanceToNextRow(reader, 0L, true);
@@ -237,7 +239,7 @@ public class RecordReaderImpl implements RecordReader {
     return metadata.readStripeFooter(stripe);
   }
 
-  enum Location {
+  static enum Location {
     BEFORE, MIN, MIDDLE, MAX, AFTER
   }
 
@@ -1050,27 +1052,31 @@ public class RecordReaderImpl implements RecordReader {
   }
 
   @Override
-  public boolean nextBatch(VectorizedRowBatch batch) throws IOException {
+  public VectorizedRowBatch nextBatch(VectorizedRowBatch previous) throws IOException {
     try {
+      final VectorizedRowBatch result;
       if (rowInStripe >= rowCountInStripe) {
         currentStripe += 1;
-        if (currentStripe >= stripes.size()) {
-          batch.size = 0;
-          return false;
-        }
         readStripe();
       }
 
-      int batchSize = computeBatchSize(batch.getMaxSize());
+      final int batchSize = computeBatchSize(VectorizedRowBatch.DEFAULT_SIZE);
 
       rowInStripe += batchSize;
-      reader.setVectorColumnCount(batch.getDataColumnCount());
-      reader.nextBatch(batch, batchSize);
+      if (previous == null) {
+        ColumnVector[] cols = (ColumnVector[]) reader.nextVector(null, (int) batchSize);
+        result = new VectorizedRowBatch(cols.length);
+        result.cols = cols;
+      } else {
+        result = previous;
+        result.selectedInUse = false;
+        reader.setVectorColumnCount(result.getDataColumnCount());
+        reader.nextVector(result.cols, batchSize);
+      }
 
-      batch.size = (int) batchSize;
-      batch.selectedInUse = false;
+      result.size = batchSize;
       advanceToNextRow(reader, rowInStripe + rowBaseInStripe, true);
-      return batch.size  != 0;
+      return result;
     } catch (IOException e) {
       // Rethrow exception with file name in log message
       throw new IOException("Error reading file: " + path, e);

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java
index 6747691..f28ca13 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java
@@ -20,12 +20,13 @@ package org.apache.hadoop.hive.ql.io.orc;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.io.orc.TreeReaderFactory.TreeReaderSchema;
+import org.apache.orc.OrcProto;
+import org.apache.orc.OrcUtils;
 import org.apache.orc.TypeDescription;
 
 /**
@@ -33,134 +34,103 @@ import org.apache.orc.TypeDescription;
  * has been schema evolution.
  */
 public class SchemaEvolution {
-  private final Map<TypeDescription, TypeDescription> readerToFile;
-  private final boolean[] included;
-  private final TypeDescription readerSchema;
+
   private static final Log LOG = LogFactory.getLog(SchemaEvolution.class);
 
-  public SchemaEvolution(TypeDescription readerSchema, boolean[] included) {
-    this.included = included;
-    readerToFile = null;
-    this.readerSchema = readerSchema;
-  }
+  public static TreeReaderSchema validateAndCreate(List<OrcProto.Type> fileTypes,
+      List<OrcProto.Type> schemaTypes) throws IOException {
 
-  public SchemaEvolution(TypeDescription fileSchema,
-                         TypeDescription readerSchema,
-                         boolean[] included) throws IOException {
-    readerToFile = new HashMap<>(readerSchema.getMaximumId() + 1);
-    this.included = included;
-    if (checkAcidSchema(fileSchema)) {
-      this.readerSchema = createEventSchema(readerSchema);
+    // For ACID, the row is the ROW field in the outer STRUCT.
+    final boolean isAcid = checkAcidSchema(fileTypes);
+    final List<OrcProto.Type> rowSchema;
+    int rowSubtype;
+    if (isAcid) {
+      rowSubtype = OrcRecordUpdater.ROW + 1;
+      rowSchema = fileTypes.subList(rowSubtype, fileTypes.size());
     } else {
-      this.readerSchema = readerSchema;
+      rowSubtype = 0;
+      rowSchema = fileTypes;
     }
-    buildMapping(fileSchema, this.readerSchema);
-  }
 
-  public TypeDescription getReaderSchema() {
-    return readerSchema;
-  }
+    // Do checking on the overlap.  Additional columns will be defaulted to NULL.
 
-  public TypeDescription getFileType(TypeDescription readerType) {
-    TypeDescription result;
-    if (readerToFile == null) {
-      if (included == null || included[readerType.getId()]) {
-        result = readerType;
-      } else {
-        result = null;
-      }
-    } else {
-      result = readerToFile.get(readerType);
-    }
-    return result;
-  }
+    int numFileColumns = rowSchema.get(0).getSubtypesCount();
+    int numDesiredColumns = schemaTypes.get(0).getSubtypesCount();
 
-  void buildMapping(TypeDescription fileType,
-                    TypeDescription readerType) throws IOException {
-    // if the column isn't included, don't map it
-    if (included != null && !included[readerType.getId()]) {
-      return;
-    }
-    boolean isOk = true;
-    // check the easy case first
-    if (fileType.getCategory() == readerType.getCategory()) {
-      switch (readerType.getCategory()) {
-        case BOOLEAN:
-        case BYTE:
-        case SHORT:
-        case INT:
-        case LONG:
-        case DOUBLE:
-        case FLOAT:
-        case STRING:
-        case TIMESTAMP:
-        case BINARY:
-        case DATE:
-          // these are always a match
-          break;
-        case CHAR:
-        case VARCHAR:
-          isOk = fileType.getMaxLength() == readerType.getMaxLength();
-          break;
-        case DECIMAL:
-          // TODO we don't enforce scale and precision checks, but probably should
-          break;
-        case UNION:
-        case MAP:
-        case LIST: {
-          // these must be an exact match
-          List<TypeDescription> fileChildren = fileType.getChildren();
-          List<TypeDescription> readerChildren = readerType.getChildren();
-          if (fileChildren.size() == readerChildren.size()) {
-            for(int i=0; i < fileChildren.size(); ++i) {
-              buildMapping(fileChildren.get(i), readerChildren.get(i));
-            }
-          } else {
-            isOk = false;
+    int numReadColumns = Math.min(numFileColumns, numDesiredColumns);
+
+    /**
+     * Check type promotion.
+     *
+     * Currently, we only support integer type promotions that can be done "implicitly".
+     * That is, we know that using a bigger integer tree reader on the original smaller integer
+     * column will "just work".
+     *
+     * In the future, other type promotions might require type conversion.
+     */
+    // short -> int -> bigint as same integer readers are used for the above types.
+
+    for (int i = 0; i < numReadColumns; i++) {
+      OrcProto.Type fColType = fileTypes.get(rowSubtype + i);
+      OrcProto.Type rColType = schemaTypes.get(i);
+      if (!fColType.getKind().equals(rColType.getKind())) {
+
+        boolean ok = false;
+        if (fColType.getKind().equals(OrcProto.Type.Kind.SHORT)) {
+
+          if (rColType.getKind().equals(OrcProto.Type.Kind.INT) ||
+              rColType.getKind().equals(OrcProto.Type.Kind.LONG)) {
+            // type promotion possible, converting SHORT to INT/LONG requested type
+            ok = true;
           }
-          break;
-        }
-        case STRUCT: {
-          // allow either side to have fewer fields than the other
-          List<TypeDescription> fileChildren = fileType.getChildren();
-          List<TypeDescription> readerChildren = readerType.getChildren();
-          int jointSize = Math.min(fileChildren.size(), readerChildren.size());
-          for(int i=0; i < jointSize; ++i) {
-            buildMapping(fileChildren.get(i), readerChildren.get(i));
+        } else if (fColType.getKind().equals(OrcProto.Type.Kind.INT)) {
+
+          if (rColType.getKind().equals(OrcProto.Type.Kind.LONG)) {
+            // type promotion possible, converting INT to LONG requested type
+            ok = true;
           }
-          break;
         }
-        default:
-          throw new IllegalArgumentException("Unknown type " + readerType);
-      }
-    } else {
-      switch (fileType.getCategory()) {
-        case SHORT:
-          if (readerType.getCategory() != TypeDescription.Category.INT &&
-              readerType.getCategory() != TypeDescription.Category.LONG) {
-            isOk = false;
-          }
-          break;
-        case INT:
-          if (readerType.getCategory() != TypeDescription.Category.LONG) {
-            isOk = false;
-          }
-          break;
-        default:
-          isOk = false;
+
+        if (!ok) {
+          throw new IOException("ORC does not support type conversion from " +
+              fColType.getKind().name() + " to " + rColType.getKind().name());
+        }
       }
     }
-    if (isOk) {
-      readerToFile.put(readerType, fileType);
+
+    List<OrcProto.Type> fullSchemaTypes;
+
+    if (isAcid) {
+      fullSchemaTypes = new ArrayList<OrcProto.Type>();
+
+      // This copies the ACID struct type which is subtype = 0.
+      // It has field names "operation" through "row".
+      // And we copy the types for all fields EXCEPT ROW (which must be last!).
+
+      for (int i = 0; i < rowSubtype; i++) {
+        fullSchemaTypes.add(fileTypes.get(i).toBuilder().build());
+      }
+
+      // Add the row struct type.
+      OrcUtils.appendOrcTypesRebuildSubtypes(fullSchemaTypes, schemaTypes, 0);
     } else {
-      throw new IOException("ORC does not support type conversion from " +
-          fileType + " to " + readerType);
+      fullSchemaTypes = schemaTypes;
     }
+
+    int innerStructSubtype = rowSubtype;
+
+    // LOG.info("Schema evolution: (fileTypes) " + fileTypes.toString() +
+    //     " (schemaEvolutionTypes) " + schemaEvolutionTypes.toString());
+
+    return new TreeReaderSchema().
+        fileTypes(fileTypes).
+        schemaTypes(fullSchemaTypes).
+        innerStructSubtype(innerStructSubtype);
   }
 
-  private static boolean checkAcidSchema(TypeDescription type) {
-    if (type.getCategory().equals(TypeDescription.Category.STRUCT)) {
-      List<String> rootFields = type.getFieldNames();
+  private static boolean checkAcidSchema(List<OrcProto.Type> fileSchema) {
+    if (fileSchema.get(0).getKind().equals(OrcProto.Type.Kind.STRUCT)) {
+      List<String> rootFields = fileSchema.get(0).getFieldNamesList();
       if (acidEventFieldNames.equals(rootFields)) {
         return true;
       }
@@ -172,14 +142,26 @@ public class SchemaEvolution {
    * @param typeDescr
    * @return ORC types for the ACID event based on the row's type description
    */
-  public static TypeDescription createEventSchema(TypeDescription typeDescr) {
-    TypeDescription result = TypeDescription.createStruct()
-        .addField("operation", TypeDescription.createInt())
-        .addField("originalTransaction", TypeDescription.createLong())
-        .addField("bucket", TypeDescription.createInt())
-        .addField("rowId", TypeDescription.createLong())
-        .addField("currentTransaction", TypeDescription.createLong())
-        .addField("row", typeDescr.clone());
+  public static List<OrcProto.Type> createEventSchema(TypeDescription typeDescr) {
+
+    List<OrcProto.Type> result = new ArrayList<OrcProto.Type>();
+
+    OrcProto.Type.Builder type = OrcProto.Type.newBuilder();
+    type.setKind(OrcProto.Type.Kind.STRUCT);
+    type.addAllFieldNames(acidEventFieldNames);
+    for (int i = 0; i < acidEventFieldNames.size(); i++) {
+      type.addSubtypes(i + 1);
+    }
+    result.add(type.build());
+
+    // Automatically add all fields except the last (ROW).
+    for (int i = 0; i < acidEventOrcTypeKinds.size() - 1; i ++) {
+      type.clear();
+      type.setKind(acidEventOrcTypeKinds.get(i));
+      result.add(type.build());
+    }
+
+    OrcUtils.appendOrcTypesRebuildSubtypes(result, typeDescr);
     return result;
   }
 
@@ -192,4 +174,14 @@ public class SchemaEvolution {
     acidEventFieldNames.add("currentTransaction");
     acidEventFieldNames.add("row");
   }
+  public static final List<OrcProto.Type.Kind> acidEventOrcTypeKinds =
+      new ArrayList<OrcProto.Type.Kind>();
+  static {
+    acidEventOrcTypeKinds.add(OrcProto.Type.Kind.INT);
+    acidEventOrcTypeKinds.add(OrcProto.Type.Kind.LONG);
+    acidEventOrcTypeKinds.add(OrcProto.Type.Kind.INT);
+    acidEventOrcTypeKinds.add(OrcProto.Type.Kind.LONG);
+    acidEventOrcTypeKinds.add(OrcProto.Type.Kind.LONG);
+    acidEventOrcTypeKinds.add(OrcProto.Type.Kind.STRUCT);
+  }
 }


[19/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 13e30db..051c1f2 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
@@ -78,6 +78,8 @@ public class ThriftHiveMetastore {
 
     public void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException;
 
+    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException;
+
     public void drop_table(String dbname, String name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
 
     public void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext environment_context) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
@@ -196,6 +198,10 @@ public class ThriftHiveMetastore {
 
     public List<String> get_index_names(String db_name, String tbl_name, short max_indexes) throws MetaException, org.apache.thrift.TException;
 
+    public PrimaryKeysResponse get_primary_keys(PrimaryKeysRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
+
+    public ForeignKeysResponse get_foreign_keys(ForeignKeysRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
+
     public boolean update_table_column_statistics(ColumnStatistics stats_obj) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException;
 
     public boolean update_partition_column_statistics(ColumnStatistics stats_obj) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException;
@@ -368,6 +374,8 @@ public class ThriftHiveMetastore {
 
     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_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, 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 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;
@@ -486,6 +494,10 @@ public class ThriftHiveMetastore {
 
     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_primary_keys(PrimaryKeysRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_foreign_keys(ForeignKeysRequest request, 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 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;
@@ -1172,6 +1184,40 @@ public class ThriftHiveMetastore {
       return;
     }
 
+    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys);
+      recv_create_table_with_constraints();
+    }
+
+    public void send_create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) throws org.apache.thrift.TException
+    {
+      create_table_with_constraints_args args = new create_table_with_constraints_args();
+      args.setTbl(tbl);
+      args.setPrimaryKeys(primaryKeys);
+      args.setForeignKeys(foreignKeys);
+      sendBase("create_table_with_constraints", args);
+    }
+
+    public void recv_create_table_with_constraints() throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      create_table_with_constraints_result result = new create_table_with_constraints_result();
+      receiveBase(result, "create_table_with_constraints");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      if (result.o3 != null) {
+        throw result.o3;
+      }
+      if (result.o4 != null) {
+        throw result.o4;
+      }
+      return;
+    }
+
     public void drop_table(String dbname, String name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
     {
       send_drop_table(dbname, name, deleteData);
@@ -3018,6 +3064,64 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_index_names failed: unknown result");
     }
 
+    public PrimaryKeysResponse get_primary_keys(PrimaryKeysRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      send_get_primary_keys(request);
+      return recv_get_primary_keys();
+    }
+
+    public void send_get_primary_keys(PrimaryKeysRequest request) throws org.apache.thrift.TException
+    {
+      get_primary_keys_args args = new get_primary_keys_args();
+      args.setRequest(request);
+      sendBase("get_primary_keys", args);
+    }
+
+    public PrimaryKeysResponse recv_get_primary_keys() throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      get_primary_keys_result result = new get_primary_keys_result();
+      receiveBase(result, "get_primary_keys");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_primary_keys failed: unknown result");
+    }
+
+    public ForeignKeysResponse get_foreign_keys(ForeignKeysRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      send_get_foreign_keys(request);
+      return recv_get_foreign_keys();
+    }
+
+    public void send_get_foreign_keys(ForeignKeysRequest request) throws org.apache.thrift.TException
+    {
+      get_foreign_keys_args args = new get_foreign_keys_args();
+      args.setRequest(request);
+      sendBase("get_foreign_keys", args);
+    }
+
+    public ForeignKeysResponse recv_get_foreign_keys() throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      get_foreign_keys_result result = new get_foreign_keys_result();
+      receiveBase(result, "get_foreign_keys");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_foreign_keys failed: unknown result");
+    }
+
     public boolean update_table_column_statistics(ColumnStatistics stats_obj) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException
     {
       send_update_table_column_statistics(stats_obj);
@@ -5393,6 +5497,44 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      create_table_with_constraints_call method_call = new create_table_with_constraints_call(tbl, primaryKeys, foreignKeys, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class create_table_with_constraints_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private Table tbl;
+      private List<SQLPrimaryKey> primaryKeys;
+      private List<SQLForeignKey> foreignKeys;
+      public create_table_with_constraints_call(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, 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.tbl = tbl;
+        this.primaryKeys = primaryKeys;
+        this.foreignKeys = foreignKeys;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("create_table_with_constraints", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        create_table_with_constraints_args args = new create_table_with_constraints_args();
+        args.setTbl(tbl);
+        args.setPrimaryKeys(primaryKeys);
+        args.setForeignKeys(foreignKeys);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, 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_create_table_with_constraints();
+      }
+    }
+
     public void drop_table(String dbname, String name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       drop_table_call method_call = new drop_table_call(dbname, name, deleteData, resultHandler, this, ___protocolFactory, ___transport);
@@ -7659,6 +7801,70 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_primary_keys(PrimaryKeysRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_primary_keys_call method_call = new get_primary_keys_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_primary_keys_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private PrimaryKeysRequest request;
+      public get_primary_keys_call(PrimaryKeysRequest request, 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.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_primary_keys", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_primary_keys_args args = new get_primary_keys_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public PrimaryKeysResponse getResult() throws MetaException, NoSuchObjectException, 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_primary_keys();
+      }
+    }
+
+    public void get_foreign_keys(ForeignKeysRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_foreign_keys_call method_call = new get_foreign_keys_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_foreign_keys_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private ForeignKeysRequest request;
+      public get_foreign_keys_call(ForeignKeysRequest request, 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.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_foreign_keys", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_foreign_keys_args args = new get_foreign_keys_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public ForeignKeysResponse getResult() throws MetaException, NoSuchObjectException, 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_foreign_keys();
+      }
+    }
+
     public void update_table_column_statistics(ColumnStatistics stats_obj, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       update_table_column_statistics_call method_call = new update_table_column_statistics_call(stats_obj, resultHandler, this, ___protocolFactory, ___transport);
@@ -9871,6 +10077,7 @@ public class ThriftHiveMetastore {
       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("create_table_with_constraints", new create_table_with_constraints());
       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());
@@ -9930,6 +10137,8 @@ public class ThriftHiveMetastore {
       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("get_primary_keys", new get_primary_keys());
+      processMap.put("get_foreign_keys", new get_foreign_keys());
       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());
@@ -10481,6 +10690,36 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class create_table_with_constraints<I extends Iface> extends org.apache.thrift.ProcessFunction<I, create_table_with_constraints_args> {
+      public create_table_with_constraints() {
+        super("create_table_with_constraints");
+      }
+
+      public create_table_with_constraints_args getEmptyArgsInstance() {
+        return new create_table_with_constraints_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public create_table_with_constraints_result getResult(I iface, create_table_with_constraints_args args) throws org.apache.thrift.TException {
+        create_table_with_constraints_result result = new create_table_with_constraints_result();
+        try {
+          iface.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys);
+        } catch (AlreadyExistsException o1) {
+          result.o1 = o1;
+        } catch (InvalidObjectException o2) {
+          result.o2 = o2;
+        } catch (MetaException o3) {
+          result.o3 = o3;
+        } catch (NoSuchObjectException o4) {
+          result.o4 = o4;
+        }
+        return result;
+      }
+    }
+
     public static class drop_table<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_table_args> {
       public drop_table() {
         super("drop_table");
@@ -12055,6 +12294,58 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_primary_keys<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_primary_keys_args> {
+      public get_primary_keys() {
+        super("get_primary_keys");
+      }
+
+      public get_primary_keys_args getEmptyArgsInstance() {
+        return new get_primary_keys_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_primary_keys_result getResult(I iface, get_primary_keys_args args) throws org.apache.thrift.TException {
+        get_primary_keys_result result = new get_primary_keys_result();
+        try {
+          result.success = iface.get_primary_keys(args.request);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        } catch (NoSuchObjectException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    public static class get_foreign_keys<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_foreign_keys_args> {
+      public get_foreign_keys() {
+        super("get_foreign_keys");
+      }
+
+      public get_foreign_keys_args getEmptyArgsInstance() {
+        return new get_foreign_keys_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_foreign_keys_result getResult(I iface, get_foreign_keys_args args) throws org.apache.thrift.TException {
+        get_foreign_keys_result result = new get_foreign_keys_result();
+        try {
+          result.success = iface.get_foreign_keys(args.request);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        } catch (NoSuchObjectException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
     public static class update_table_column_statistics<I extends Iface> extends org.apache.thrift.ProcessFunction<I, update_table_column_statistics_args> {
       public update_table_column_statistics() {
         super("update_table_column_statistics");
@@ -13672,6 +13963,7 @@ public class ThriftHiveMetastore {
       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("create_table_with_constraints", new create_table_with_constraints());
       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());
@@ -13731,6 +14023,8 @@ public class ThriftHiveMetastore {
       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("get_primary_keys", new get_primary_keys());
+      processMap.put("get_foreign_keys", new get_foreign_keys());
       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());
@@ -14942,6 +15236,77 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class create_table_with_constraints<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_table_with_constraints_args, Void> {
+      public create_table_with_constraints() {
+        super("create_table_with_constraints");
+      }
+
+      public create_table_with_constraints_args getEmptyArgsInstance() {
+        return new create_table_with_constraints_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) {
+            create_table_with_constraints_result result = new create_table_with_constraints_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;
+            create_table_with_constraints_result result = new create_table_with_constraints_result();
+            if (e instanceof AlreadyExistsException) {
+                        result.o1 = (AlreadyExistsException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof InvalidObjectException) {
+                        result.o2 = (InvalidObjectException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof NoSuchObjectException) {
+                        result.o4 = (NoSuchObjectException) e;
+                        result.setO4IsSet(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, create_table_with_constraints_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys,resultHandler);
+      }
+    }
+
     public static class drop_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_args, Void> {
       public drop_table() {
         super("drop_table");
@@ -18673,6 +19038,130 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_primary_keys<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_primary_keys_args, PrimaryKeysResponse> {
+      public get_primary_keys() {
+        super("get_primary_keys");
+      }
+
+      public get_primary_keys_args getEmptyArgsInstance() {
+        return new get_primary_keys_args();
+      }
+
+      public AsyncMethodCallback<PrimaryKeysResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<PrimaryKeysResponse>() { 
+          public void onComplete(PrimaryKeysResponse o) {
+            get_primary_keys_result result = new get_primary_keys_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_primary_keys_result result = new get_primary_keys_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_primary_keys_args args, org.apache.thrift.async.AsyncMethodCallback<PrimaryKeysResponse> resultHandler) throws TException {
+        iface.get_primary_keys(args.request,resultHandler);
+      }
+    }
+
+    public static class get_foreign_keys<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_foreign_keys_args, ForeignKeysResponse> {
+      public get_foreign_keys() {
+        super("get_foreign_keys");
+      }
+
+      public get_foreign_keys_args getEmptyArgsInstance() {
+        return new get_foreign_keys_args();
+      }
+
+      public AsyncMethodCallback<ForeignKeysResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<ForeignKeysResponse>() { 
+          public void onComplete(ForeignKeysResponse o) {
+            get_foreign_keys_result result = new get_foreign_keys_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_foreign_keys_result result = new get_foreign_keys_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_foreign_keys_args args, org.apache.thrift.async.AsyncMethodCallback<ForeignKeysResponse> resultHandler) throws TException {
+        iface.get_foreign_keys(args.request,resultHandler);
+      }
+    }
+
     public static class update_table_column_statistics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_table_column_statistics_args, Boolean> {
       public update_table_column_statistics() {
         super("update_table_column_statistics");
@@ -27891,13 +28380,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list592 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list592.size);
-                  String _elem593;
-                  for (int _i594 = 0; _i594 < _list592.size; ++_i594)
+                  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)
                   {
-                    _elem593 = iprot.readString();
-                    struct.success.add(_elem593);
+                    _elem609 = iprot.readString();
+                    struct.success.add(_elem609);
                   }
                   iprot.readListEnd();
                 }
@@ -27932,9 +28421,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 _iter595 : struct.success)
+            for (String _iter611 : struct.success)
             {
-              oprot.writeString(_iter595);
+              oprot.writeString(_iter611);
             }
             oprot.writeListEnd();
           }
@@ -27973,9 +28462,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter596 : struct.success)
+            for (String _iter612 : struct.success)
             {
-              oprot.writeString(_iter596);
+              oprot.writeString(_iter612);
             }
           }
         }
@@ -27990,13 +28479,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list597 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list597.size);
-            String _elem598;
-            for (int _i599 = 0; _i599 < _list597.size; ++_i599)
+            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)
             {
-              _elem598 = iprot.readString();
-              struct.success.add(_elem598);
+              _elem614 = iprot.readString();
+              struct.success.add(_elem614);
             }
           }
           struct.setSuccessIsSet(true);
@@ -28650,13 +29139,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list616 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list616.size);
+                  String _elem617;
+                  for (int _i618 = 0; _i618 < _list616.size; ++_i618)
                   {
-                    _elem601 = iprot.readString();
-                    struct.success.add(_elem601);
+                    _elem617 = iprot.readString();
+                    struct.success.add(_elem617);
                   }
                   iprot.readListEnd();
                 }
@@ -28691,9 +29180,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 _iter603 : struct.success)
+            for (String _iter619 : struct.success)
             {
-              oprot.writeString(_iter603);
+              oprot.writeString(_iter619);
             }
             oprot.writeListEnd();
           }
@@ -28732,9 +29221,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter604 : struct.success)
+            for (String _iter620 : struct.success)
             {
-              oprot.writeString(_iter604);
+              oprot.writeString(_iter620);
             }
           }
         }
@@ -28749,13 +29238,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list621 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list621.size);
+            String _elem622;
+            for (int _i623 = 0; _i623 < _list621.size; ++_i623)
             {
-              _elem606 = iprot.readString();
-              struct.success.add(_elem606);
+              _elem622 = iprot.readString();
+              struct.success.add(_elem622);
             }
           }
           struct.setSuccessIsSet(true);
@@ -33362,16 +33851,16 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map608 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map608.size);
-                  String _key609;
-                  Type _val610;
-                  for (int _i611 = 0; _i611 < _map608.size; ++_i611)
+                  org.apache.thrift.protocol.TMap _map624 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map624.size);
+                  String _key625;
+                  Type _val626;
+                  for (int _i627 = 0; _i627 < _map624.size; ++_i627)
                   {
-                    _key609 = iprot.readString();
-                    _val610 = new Type();
-                    _val610.read(iprot);
-                    struct.success.put(_key609, _val610);
+                    _key625 = iprot.readString();
+                    _val626 = new Type();
+                    _val626.read(iprot);
+                    struct.success.put(_key625, _val626);
                   }
                   iprot.readMapEnd();
                 }
@@ -33406,10 +33895,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> _iter612 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter628 : struct.success.entrySet())
             {
-              oprot.writeString(_iter612.getKey());
-              _iter612.getValue().write(oprot);
+              oprot.writeString(_iter628.getKey());
+              _iter628.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -33448,10 +33937,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter613 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter629 : struct.success.entrySet())
             {
-              oprot.writeString(_iter613.getKey());
-              _iter613.getValue().write(oprot);
+              oprot.writeString(_iter629.getKey());
+              _iter629.getValue().write(oprot);
             }
           }
         }
@@ -33466,16 +33955,16 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map614 = 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*_map614.size);
-            String _key615;
-            Type _val616;
-            for (int _i617 = 0; _i617 < _map614.size; ++_i617)
+            org.apache.thrift.protocol.TMap _map630 = 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*_map630.size);
+            String _key631;
+            Type _val632;
+            for (int _i633 = 0; _i633 < _map630.size; ++_i633)
             {
-              _key615 = iprot.readString();
-              _val616 = new Type();
-              _val616.read(iprot);
-              struct.success.put(_key615, _val616);
+              _key631 = iprot.readString();
+              _val632 = new Type();
+              _val632.read(iprot);
+              struct.success.put(_key631, _val632);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34510,14 +34999,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list618.size);
-                  FieldSchema _elem619;
-                  for (int _i620 = 0; _i620 < _list618.size; ++_i620)
+                  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)
                   {
-                    _elem619 = new FieldSchema();
-                    _elem619.read(iprot);
-                    struct.success.add(_elem619);
+                    _elem635 = new FieldSchema();
+                    _elem635.read(iprot);
+                    struct.success.add(_elem635);
                   }
                   iprot.readListEnd();
                 }
@@ -34570,9 +35059,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 _iter621 : struct.success)
+            for (FieldSchema _iter637 : struct.success)
             {
-              _iter621.write(oprot);
+              _iter637.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -34627,9 +35116,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter622 : struct.success)
+            for (FieldSchema _iter638 : struct.success)
             {
-              _iter622.write(oprot);
+              _iter638.write(oprot);
             }
           }
         }
@@ -34650,14 +35139,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list623.size);
-            FieldSchema _elem624;
-            for (int _i625 = 0; _i625 < _list623.size; ++_i625)
+            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)
             {
-              _elem624 = new FieldSchema();
-              _elem624.read(iprot);
-              struct.success.add(_elem624);
+              _elem640 = new FieldSchema();
+              _elem640.read(iprot);
+              struct.success.add(_elem640);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35811,14 +36300,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  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)
                   {
-                    _elem627 = new FieldSchema();
-                    _elem627.read(iprot);
-                    struct.success.add(_elem627);
+                    _elem643 = new FieldSchema();
+                    _elem643.read(iprot);
+                    struct.success.add(_elem643);
                   }
                   iprot.readListEnd();
                 }
@@ -35871,9 +36360,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 _iter629 : struct.success)
+            for (FieldSchema _iter645 : struct.success)
             {
-              _iter629.write(oprot);
+              _iter645.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -35928,9 +36417,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter630 : struct.success)
+            for (FieldSchema _iter646 : struct.success)
             {
-              _iter630.write(oprot);
+              _iter646.write(oprot);
             }
           }
         }
@@ -35951,14 +36440,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            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)
+            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)
             {
-              _elem632 = new FieldSchema();
-              _elem632.read(iprot);
-              struct.success.add(_elem632);
+              _elem648 = new FieldSchema();
+              _elem648.read(iprot);
+              struct.success.add(_elem648);
             }
           }
           struct.setSuccessIsSet(true);
@@ -37003,14 +37492,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  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)
                   {
-                    _elem635 = new FieldSchema();
-                    _elem635.read(iprot);
-                    struct.success.add(_elem635);
+                    _elem651 = new FieldSchema();
+                    _elem651.read(iprot);
+                    struct.success.add(_elem651);
                   }
                   iprot.readListEnd();
                 }
@@ -37063,9 +37552,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 _iter637 : struct.success)
+            for (FieldSchema _iter653 : struct.success)
             {
-              _iter637.write(oprot);
+              _iter653.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -37120,9 +37609,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter638 : struct.success)
+            for (FieldSchema _iter654 : struct.success)
             {
-              _iter638.write(oprot);
+              _iter654.write(oprot);
             }
           }
         }
@@ -37143,14 +37632,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            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)
+            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)
             {
-              _elem640 = new FieldSchema();
-              _elem640.read(iprot);
-              struct.success.add(_elem640);
+              _elem656 = new FieldSchema();
+              _elem656.read(iprot);
+              struct.success.add(_elem656);
             }
           }
           struct.setSuccessIsSet(true);
@@ -38304,14 +38793,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list658.size);
+                  FieldSchema _elem659;
+                  for (int _i660 = 0; _i660 < _list658.size; ++_i660)
                   {
-                    _elem643 = new FieldSchema();
-                    _elem643.read(iprot);
-                    struct.success.add(_elem643);
+                    _elem659 = new FieldSchema();
+                    _elem659.read(iprot);
+                    struct.success.add(_elem659);
                   }
                   iprot.readListEnd();
                 }
@@ -38364,9 +38853,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 _iter645 : struct.success)
+            for (FieldSchema _iter661 : struct.success)
             {
-              _iter645.write(oprot);
+              _iter661.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -38421,9 +38910,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter646 : struct.success)
+            for (FieldSchema _iter662 : struct.success)
             {
-              _iter646.write(oprot);
+              _iter662.write(oprot);
             }
           }
         }
@@ -38444,14 +38933,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list663.size);
+            FieldSchema _elem664;
+            for (int _i665 = 0; _i665 < _list663.size; ++_i665)
             {
-              _elem648 = new FieldSchema();
-              _elem648.read(iprot);
-              struct.success.add(_elem648);
+              _elem664 = new FieldSchema();
+              _elem664.read(iprot);
+              struct.success.add(_elem664);
             }
           }
           struct.setSuccessIsSet(true);
@@ -40667,28 +41156,28 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class drop_table_args implements org.apache.thrift.TBase<drop_table_args, drop_table_args._Fields>, java.io.Serializable, Cloneable, Comparable<drop_table_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drop_table_args");
+  public static class create_table_with_constraints_args implements org.apache.thrift.TBase<create_table_with_constraints_args, create_table_with_constraints_args._Fields>, java.io.Serializable, Cloneable, Comparable<create_table_with_constraints_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("create_table_with_constraints_args");
 
-    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);
-    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)2);
-    private static final org.apache.thrift.protocol.TField DELETE_DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("deleteData", org.apache.thrift.protocol.TType.BOOL, (short)3);
+    private static final org.apache.thrift.protocol.TField TBL_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField PRIMARY_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("primaryKeys", org.apache.thrift.protocol.TType.LIST, (short)2);
+    private static final org.apache.thrift.protocol.TField FOREIGN_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("foreignKeys", 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 drop_table_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new drop_table_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new create_table_with_constraints_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new create_table_with_constraints_argsTupleSchemeFactory());
     }
 
-    private String dbname; // required
-    private String name; // required
-    private boolean deleteData; // required
+    private Table tbl; // required
+    private List<SQLPrimaryKey> primaryKeys; // required
+    private List<SQLForeignKey> foreignKeys; // 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 {
-      DBNAME((short)1, "dbname"),
-      NAME((short)2, "name"),
-      DELETE_DATA((short)3, "deleteData");
+      TBL((short)1, "tbl"),
+      PRIMARY_KEYS((short)2, "primaryKeys"),
+      FOREIGN_KEYS((short)3, "foreignKeys");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -40703,12 +41192,12 @@ public class ThriftHiveMetastore {
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // DBNAME
-            return DBNAME;
-          case 2: // NAME
-            return NAME;
-          case 3: // DELETE_DATA
-            return DELETE_DATA;
+          case 1: // TBL
+            return TBL;
+          case 2: // PRIMARY_KEYS
+            return PRIMARY_KEYS;
+          case 3: // FOREIGN_KEYS
+            return FOREIGN_KEYS;
           default:
             return null;
         }
@@ -40749,153 +41238,191 @@ public class ThriftHiveMetastore {
     }
 
     // isset id assignments
-    private static final int __DELETEDATA_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.DBNAME, new org.apache.thrift.meta_data.FieldMetaData("dbname", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.DELETE_DATA, new org.apache.thrift.meta_data.FieldMetaData("deleteData", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      tmpMap.put(_Fields.TBL, new org.apache.thrift.meta_data.FieldMetaData("tbl", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Table.class)));
+      tmpMap.put(_Fields.PRIMARY_KEYS, new org.apache.thrift.meta_data.FieldMetaData("primaryKeys", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          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, SQLPrimaryKey.class))));
+      tmpMap.put(_Fields.FOREIGN_KEYS, new org.apache.thrift.meta_data.FieldMetaData("foreignKeys", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          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, SQLForeignKey.class))));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(drop_table_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(create_table_with_constraints_args.class, metaDataMap);
     }
 
-    public drop_table_args() {
+    public create_table_with_constraints_args() {
     }
 
-    public drop_table_args(
-      String dbname,
-      String name,
-      boolean deleteData)
+    public create_table_with_constraints_args(
+      Table tbl,
+      List<SQLPrimaryKey> primaryKeys,
+      List<SQLForeignKey> foreignKeys)
     {
       this();
-      this.dbname = dbname;
-      this.name = name;
-      this.deleteData = deleteData;
-      setDeleteDataIsSet(true);
+      this.tbl = tbl;
+      this.primaryKeys = primaryKeys;
+      this.foreignKeys = foreignKeys;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public drop_table_args(drop_table_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      if (other.isSetDbname()) {
-        this.dbname = other.dbname;
+    public create_table_with_constraints_args(create_table_with_constraints_args other) {
+      if (other.isSetTbl()) {
+        this.tbl = new Table(other.tbl);
       }
-      if (other.isSetName()) {
-        this.name = other.name;
+      if (other.isSetPrimaryKeys()) {
+        List<SQLPrimaryKey> __this__primaryKeys = new ArrayList<SQLPrimaryKey>(other.primaryKeys.size());
+        for (SQLPrimaryKey other_element : other.primaryKeys) {
+          __this__primaryKeys.add(new SQLPrimaryKey(other_element));
+        }
+        this.primaryKeys = __this__primaryKeys;
+      }
+      if (other.isSetForeignKeys()) {
+        List<SQLForeignKey> __this__foreignKeys = new ArrayList<SQLForeignKey>(other.foreignKeys.size());
+        for (SQLForeignKey other_element : other.foreignKeys) {
+          __this__foreignKeys.add(new SQLForeignKey(other_element));
+        }
+        this.foreignKeys = __this__foreignKeys;
       }
-      this.deleteData = other.deleteData;
     }
 
-    public drop_table_args deepCopy() {
-      return new drop_table_args(this);
+    public create_table_with_constraints_args deepCopy() {
+      return new create_table_with_constraints_args(this);
     }
 
     @Override
     public void clear() {
-      this.dbname = null;
-      this.name = null;
-      setDeleteDataIsSet(false);
-      this.deleteData = false;
+      this.tbl = null;
+      this.primaryKeys = null;
+      this.foreignKeys = null;
     }
 
-    public String getDbname() {
-      return this.dbname;
+    public Table getTbl() {
+      return this.tbl;
     }
 
-    public void setDbname(String dbname) {
-      this.dbname = dbname;
+    public void setTbl(Table tbl) {
+      this.tbl = tbl;
     }
 
-    public void unsetDbname() {
-      this.dbname = null;
+    public void unsetTbl() {
+      this.tbl = null;
     }
 
-    /** Returns true if field dbname is set (has been assigned a value) and false otherwise */
-    public boolean isSetDbname() {
-      return this.dbname != null;
+    /** Returns true if field tbl is set (has been assigned a value) and false otherwise */
+    public boolean isSetTbl() {
+      return this.tbl != null;
     }
 
-    public void setDbnameIsSet(boolean value) {
+    public void setTblIsSet(boolean value) {
       if (!value) {
-        this.dbname = null;
+        this.tbl = null;
       }
     }
 
-    public String getName() {
-      return this.name;
+    public int getPrimaryKeysSize() {
+      return (this.primaryKeys == null) ? 0 : this.primaryKeys.size();
     }
 
-    public void setName(String name) {
-      this.name = name;
+    public java.util.Iterator<SQLPrimaryKey> getPrimaryKeysIterator() {
+      return (this.primaryKeys == null) ? null : this.primaryKeys.iterator();
     }
 
-    public void unsetName() {
-      this.name = null;
+    public void addToPrimaryKeys(SQLPrimaryKey elem) {
+      if (this.primaryKeys == null) {
+        this.primaryKeys = new ArrayList<SQLPrimaryKey>();
+      }
+      this.primaryKeys.add(elem);
     }
 
-    /** Returns true if field name is set (has been assigned a value) and false otherwise */
-    public boolean isSetName() {
-      return this.name != null;
+    public List<SQLPrimaryKey> getPrimaryKeys() {
+      return this.primaryKeys;
     }
 
-    public void setNameIsSet(boolean value) {
+    public void setPrimaryKeys(List<SQLPrimaryKey> primaryKeys) {
+      this.primaryKeys = primaryKeys;
+    }
+
+    public void unsetPrimaryKeys() {
+      this.primaryKeys = null;
+    }
+
+    /** Returns true if field primaryKeys is set (has been assigned a value) and false otherwise */
+    public boolean isSetPrimaryKeys() {
+      return this.primaryKeys != null;
+    }
+
+    public void setPrimaryKeysIsSet(boolean value) {
       if (!value) {
-        this.name = null;
+        this.primaryKeys = null;
       }
     }
 
-    public boolean isDeleteData() {
-      return this.deleteData;
+    public int getForeignKeysSize() {
+      return (this.foreignKeys == null) ? 0 : this.foreignKeys.size();
     }
 
-    public void setDeleteData(boolean deleteData) {
-      this.deleteData = deleteData;
-      setDeleteDataIsSet(true);
+    public java.util.Iterator<SQLForeignKey> getForeignKeysIterator() {
+      return (this.foreignKeys == null) ? null : this.foreignKeys.iterator();
     }
 
-    public void unsetDeleteData() {
-      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DELETEDATA_ISSET_ID);
+    public void addToForeignKeys(SQLForeignKey elem) {
+      if (this.foreignKeys == null) {
+        this.foreignKeys = new ArrayList<SQLForeignKey>();
+      }
+      this.foreignKeys.add(elem);
     }
 
-    /** Returns true if field deleteData is set (has been assigned a value) and false otherwise */
-    public boolean isSetDeleteData() {
-      return EncodingUtils.testBit(__isset_bitfield, __DELETEDATA_ISSET_ID);
+    public List<SQLForeignKey> getForeignKeys() {
+      return this.foreignKeys;
     }
 
-    public void setDeleteDataIsSet(boolean value) {
-      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DELETEDATA_ISSET_ID, value);
+    public void setForeignKeys(List<SQLForeignKey> foreignKeys) {
+      this.foreignKeys = foreignKeys;
+    }
+
+    public void unsetForeignKeys() {
+      this.foreignKeys = null;
+    }
+
+    /** Returns true if field foreignKeys is set (has been assigned a value) and false otherwise */
+    public boolean isSetForeignKeys() {
+      return this.foreignKeys != null;
+    }
+
+    public void setForeignKeysIsSet(boolean value) {
+      if (!value) {
+        this.foreignKeys = null;
+      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case DBNAME:
+      case TBL:
         if (value == null) {
-          unsetDbname();
+          unsetTbl();
         } else {
-          setDbname((String)value);
+          setTbl((Table)value);
         }
         break;
 
-      case NAME:
+      case PRIMARY_KEYS:
         if (value == null) {
-          unsetName();
+          unsetPrimaryKeys();
         } else {
-          setName((String)value);
+          setPrimaryKeys((List<SQLPrimaryKey>)value);
         }
         break;
 
-      case DELETE_DATA:
+      case FOREIGN_KEYS:
         if (value == null) {
-          unsetDeleteData();
+          unsetForeignKeys();
         } else {
-          setDeleteData((Boolean)value);
+          setForeignKeys((List<SQLForeignKey>)value);
         }
         break;
 
@@ -40904,14 +41431,14 @@ public class ThriftHiveMetastore {
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case DBNAME:
-        return getDbname();
+      case TBL:
+        return getTbl();
 
-      case NAME:
-        return getName();
+      case PRIMARY_KEYS:
+        return getPrimaryKeys();
 
-      case DELETE_DATA:
-        return isDeleteData();
+      case FOREIGN_KEYS:
+        return getForeignKeys();
 
       }
       throw new IllegalStateException();
@@ -40924,12 +41451,12 @@ public class ThriftHiveMetastore {
       }
 
       switch (field) {
-      case DBNAME:
-        return isSetDbname();
-      case NAME:
-        return isSetName();
-      case DELETE_DATA:
-        return isSetDeleteData();
+      case TBL:
+        return isSetTbl();
+      case PRIMARY_KEYS:
+        return isSetPrimaryKeys();
+      case FOREIGN_KEYS:
+        return isSetForeignKeys();
       }
       throw new IllegalStateException();
     }
@@ -40938,39 +41465,39 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof drop_table_args)
-        return this.equals((drop_table_args)that);
+      if (that instanceof create_table_with_constraints_args)
+        return this.equals((create_table_with_constraints_args)that);
       return false;
     }
 
-    public boolean equals(drop_table_args that) {
+    public boolean equals(create_table_with_constraints_args that) {
       if (that == null)
         return false;
 
-      boolean this_present_dbname = true && this.isSetDbname();
-      boolean that_present_dbname = true && that.isSetDbname();
-      if (this_present_dbname || that_present_dbname) {
-        if (!(this_present_dbname && that_present_dbname))
+      boolean this_present_tbl = true && this.isSetTbl();
+      boolean that_present_tbl = true && that.isSetTbl();
+      if (this_present_tbl || that_present_tbl) {
+        if (!(this_present_tbl && that_present_tbl))
           return false;
-        if (!this.dbname.equals(that.dbname))
+        if (!this.tbl.equals(that.tbl))
           return false;
       }
 
-      boolean this_present_name = true && this.isSetName();
-      boolean that_present_name = true && that.isSetName();
-      if (this_present_name || that_present_name) {
-        if (!(this_present_name && that_present_name))
+      boolean this_present_primaryKeys = true && this.isSetPrimaryKeys();
+      boolean that_present_primaryKeys = true && that.isSetPrimaryKeys();
+      if (this_present_primaryKeys || that_present_primaryKeys) {
+        if (!(this_present_primaryKeys && that_present_primaryKeys))
           return false;
-        if (!this.name.equals(that.name))
+        if (!this.primaryKeys.equals(that.primaryKeys))
           return false;
       }
 
-      boolean this_present_deleteData = true;
-      boolean that_present_deleteData = true;
-      if (this_present_deleteData || that_present_deleteData) {
-        if (!(this_present_deleteData && that_present_deleteData))
+      boolean this_present_foreignKeys = true && this.isSetForeignKeys();
+      boolean that_present_foreignKeys = true && that.isSetForeignKeys();
+      if (this_present_foreignKeys || that_present_foreignKeys) {
+        if (!(this_present_foreignKeys && that_present_foreignKeys))
           return false;
-        if (this.deleteData != that.deleteData)
+        if (!this.foreignKeys.equals(that.foreignKeys))
           return false;
       }
 
@@ -40981,58 +41508,58 @@ public class ThriftHiveMetastore {
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_dbname = true && (isSetDbname());
-      list.add(present_dbname);
-      if (present_dbname)
-        list.add(dbname);
+      boolean present_tbl = true && (isSetTbl());
+      list.add(present_tbl);
+      if (present_tbl)
+        list.add(tbl);
 
-      boolean present_name = true && (isSetName());
-      list.add(present_name);
-      if (present_name)
-        list.add(name);
+      boolean present_primaryKeys = true && (isSetPrimaryKeys());
+      list.add(present_primaryKeys);
+      if (present_primaryKeys)
+        list.add(primaryKeys);
 
-      boolean present_deleteData = true;
-      list.add(present_deleteData);
-      if (present_deleteData)
-        list.add(deleteData);
+      boolean present_foreignKeys = true && (isSetForeignKeys());
+      list.add(present_foreignKeys);
+      if (present_foreignKeys)
+        list.add(foreignKeys);
 
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(drop_table_args other) {
+    public int compareTo(create_table_with_constraints_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetDbname()).compareTo(other.isSetDbname());
+      lastComparison = Boolean.valueOf(isSetTbl()).compareTo(other.isSetTbl());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetDbname()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbname, other.dbname);
+      if (isSetTbl()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl, other.tbl);
         if (lastComparison != 0) {
           return lastComparison;
         }
       }
-      lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+      lastComparison = Boolean.valueOf(isSetPrimaryKeys()).compareTo(other.isSetPrimaryKeys());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetName()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (isSetPrimaryKeys()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.primaryKeys, other.primaryKeys);
         if (lastComparison != 0) {
           return lastComparison;
         }
       }
-      lastComparison = Boolean.valueOf(isSetDeleteData()).compareTo(other.isSetDeleteData());
+      lastComparison = Boolean.valueOf(isSetForeignKeys()).compareTo(other.isSetForeignKeys());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetDeleteData()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deleteData, other.deleteData);
+      if (isSetForeignKeys()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.foreignKeys, other.foreignKeys);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -41054,27 +41581,31 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("drop_table_args(");
+      StringBuilder sb = new StringBuilder("create_table_with_constraints_args(");
       boolean first = true;
 
-      sb.append("dbname:");
-      if (this.dbname == null) {
+      sb.append("tbl:");
+      if (this.tbl == null) {
         sb.append("null");
       } else {
-        sb.append(this.dbname);
+        sb.append(this.tbl);
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("name:");
-      if (this.name == null) {
+      sb.append("primaryKeys:");
+      if (this.primaryKeys == null) {
         sb.append("null");
       } else {
-        sb.append(this.name);
+        sb.append(this.primaryKeys);
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("deleteData:");
-      sb.append(this.deleteData);
+      sb.append("foreignKeys:");
+      if (this.foreignKeys == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.foreignKeys);
+      }
       first = false;
       sb.append(")");
       return sb.toString();
@@ -41083,6 +41614,9 @@ public class ThriftHiveMetastore {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (tbl != null) {
+        tbl.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -41095,23 +41629,21 @@ public class ThriftHiveMetastore {
 
     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 drop_table_argsStandardSchemeFactory implements SchemeFactory {
-      public drop_table_argsStandardScheme getScheme() {
-        return new drop_table_argsStandardScheme();
+    private static class create_table_with_constraints_argsStandardSchemeFactory implements SchemeFactory {
+      public create_table_with_constraints_argsStandardScheme getScheme() {
+        return new create_table_with_constraints_argsStandardScheme();
       }
     }
 
-    private static class drop_table_argsStandardScheme extends StandardScheme<drop_table_args> {
+    private static class create_table_with_constraints_argsStandardScheme extends StandardScheme<create_table_with_constraints_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, drop_table_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_constraints_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -41121,26 +41653,49 @@ public class ThriftHiveMetastore {
             break;
           }
           switch (schemeField.id) {
-            case 1: // DBNAME
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.dbname = iprot.readString();
-                struct.setDbnameIsSet(true);
+            case 1: // TBL
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tbl = new Table();
+                struct.tbl.read(iprot);
+                struct.setTblIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // NAME
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.name = iprot.readString();
-                struct.setNameIsSet(true);
+            case 2: // PRIMARY_KEYS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list666.size);
+                  SQLPrimaryKey _elem667;
+                  for (int _i668 = 0; _i668 < _list666.size; ++_i668)
+                  {
+                    _elem667 = new SQLPrimaryKey();
+                    _elem667.read(iprot);
+                    struct.primaryKeys.add(_elem667);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setPrimaryKeysIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 3: // DELETE_DATA
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.deleteData = iprot.readBool();
-                struct.setDeleteDataIsSet(true);
+            case 3: // FOREIGN_KEYS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list669 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list669.size);
+                  SQLForeignKey _elem670;
+                  for (int _i671 = 0; _i671 < _list669.size; ++_i671)
+                  {
+                    _elem670 = new SQLForeignKey();
+                    _elem670.read(iprot);
+                    struct.foreignKeys.add(_elem670);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setForeignKeysIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -41154,102 +41709,157 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, drop_table_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_constraints_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.dbname != null) {
-          oprot.writeFieldBegin(DBNAME_FIELD_DESC);
-          oprot.writeString(struct.dbname);
+        if (struct.tbl != null) {
+          oprot.writeFieldBegin(TBL_FIELD_DESC);
+          struct.tbl.write(oprot);
           oprot.writeFieldEnd();
         }
-        if (struct.name != null) {
-          oprot.writeFieldBegin(NAME_FIELD_DESC);
-          oprot.writeString(struct.name);
+        if (struct.primaryKeys != null) {
+          oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
+            for (SQLPrimaryKey _iter672 : struct.primaryKeys)
+            {
+              _iter672.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.foreignKeys != null) {
+          oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
+            for (SQLForeignKey _iter673 : struct.foreignKeys)
+            {
+              _iter673.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(DELETE_DATA_FIELD_DESC);
-        oprot.writeBool(struct.deleteData);
-        oprot.writeFieldEnd();
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class drop_table_argsTupleSchemeFactory implements SchemeFactory {
-      public drop_table_argsTupleScheme getScheme() {
-        return new drop_table_argsTupleScheme();
+    private static class create_table_with_constraints_argsTupleSchemeFactory implements SchemeFactory {
+      public create_table_with_constraints_argsTupleScheme getScheme() {
+        return new create_table_with_constraints_argsTupleScheme();
       }
     }
 
-    private static class drop_table_argsTupleScheme extends TupleScheme<drop_table_args> {
+    private static class create_table_with_constraints_argsTupleScheme extends TupleScheme<create_table_with_constraints_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, drop_table_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, create_table_with_constraints_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetDbname()) {
+        if (struct.isSetTbl()) {
           optionals.set(0);
         }
-        if (struct.isSetName()) {
+        if (struct.isSetPrimaryKeys()) {
           optionals.set(1);
         }
-        if (struct.isSetDeleteData()) {
+        if (struct.isSetForeignKeys()) {
           optionals.set(2);
         }
         oprot.writeBitSet(optionals, 3);
-        if (struct.isSetDbname()) {
-          oprot.writeString(struct.dbname);
+        if (struct.isSetTbl()) {
+          struct.tbl.write(oprot);
         }
-        if (struct.isSetName()) {
-          oprot.writeString(struct.name);
+        if (struct.isSetPrimaryKeys()) {
+          {
+            oprot.writeI32(struct.primaryKeys.size());
+            for (SQLPrimaryKey _iter674 : struct.primaryKeys)
+            {
+              _iter674.write(oprot);
+            }
+          }
         }
-        if (struct.isSetDeleteData()) {
-          oprot.writeBool(struct.deleteData);
+        if (struct.isSetForeignKeys()) {
+          {
+            oprot.writeI32(struct.foreignKeys.size());
+            for (SQLForeignKey _iter675 : struct.foreignKeys)
+            {
+              _iter675.write(oprot);
+            }
+          }
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, drop_table_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, create_table_with_constraints_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
-          struct.dbname = iprot.readString();
-          struct.setDbnameIsSet(true);
+          struct.tbl = new Table();
+          struct.tbl.read(iprot);
+          struct.setTblIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.name = iprot.readString();
-          struct.setNameIsSet(true);
+          {
+            org.apache.thrift.protocol.TList _list676 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list676.size);
+            SQLPrimaryKey _elem677;
+            for (int _i678 = 0; _i678 < _list676.size; ++_i678)
+            {
+              _elem677 = new SQLPrimaryKey();
+              _elem677.read(iprot);
+              struct.primaryKeys.add(_elem677);
+            }
+          }
+          struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
-          struct.deleteData = iprot.readBool();
-          struct.setDeleteDataIsSet(true);
+          {
+            org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list679.size);
+            SQLForeignKey _elem680;
+            for (int _i681 = 0; _i681 < _list679.size; ++_i681)
+            {
+              _elem680 = new SQLForeignKey();
+              _elem680.read(iprot);
+              struct.foreignKeys.add(_elem680);
+            }
+          }
+          struct.setForeignKeysIsSet(true);
         }
       }
     }
 
   }
 
-  public static class drop_table_result implements org.apache.thrift.TBase<drop_table_result, drop_table_result._Fields>, java.io.Serializable, Cloneable, Comparable<drop_table_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drop_table_result");
+  public static class create_table_with_constraints_result implements org.apache.thrift.TBase<create_table_with_constraints_result, create_table_with_constraints_result._Fields>, java.io.Serializable, Cloneable, Comparable<create_table_with_constraints_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("create_table_with_constraints_result");
 
     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 O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField O4_FIELD_DESC = new org.apache.thrift.protocol.TField("o4", org.apache.thrift.protocol.TType.STRUCT, (short)4);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new drop_table_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new drop_table_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new

<TRUNCATED>

[13/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 e782bb5..99a764e 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -318,6 +318,24 @@ module ThriftHiveMetastore
       return
     end
 
+    def create_table_with_constraints(tbl, primaryKeys, foreignKeys)
+      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys)
+      recv_create_table_with_constraints()
+    end
+
+    def send_create_table_with_constraints(tbl, primaryKeys, foreignKeys)
+      send_message('create_table_with_constraints', Create_table_with_constraints_args, :tbl => tbl, :primaryKeys => primaryKeys, :foreignKeys => foreignKeys)
+    end
+
+    def recv_create_table_with_constraints()
+      result = receive_message(Create_table_with_constraints_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise result.o3 unless result.o3.nil?
+      raise result.o4 unless result.o4.nil?
+      return
+    end
+
     def drop_table(dbname, name, deleteData)
       send_drop_table(dbname, name, deleteData)
       recv_drop_table()
@@ -1324,6 +1342,40 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_index_names failed: unknown result')
     end
 
+    def get_primary_keys(request)
+      send_get_primary_keys(request)
+      return recv_get_primary_keys()
+    end
+
+    def send_get_primary_keys(request)
+      send_message('get_primary_keys', Get_primary_keys_args, :request => request)
+    end
+
+    def recv_get_primary_keys()
+      result = receive_message(Get_primary_keys_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_primary_keys failed: unknown result')
+    end
+
+    def get_foreign_keys(request)
+      send_get_foreign_keys(request)
+      return recv_get_foreign_keys()
+    end
+
+    def send_get_foreign_keys(request)
+      send_message('get_foreign_keys', Get_foreign_keys_args, :request => request)
+    end
+
+    def recv_get_foreign_keys()
+      result = receive_message(Get_foreign_keys_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_foreign_keys failed: unknown result')
+    end
+
     def update_table_column_statistics(stats_obj)
       send_update_table_column_statistics(stats_obj)
       return recv_update_table_column_statistics()
@@ -2635,6 +2687,23 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'create_table_with_environment_context', seqid)
     end
 
+    def process_create_table_with_constraints(seqid, iprot, oprot)
+      args = read_args(iprot, Create_table_with_constraints_args)
+      result = Create_table_with_constraints_result.new()
+      begin
+        @handler.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys)
+      rescue ::AlreadyExistsException => o1
+        result.o1 = o1
+      rescue ::InvalidObjectException => o2
+        result.o2 = o2
+      rescue ::MetaException => o3
+        result.o3 = o3
+      rescue ::NoSuchObjectException => o4
+        result.o4 = o4
+      end
+      write_result(result, oprot, 'create_table_with_constraints', seqid)
+    end
+
     def process_drop_table(seqid, iprot, oprot)
       args = read_args(iprot, Drop_table_args)
       result = Drop_table_result.new()
@@ -3432,6 +3501,32 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_index_names', seqid)
     end
 
+    def process_get_primary_keys(seqid, iprot, oprot)
+      args = read_args(iprot, Get_primary_keys_args)
+      result = Get_primary_keys_result.new()
+      begin
+        result.success = @handler.get_primary_keys(args.request)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      rescue ::NoSuchObjectException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_primary_keys', seqid)
+    end
+
+    def process_get_foreign_keys(seqid, iprot, oprot)
+      args = read_args(iprot, Get_foreign_keys_args)
+      result = Get_foreign_keys_result.new()
+      begin
+        result.success = @handler.get_foreign_keys(args.request)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      rescue ::NoSuchObjectException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_foreign_keys', seqid)
+    end
+
     def process_update_table_column_statistics(seqid, iprot, oprot)
       args = read_args(iprot, Update_table_column_statistics_args)
       result = Update_table_column_statistics_result.new()
@@ -4817,6 +4912,48 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Create_table_with_constraints_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    TBL = 1
+    PRIMARYKEYS = 2
+    FOREIGNKEYS = 3
+
+    FIELDS = {
+      TBL => {:type => ::Thrift::Types::STRUCT, :name => 'tbl', :class => ::Table},
+      PRIMARYKEYS => {:type => ::Thrift::Types::LIST, :name => 'primaryKeys', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLPrimaryKey}},
+      FOREIGNKEYS => {:type => ::Thrift::Types::LIST, :name => 'foreignKeys', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLForeignKey}}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Create_table_with_constraints_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+    O3 = 3
+    O4 = 4
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::AlreadyExistsException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException},
+      O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException},
+      O4 => {:type => ::Thrift::Types::STRUCT, :name => 'o4', :class => ::NoSuchObjectException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Drop_table_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DBNAME = 1
@@ -7205,6 +7342,78 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_primary_keys_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQUEST = 1
+
+    FIELDS = {
+      REQUEST => {:type => ::Thrift::Types::STRUCT, :name => 'request', :class => ::PrimaryKeysRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_primary_keys_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::PrimaryKeysResponse},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::NoSuchObjectException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_foreign_keys_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQUEST = 1
+
+    FIELDS = {
+      REQUEST => {:type => ::Thrift::Types::STRUCT, :name => 'request', :class => ::ForeignKeysRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_foreign_keys_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::ForeignKeysResponse},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::NoSuchObjectException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Update_table_column_statistics_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     STATS_OBJ = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 c9fadad..ed2057a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Multimaps;
+
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -112,6 +113,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.jdo.JDOException;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.text.DateFormat;
@@ -1310,7 +1312,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     private void create_table_core(final RawStore ms, final Table tbl,
-        final EnvironmentContext envContext)
+        final EnvironmentContext envContext, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
         throws AlreadyExistsException, MetaException,
         InvalidObjectException, NoSuchObjectException {
 
@@ -1395,7 +1397,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
             tbl.getParameters().get(hive_metastoreConstants.DDL_TIME) == null) {
           tbl.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
         }
-        ms.createTable(tbl);
+        if (primaryKeys == null && foreignKeys == null) {
+          ms.createTable(tbl);
+        } else {
+          ms.createTableWithConstraints(tbl, primaryKeys, foreignKeys);
+        }
         success = ms.commitTransaction();
 
       } finally {
@@ -1428,7 +1434,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       boolean success = false;
       Exception ex = null;
       try {
-        create_table_core(getMS(), tbl, envContext);
+        create_table_core(getMS(), tbl, envContext, null, null);
         success = true;
       } catch (NoSuchObjectException e) {
         ex = e;
@@ -1449,6 +1455,34 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
+    public void create_table_with_constraints(final Table tbl,
+        final List<SQLPrimaryKey> primaryKeys, final List<SQLForeignKey> foreignKeys)
+        throws AlreadyExistsException, MetaException, InvalidObjectException {
+      startFunction("create_table", ": " + tbl.toString());
+      boolean success = false;
+      Exception ex = null;
+      try {
+        create_table_core(getMS(), tbl, null, primaryKeys, foreignKeys);
+        success = true;
+      } catch (NoSuchObjectException e) {
+        ex = e;
+        throw new InvalidObjectException(e.getMessage());
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else if (e instanceof InvalidObjectException) {
+          throw (InvalidObjectException) e;
+        } else if (e instanceof AlreadyExistsException) {
+          throw (AlreadyExistsException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        endFunction("create_table", success, ex, tbl.getTableName());
+      }
+    }
     private boolean is_table_exists(RawStore ms, String dbname, String name)
         throws MetaException {
       return (ms.getTable(dbname, name) != null);
@@ -6131,6 +6165,63 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         throws TException {
       return new GetChangeVersionResult(getMS().getChangeVersion(req.getTopic()));
     }
+
+
+    @Override
+    public PrimaryKeysResponse get_primary_keys(PrimaryKeysRequest request)
+      throws MetaException, NoSuchObjectException, TException {
+      String db_name = request.getDb_name();
+      String tbl_name = request.getTbl_name();
+      startTableFunction("get_primary_keys", db_name, tbl_name);
+      List<SQLPrimaryKey> ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getPrimaryKeys(db_name, tbl_name);
+      } catch (Exception e) {
+       ex = e;
+       if (e instanceof MetaException) {
+         throw (MetaException) e;
+       } else if (e instanceof NoSuchObjectException) {
+         throw (NoSuchObjectException) e;
+       } else {
+         throw newMetaException(e);
+       }
+     } finally {
+       endFunction("get_primary_keys", ret != null, ex, tbl_name);
+     }
+     return new PrimaryKeysResponse(ret);
+    }
+
+    @Override
+    public ForeignKeysResponse get_foreign_keys(ForeignKeysRequest request) throws MetaException,
+      NoSuchObjectException, TException {
+      String parent_db_name = request.getParent_db_name();
+      String parent_tbl_name = request.getParent_tbl_name();
+      String foreign_db_name = request.getForeign_db_name();
+      String foreign_tbl_name = request.getForeign_tbl_name();
+      startFunction("get_foreign_keys", " : parentdb=" + parent_db_name +
+        " parenttbl=" + parent_tbl_name + " foreigndb=" + foreign_db_name +
+        " foreigntbl=" + foreign_tbl_name);
+      List<SQLForeignKey> ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getForeignKeys(parent_db_name, parent_tbl_name,
+              foreign_db_name, foreign_tbl_name);
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else if (e instanceof NoSuchObjectException) {
+          throw (NoSuchObjectException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        endFunction("get_foreign_keys", ret != null, ex, foreign_tbl_name);
+      }
+      return new ForeignKeysResponse(ret);
+    }
+
   }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index cdd12ab..7d37d07 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.FireEventRequest;
 import org.apache.hadoop.hive.metastore.api.FireEventResponse;
+import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
 import org.apache.hadoop.hive.metastore.api.GetChangeVersionRequest;
@@ -94,12 +95,15 @@ import org.apache.hadoop.hive.metastore.api.PartitionEventType;
 import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
 import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
 import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
 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.PutFileMetadataRequest;
 import org.apache.hadoop.hive.metastore.api.RequestPartsSpec;
 import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
@@ -136,6 +140,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.security.auth.login.LoginException;
+
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationHandler;
@@ -736,7 +741,32 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     }
   }
 
-  /**
+  @Override
+  public void createTableWithConstraints(Table tbl,
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    throws AlreadyExistsException, InvalidObjectException,
+    MetaException, NoSuchObjectException, TException {
+    HiveMetaHook hook = getHook(tbl);
+    if (hook != null) {
+      hook.preCreateTable(tbl);
+    }
+    boolean success = false;
+    try {
+      // Subclasses can override this step (for example, for temporary tables)
+      client.create_table_with_constraints(tbl, primaryKeys, foreignKeys);
+      if (hook != null) {
+        hook.commitCreateTable(tbl);
+      }
+      success = true;
+    } finally {
+      if (!success && (hook != null)) {
+        hook.rollbackCreateTable(tbl);
+      }
+    }
+  }
+
+
+/**
    * @param type
    * @return true or false
    * @throws AlreadyExistsException
@@ -1528,6 +1558,19 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return filterHook.filterIndexes(client.get_indexes(dbName, tblName, max));
   }
 
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys(PrimaryKeysRequest req)
+    throws MetaException, NoSuchObjectException, TException {
+    return client.get_primary_keys(req).getPrimaryKeys();
+  }
+
+  @Override
+  public List<SQLForeignKey> getForeignKeys(ForeignKeysRequest req) throws MetaException,
+    NoSuchObjectException, TException {
+    return client.get_foreign_keys(req).getForeignKeys();
+  }
+
+
   /** {@inheritDoc} */
   @Override
   public boolean updateTableColumnStatistics(ColumnStatistics statsObj)
@@ -2375,4 +2418,5 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
   public long getChangeVersion(String topic) throws TException {
     return client.get_change_version(new GetChangeVersionRequest(topic)).getVersion();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 39cf927..c900a2d 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.FireEventRequest;
 import org.apache.hadoop.hive.metastore.api.FireEventResponse;
+import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
@@ -64,10 +65,13 @@ import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
 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.Role;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
@@ -1554,4 +1558,16 @@ public interface IMetaStoreClient {
       boolean allParts) throws TException;
 
   long getChangeVersion(String topic) throws TException;
+
+  List<SQLPrimaryKey> getPrimaryKeys(PrimaryKeysRequest request)
+    throws MetaException, NoSuchObjectException, TException;
+
+  List<SQLForeignKey> getForeignKeys(ForeignKeysRequest request) throws MetaException,
+    NoSuchObjectException, TException;
+
+  void createTableWithConstraints(
+    org.apache.hadoop.hive.metastore.api.Table tTbl,
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, TException;
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 06e9f78..744512f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -53,10 +53,13 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.model.MConstraint;
 import org.apache.hadoop.hive.metastore.model.MDatabase;
 import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics;
 import org.apache.hadoop.hive.metastore.model.MTableColumnStatistics;
@@ -1809,4 +1812,135 @@ class MetaStoreDirectSql {
     }
     return result;
   }
+
+  public List<SQLForeignKey> getForeignKeys(String parent_db_name, String parent_tbl_name, String foreign_db_name, String foreign_tbl_name) throws MetaException {
+    List<SQLForeignKey> ret = new ArrayList<SQLForeignKey>();
+    String queryText =
+      "SELECT  \"D2\".\"NAME\", \"T2\".\"TBL_NAME\", \"C2\".\"COLUMN_NAME\","
+      + "\"DBS\".\"NAME\", \"TBLS\".\"TBL_NAME\", \"COLUMNS_V2\".\"COLUMN_NAME\", "
+      + "\"KEY_CONSTRAINTS\".\"POSITION\", \"KEY_CONSTRAINTS\".\"UPDATE_RULE\", \"KEY_CONSTRAINTS\".\"DELETE_RULE\", "
+      + "\"KEY_CONSTRAINTS\".\"CONSTRAINT_NAME\" , \"KEY_CONSTRAINTS2\".\"CONSTRAINT_NAME\", \"KEY_CONSTRAINTS\".\"ENABLE_VALIDATE_RELY\""
+      + " FROM \"TBLS\" "
+      + " INNER JOIN \"KEY_CONSTRAINTS\" ON \"TBLS\".\"TBL_ID\" = \"KEY_CONSTRAINTS\".\"CHILD_TBL_ID\" "
+      + " INNER JOIN \"KEY_CONSTRAINTS\" \"KEY_CONSTRAINTS2\" ON \"KEY_CONSTRAINTS2\".\"PARENT_TBL_ID\"  = \"KEY_CONSTRAINTS\".\"PARENT_TBL_ID\" "
+      + " INNER JOIN \"DBS\" ON \"TBLS\".\"DB_ID\" = \"DBS\".\"DB_ID\" "
+      + " INNER JOIN \"TBLS\" \"T2\" ON  \"KEY_CONSTRAINTS\".\"PARENT_TBL_ID\" = \"T2\".\"TBL_ID\" "
+      + " INNER JOIN \"DBS\" \"D2\" ON \"T2\".\"DB_ID\" = \"D2\".\"DB_ID\" "
+      + " INNER JOIN \"COLUMNS_V2\"  ON \"COLUMNS_V2\".\"CD_ID\" = \"KEY_CONSTRAINTS\".\"CHILD_CD_ID\" "
+      + " INNER JOIN \"COLUMNS_V2\" \"C2\" ON \"C2\".\"CD_ID\" = \"KEY_CONSTRAINTS\".\"PARENT_CD_ID\" "
+      + " WHERE \"KEY_CONSTRAINTS\".\"CONSTRAINT_TYPE\" = "
+      + MConstraint.FOREIGN_KEY_CONSTRAINT
+      + " AND \"KEY_CONSTRAINTS2\".\"CONSTRAINT_TYPE\" = "
+      + MConstraint.PRIMARY_KEY_CONSTRAINT
+      + (foreign_db_name == null ? "" : "\"DBS\".\"NAME\" = ? AND")
+      + (foreign_tbl_name == null ? "" : " \"TBLS\".\"TBL_NAME\" = ? AND ")
+      + (parent_tbl_name == null ? "" : " \"T2\".\"TBL_NAME\" = ? AND ")
+      + (parent_db_name == null ? "" : "\"D2\".\"NAME\" = ?") ;
+
+    queryText = queryText.trim();
+    if (queryText.endsWith("WHERE")) {
+      queryText = queryText.substring(0, queryText.length()-5);
+    }
+    if (queryText.endsWith("AND")) {
+      queryText = queryText.substring(0, queryText.length()-3);
+    }
+    List<String> pms = new ArrayList<String>();
+    if (foreign_db_name != null) {
+      pms.add(foreign_db_name);
+    }
+    if (foreign_tbl_name != null) {
+      pms.add(foreign_tbl_name);
+    }
+    if (parent_tbl_name != null) {
+      pms.add(parent_tbl_name);
+    }
+    if (parent_db_name != null) {
+      pms.add(parent_db_name);
+    }
+
+    Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
+      List<Object[]> sqlResult = ensureList(executeWithArray(
+        queryParams, pms.toArray(), queryText));
+
+    if (!sqlResult.isEmpty()) {
+      for (Object[] line : sqlResult) {
+        int enableValidateRely = extractSqlInt(line[11]);
+        boolean enable = (enableValidateRely & 4) != 0;
+        boolean validate = (enableValidateRely & 2) != 0;
+        boolean rely = (enableValidateRely & 1) != 0;
+        SQLForeignKey currKey = new SQLForeignKey(
+          extractSqlString(line[0]),
+          extractSqlString(line[1]),
+          extractSqlString(line[2]),
+          extractSqlString(line[3]),
+          extractSqlString(line[4]),
+          extractSqlString(line[5]),
+          extractSqlInt(line[6]),
+          extractSqlInt(line[7]),
+          extractSqlInt(line[8]),
+          extractSqlString(line[9]),
+          extractSqlString(line[10]),
+          enable,
+          validate,
+          rely
+          );
+          ret.add(currKey);
+      }
+    }
+    return ret;
+  }
+
+  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name) throws MetaException {
+    List<SQLPrimaryKey> ret = new ArrayList<SQLPrimaryKey>();
+    String queryText =
+      "SELECT \"DBS\".\"NAME\", \"TBLS\".\"TBL_NAME\", \"COLUMNS_V2\".\"COLUMN_NAME\","
+      + "\"KEY_CONSTRAINTS\".\"POSITION\", "
+      + "\"KEY_CONSTRAINTS\".\"CONSTRAINT_NAME\", \"KEY_CONSTRAINTS\".\"ENABLE_VALIDATE_RELY\" "
+      + " FROM  \"TBLS\" "
+      + " INNER  JOIN \"KEY_CONSTRAINTS\" ON \"TBLS\".\"TBL_ID\" = \"KEY_CONSTRAINTS\".\"PARENT_TBL_ID\" "
+      + " INNER JOIN \"DBS\" ON \"TBLS\".\"DB_ID\" = \"DBS\".\"DB_ID\" "
+      + " INNER JOIN \"TBLS\" ON \"KEY_CONSTRAINTS\".\"PARENT_TBL_ID\" = \"TBLS\".\"TBL_ID\" "
+      + " INNER JOIN \"COLUMNS_V2\" ON \"COLUMNS_V2\".\"CD_ID\" = \"KEY_CONSTRAINTS\".\"PARENT_CD_ID\" "
+      + " WHERE \"KEY_CONSTRAINTS\".\"CONSTRAINT_TYPE\" = "+ MConstraint.PRIMARY_KEY_CONSTRAINT + " AND "
+      + (db_name == null ? "" : "\"DBS\".\"NAME\" = ? AND")
+      + (tbl_name == null ? "" : " \"TBLS\".\"TBL_NAME\" = ? ") ;
+
+    queryText = queryText.trim();
+    if (queryText.endsWith("WHERE")) {
+      queryText = queryText.substring(0, queryText.length()-5);
+    }
+    if (queryText.endsWith("AND")) {
+      queryText = queryText.substring(0, queryText.length()-3);
+    }
+    List<String> pms = new ArrayList<String>();
+    if (db_name != null) {
+      pms.add(db_name);
+    }
+    if (tbl_name != null) {
+      pms.add(tbl_name);
+    }
+
+    Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
+      List<Object[]> sqlResult = ensureList(executeWithArray(
+        queryParams, pms.toArray(), queryText));
+
+    if (!sqlResult.isEmpty()) {
+      for (Object[] line : sqlResult) {
+          int enableValidateRely = extractSqlInt(line[5]);
+          boolean enable = (enableValidateRely & 4) != 0;
+          boolean validate = (enableValidateRely & 2) != 0;
+          boolean rely = (enableValidateRely & 1) != 0;
+        SQLPrimaryKey currKey = new SQLPrimaryKey(
+          extractSqlString(line[0]),
+          extractSqlString(line[1]),
+          extractSqlString(line[2]),
+          extractSqlInt(line[3]), extractSqlString(line[4]),
+          enable,
+          validate,
+          rely);
+          ret.add(currKey);
+      }
+    }
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 ac293b9..ae6f084 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -54,6 +54,7 @@ import javax.jdo.Transaction;
 import javax.jdo.datastore.DataStoreCache;
 import javax.jdo.identity.IntIdentity;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -97,6 +98,8 @@ import org.apache.hadoop.hive.metastore.api.ResourceType;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -108,6 +111,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.model.MChangeVersion;
 import org.apache.hadoop.hive.metastore.model.MColumnDescriptor;
+import org.apache.hadoop.hive.metastore.model.MConstraint;
 import org.apache.hadoop.hive.metastore.model.MDBPrivilege;
 import org.apache.hadoop.hive.metastore.model.MDatabase;
 import org.apache.hadoop.hive.metastore.model.MDelegationToken;
@@ -899,12 +903,31 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
+  public void createTableWithConstraints(Table tbl,
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    throws InvalidObjectException, MetaException {
+    boolean success = false;
+    try {
+      openTransaction();
+      createTable(tbl);
+      addPrimaryKeys(primaryKeys);
+      addForeignKeys(foreignKeys);
+	  success = commitTransaction();
+    } finally {
+      if (!success) {
+        rollbackTransaction();
+      }
+    }
+  }
+
+  @Override
   public void createTable(Table tbl) throws InvalidObjectException, MetaException {
     boolean commited = false;
     try {
       openTransaction();
       MTable mtbl = convertToMTable(tbl);
       pm.makePersistent(mtbl);
+
       PrincipalPrivilegeSet principalPrivs = tbl.getPrivileges();
       List<Object> toPersistPrivObjs = new ArrayList<Object>();
       if (principalPrivs != null) {
@@ -1328,7 +1351,7 @@ public class ObjectStore implements RawStore, Configurable {
     // A new table is always created with a new column descriptor
     return new MTable(HiveStringUtils.normalizeIdentifier(tbl.getTableName()), mdb,
         convertToMStorageDescriptor(tbl.getSd()), tbl.getOwner(), tbl
-            .getCreateTime(), tbl.getLastAccessTime(), tbl.getRetention(),
+        .getCreateTime(), tbl.getLastAccessTime(), tbl.getRetention(),
         convertToMFieldSchemas(tbl.getPartitionKeys()), tbl.getParameters(),
         tbl.getViewOriginalText(), tbl.getViewExpandedText(),
         tableType);
@@ -3200,6 +3223,165 @@ public class ObjectStore implements RawStore, Configurable {
     return sds;
   }
 
+  private MColumnDescriptor getColumnFromTable(MTable mtbl, String col) {
+   for (MFieldSchema mfs: mtbl.getSd().getCD().getCols()) {
+     if (mfs.getName().equals(col)) {
+       List<MFieldSchema> mfsl = new ArrayList<MFieldSchema>();
+       mfsl.add(mfs);
+       return new MColumnDescriptor(mfsl);
+     }
+   }
+   return null;
+  }
+
+  private  boolean constraintNameAlreadyExists(String name) {
+    boolean commited = false;
+    Query constraintExistsQuery = null;
+    String constraintNameIfExists = null;
+    try {
+      openTransaction();
+      name = HiveStringUtils.normalizeIdentifier(name);
+      constraintExistsQuery = pm.newQuery(MConstraint.class, "constraintName == name");
+      constraintExistsQuery.declareParameters("java.lang.String name");
+      constraintExistsQuery.setUnique(true);
+      constraintExistsQuery.setResult("name");
+      constraintNameIfExists = (String) constraintExistsQuery.execute(name);
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (constraintExistsQuery != null) {
+          constraintExistsQuery.closeAll();
+      }
+    }
+    return constraintNameIfExists != null && !constraintNameIfExists.isEmpty();
+  }
+
+  private String generateConstraintName(String... parameters) throws MetaException {
+    int hashcode = ArrayUtils.toString(parameters).hashCode();
+    int counter = 0;
+    final int MAX_RETRIES = 10;
+    while (counter < MAX_RETRIES) {
+      String currName = (parameters.length == 0 ? "constraint_" : parameters[parameters.length-1]) +
+        "_" + hashcode + "_" + System.currentTimeMillis() + "_" + (counter++);
+      if (!constraintNameAlreadyExists(currName)) {
+        return currName;
+      }
+    }
+    throw new MetaException("Error while trying to generate the constraint name for " + ArrayUtils.toString(parameters));
+  }
+
+  private void addForeignKeys(
+    List<SQLForeignKey> fks) throws InvalidObjectException,
+    MetaException {
+    List<MConstraint> mpkfks = new ArrayList<MConstraint>();
+    String currentConstraintName = null;
+
+    for (int i = 0; i < fks.size(); i++) {
+      MTable parentTable =
+        getMTable(fks.get(i).getPktable_db(), fks.get(i).getPktable_name());
+      MTable childTable =
+        getMTable(fks.get(i).getFktable_db(), fks.get(i).getFktable_name());
+      MColumnDescriptor parentColumn =
+        getColumnFromTable(parentTable, fks.get(i).getPkcolumn_name());
+      MColumnDescriptor childColumn =
+        getColumnFromTable(childTable, fks.get(i).getFkcolumn_name());
+      if (parentTable == null) {
+        throw new InvalidObjectException("Parent table not found: " + fks.get(i).getPktable_name());
+      }
+      if (childTable == null) {
+        throw new InvalidObjectException("Child table not found: " + fks.get(i).getFktable_name());
+      }
+      if (parentColumn == null) {
+        throw new InvalidObjectException("Parent column not found: " + fks.get(i).getPkcolumn_name());
+      }
+      if (childColumn == null) {
+        throw new InvalidObjectException("Child column not found" + fks.get(i).getFkcolumn_name());
+      }
+      if (fks.get(i).getFk_name() == null) {
+        // When there is no explicit foreign key name associated with the constraint and the key is composite,
+        // we expect the foreign keys to be send in order in the input list.
+        // Otherwise, the below code will break.
+        // If this is the first column of the FK constraint, generate the foreign key name
+        // NB: The below code can result in race condition where duplicate names can be generated (in theory).
+        // However, this scenario can be ignored for practical purposes because of
+        // the uniqueness of the generated constraint name.
+        if (fks.get(i).getKey_seq() == 1) {
+          currentConstraintName = generateConstraintName(fks.get(i).getFktable_db(), fks.get(i).getFktable_name(),
+            fks.get(i).getPktable_db(), fks.get(i).getPktable_name(),
+            fks.get(i).getPkcolumn_name(), fks.get(i).getFkcolumn_name(), "fk");
+        }
+      } else {
+        currentConstraintName = fks.get(i).getFk_name();
+      }
+      Integer updateRule = fks.get(i).getUpdate_rule();
+      Integer deleteRule = fks.get(i).getDelete_rule();
+      int enableValidateRely = (fks.get(i).isEnable_cstr() ? 4 : 0) +
+      (fks.get(i).isValidate_cstr() ? 2 : 0) + (fks.get(i).isRely_cstr() ? 1 : 0);
+      MConstraint mpkfk = new MConstraint(
+        currentConstraintName,
+        MConstraint.FOREIGN_KEY_CONSTRAINT,
+        fks.get(i).getKey_seq(),
+        deleteRule,
+        updateRule,
+        enableValidateRely,
+        parentTable,
+        childTable,
+        parentColumn,
+        childColumn
+      );
+      mpkfks.add(mpkfk);
+    }
+    pm.makePersistentAll(mpkfks);
+  }
+
+  private void addPrimaryKeys(List<SQLPrimaryKey> pks) throws InvalidObjectException,
+    MetaException {
+    List<MConstraint> mpks = new ArrayList<MConstraint>();
+    String constraintName = null;
+    for (int i = 0; i < pks.size(); i++) {
+      MTable parentTable =
+        getMTable(pks.get(i).getTable_db(), pks.get(i).getTable_name());
+      MColumnDescriptor parentColumn =
+        getColumnFromTable(parentTable, pks.get(i).getColumn_name());
+      if (parentTable == null) {
+        throw new InvalidObjectException("Parent table not found: " + pks.get(i).getTable_name());
+      }
+      if (parentColumn == null) {
+        throw new InvalidObjectException("Parent column not found: " + pks.get(i).getColumn_name());
+      }
+      if (getPrimaryKeyConstraintName(
+          parentTable.getDatabase().getName(), parentTable.getTableName()) != null) {
+        throw new MetaException(" Primary key already exists for: " +
+          parentTable.getDatabase().getName() + "." + pks.get(i).getTable_name());
+      }
+      if (pks.get(i).getPk_name() == null) {
+        if (pks.get(i).getKey_seq() == 1) {
+          constraintName = generateConstraintName(pks.get(i).getTable_db(), pks.get(i).getTable_name(),
+            pks.get(i).getColumn_name(), "pk");
+        }
+      } else {
+        constraintName = pks.get(i).getPk_name();
+      }
+      int enableValidateRely = (pks.get(i).isEnable_cstr() ? 4 : 0) +
+      (pks.get(i).isValidate_cstr() ? 2 : 0) + (pks.get(i).isRely_cstr() ? 1 : 0);
+      MConstraint mpk = new MConstraint(
+        constraintName,
+        MConstraint.PRIMARY_KEY_CONSTRAINT,
+        pks.get(i).getKey_seq(),
+        null,
+        null,
+        enableValidateRely,
+        parentTable,
+        null,
+        parentColumn,
+        null);
+      mpks.add(mpk);
+    }
+    pm.makePersistentAll(mpks);
+  }
+
   @Override
   public boolean addIndex(Index index) throws InvalidObjectException,
       MetaException {
@@ -7940,4 +8122,217 @@ public class ObjectStore implements RawStore, Configurable {
       }
     }
   }
+
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name) throws MetaException {
+    try {
+      return getPrimaryKeysInternal(db_name, tbl_name, true, true);
+    } catch (NoSuchObjectException e) {
+      throw new MetaException(e.getMessage());
+    }
+  }
+
+  protected List<SQLPrimaryKey> getPrimaryKeysInternal(final String db_name,
+    final String tbl_name,
+    boolean allowSql, boolean allowJdo)
+  throws MetaException, NoSuchObjectException {
+    return new GetListHelper<SQLPrimaryKey>(db_name, tbl_name, allowSql, allowJdo) {
+
+      @Override
+      protected List<SQLPrimaryKey> getSqlResult(GetHelper<List<SQLPrimaryKey>> ctx) throws MetaException {
+        return directSql.getPrimaryKeys(db_name, tbl_name);
+      }
+
+      @Override
+      protected List<SQLPrimaryKey> getJdoResult(
+        GetHelper<List<SQLPrimaryKey>> ctx) throws MetaException, NoSuchObjectException {
+        return getPrimaryKeysViaJdo(db_name, tbl_name);
+      }
+    }.run(false);
+  }
+
+  private List<SQLPrimaryKey> getPrimaryKeysViaJdo(String db_name, String tbl_name) throws MetaException {
+    boolean commited = false;
+    List<SQLPrimaryKey> primaryKeys = null;
+    Query query = null;
+    try {
+      openTransaction();
+      query = pm.newQuery(MConstraint.class,
+        "parentTable.tableName == tbl_name && parentTable.database.name == db_name &&"
+        + " constraintType == MConstraint.PRIMARY_KEY_CONSTRAINT");
+      query.declareParameters("java.lang.String tbl_name, java.lang.String db_name");
+      Collection<?> constraints = (Collection<?>) query.execute(tbl_name, db_name);
+      pm.retrieveAll(constraints);
+      primaryKeys = new ArrayList<SQLPrimaryKey>();
+      for (Iterator<?> i = constraints.iterator(); i.hasNext();) {
+        MConstraint currPK = (MConstraint) i.next();
+        int enableValidateRely = currPK.getEnableValidateRely();
+        boolean enable = (enableValidateRely & 4) != 0;
+        boolean validate = (enableValidateRely & 2) != 0;
+        boolean rely = (enableValidateRely & 1) != 0;
+        primaryKeys.add(new SQLPrimaryKey(db_name,
+         tbl_name,
+         currPK.getParentColumn().getCols().get(0).getName(),
+         currPK.getPosition(),
+         currPK.getConstraintName(), enable, validate, rely));
+      }
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return primaryKeys;
+  }
+
+  private String getPrimaryKeyConstraintName(String db_name, String tbl_name) throws MetaException {
+    boolean commited = false;
+    String ret = null;
+    Query query = null;
+
+    try {
+      openTransaction();
+      query = pm.newQuery(MConstraint.class,
+        "parentTable.tableName == tbl_name && parentTable.database.name == db_name &&"
+        + " constraintType == MConstraint.PRIMARY_KEY_CONSTRAINT");
+      query.declareParameters("java.lang.String tbl_name, java.lang.String db_name");
+      Collection<?> constraints = (Collection<?>) query.execute(tbl_name, db_name);
+      pm.retrieveAll(constraints);
+      for (Iterator<?> i = constraints.iterator(); i.hasNext();) {
+        MConstraint currPK = (MConstraint) i.next();
+        ret = currPK.getConstraintName();
+        break;
+      }
+      commited = commitTransaction();
+     } finally {
+       if (!commited) {
+        rollbackTransaction();
+       }
+       if (query != null) {
+        query.closeAll();
+       }
+     }
+     return ret;
+   }
+
+  @Override
+  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+    String parent_tbl_name, String foreign_db_name, String foreign_tbl_name) throws MetaException {
+    try {
+      return getForeignKeysInternal(parent_db_name,
+        parent_tbl_name, foreign_db_name, foreign_tbl_name, true, true);
+    } catch (NoSuchObjectException e) {
+      throw new MetaException(e.getMessage());
+    }
+  }
+
+  protected List<SQLForeignKey> getForeignKeysInternal(final String parent_db_name,
+    final String parent_tbl_name, final String foreign_db_name, final String foreign_tbl_name,
+    boolean allowSql, boolean allowJdo) throws MetaException, NoSuchObjectException {
+    return new GetListHelper<SQLForeignKey>(foreign_db_name, foreign_tbl_name, allowSql, allowJdo) {
+
+      @Override
+      protected List<SQLForeignKey> getSqlResult(GetHelper<List<SQLForeignKey>> ctx) throws MetaException {
+        return directSql.getForeignKeys(parent_db_name,
+          parent_tbl_name, foreign_db_name, foreign_tbl_name);
+      }
+
+      @Override
+      protected List<SQLForeignKey> getJdoResult(
+        GetHelper<List<SQLForeignKey>> ctx) throws MetaException, NoSuchObjectException {
+        return getForeignKeysViaJdo(parent_db_name,
+          parent_tbl_name, foreign_db_name, foreign_tbl_name);
+      }
+    }.run(false);
+  }
+
+  private List<SQLForeignKey> getForeignKeysViaJdo(String parent_db_name,
+    String parent_tbl_name, String foreign_db_name, String foreign_tbl_name) throws MetaException {
+    boolean commited = false;
+    List<SQLForeignKey> foreignKeys = null;
+    Collection<?> constraints = null;
+    Query query = null;
+    Map<String, String> tblToConstraint = new HashMap<String, String>();
+    try {
+      openTransaction();
+      String queryText = (parent_tbl_name != null ? "parentTable.tableName == parent_tbl_name &&" : "")
+        + (parent_db_name != null ? "parentTable.database.name == parent_db_name &&" : "")
+        + (foreign_tbl_name != null ? "childTable.tableName == foreign_tbl_name &&" : "")
+        + (parent_db_name != null ? "childTable.database.name == foreign_db_name &&" : "")
+        + "constraintType == MConstraint.FOREIGN_KEY_CONSTRAINT";
+      queryText = queryText.trim();
+      query = pm.newQuery(MConstraint.class, queryText);
+      String paramText = (parent_tbl_name == null ? "" : "java.lang.String parent_tbl_name,")
+        + (parent_db_name == null ? "" : " java.lang.String parent_db_name, ")
+        + (foreign_tbl_name == null ? "" : "java.lang.String foreign_tbl_name,")
+        + (foreign_db_name == null ? "" : " java.lang.String foreign_db_name");
+      paramText=paramText.trim();
+      if (paramText.endsWith(",")) {
+        paramText = paramText.substring(0, paramText.length()-1);
+      }
+      query.declareParameters(paramText);
+      List<String> params = new ArrayList<String>();
+      if (parent_tbl_name != null) {
+        params.add(parent_tbl_name);
+      }
+      if (parent_db_name != null) {
+        params.add(parent_db_name);
+      }
+      if (foreign_tbl_name != null) {
+        params.add(foreign_tbl_name);
+      }
+      if (parent_db_name != null) {
+        params.add(foreign_db_name);
+      }
+      if (params.size() == 0) {
+        constraints = (Collection<?>) query.execute();
+      } else {
+        constraints = (Collection<?>) query.executeWithArray(params);
+      }
+      pm.retrieveAll(constraints);
+      foreignKeys = new ArrayList<SQLForeignKey>();
+      for (Iterator<?> i = constraints.iterator(); i.hasNext();) {
+        MConstraint currPKFK = (MConstraint) i.next();
+        int enableValidateRely = currPKFK.getEnableValidateRely();
+        boolean enable = (enableValidateRely & 4) != 0;
+        boolean validate = (enableValidateRely & 2) != 0;
+        boolean rely = (enableValidateRely & 1) != 0;
+        String consolidatedtblName =
+          currPKFK.getParentTable().getDatabase().getName() + "." +
+          currPKFK.getParentTable().getTableName();
+        String pkName;
+        if (tblToConstraint.containsKey(consolidatedtblName)) {
+          pkName = tblToConstraint.get(consolidatedtblName);
+        } else {
+          pkName = getPrimaryKeyConstraintName(currPKFK.getParentTable().getDatabase().getName(),
+            currPKFK.getParentTable().getDatabase().getName());
+          tblToConstraint.put(consolidatedtblName, pkName);
+        }
+        foreignKeys.add(new SQLForeignKey(
+          currPKFK.getParentTable().getDatabase().getName(),
+          currPKFK.getParentTable().getDatabase().getName(),
+          currPKFK.getParentColumn().getCols().get(0).getName(),
+          currPKFK.getChildTable().getDatabase().getName(),
+          currPKFK.getChildTable().getTableName(),
+          currPKFK.getChildColumn().getCols().get(0).getName(),
+          currPKFK.getPosition(),
+          currPKFK.getUpdateRule(),
+          currPKFK.getDeleteRule(),
+          currPKFK.getConstraintName(), pkName, enable, validate, rely));
+      }
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return foreignKeys;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 e49f757..100c396 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -51,6 +51,8 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -663,4 +665,14 @@ public interface RawStore extends Configurable {
 
   @InterfaceStability.Evolving
   long getChangeVersion(String topic) throws MetaException;
+
+  public abstract List<SQLPrimaryKey> getPrimaryKeys(String db_name,
+    String tbl_name) throws MetaException;
+
+  public abstract List<SQLForeignKey> getForeignKeys(String parent_db_name,
+    String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
+    throws MetaException;
+
+  void createTableWithConstraints(Table tbl, List<SQLPrimaryKey> primaryKeys,
+    List<SQLForeignKey> foreignKeys) throws InvalidObjectException, MetaException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 a73dbeb..d4e5da4 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
@@ -63,6 +63,8 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -2591,4 +2593,26 @@ public class HBaseStore implements RawStore {
       commitOrRoleBack(commit);
     }
   }
+
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+    String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public void createTableWithConstraints(Table tbl,
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java
----------------------------------------------------------------------
diff --git a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java
new file mode 100644
index 0000000..3806e28
--- /dev/null
+++ b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java
@@ -0,0 +1,148 @@
+package org.apache.hadoop.hive.metastore.model;
+
+import java.io.Serializable;
+
+public class MConstraint
+{
+  String constraintName;
+  int constraintType;
+  int position;
+  Integer deleteRule;
+  Integer updateRule;
+  MTable parentTable;
+  MTable childTable;
+  MColumnDescriptor parentColumn;
+  MColumnDescriptor childColumn;
+  int enableValidateRely;
+
+  // 0 - Primary Key
+  // 1 - PK-FK relationship
+  public final static int PRIMARY_KEY_CONSTRAINT = 0;
+  public final static int FOREIGN_KEY_CONSTRAINT = 1;
+
+  @SuppressWarnings("serial")
+  public static class PK implements Serializable {
+    public String constraintName;
+    public int position;
+
+    public PK() {}
+
+    public PK(String constraintName, int position) {
+      this.constraintName = constraintName;
+      this.position = position;
+    }
+
+    public String toString() {
+      return constraintName+":"+position;
+    }
+
+    public int hashCode() {
+      return toString().hashCode();
+    }
+
+    public boolean equals(Object other) {
+      if (other != null && (other instanceof PK)) {
+        PK otherPK = (PK) other;
+        return otherPK.constraintName.equals(constraintName) && otherPK.position == position;
+      }
+      return false;
+    }
+  }
+
+  public MConstraint() {}
+
+  public MConstraint(String constraintName, int constraintType, int position, Integer deleteRule, Integer updateRule, int enableRelyValidate, MTable parentTable,
+    MTable childTable, MColumnDescriptor parentColumn,
+    MColumnDescriptor childColumn) {
+   this.constraintName = constraintName;
+   this.constraintType = constraintType;
+   this.parentColumn = parentColumn;
+   this.parentTable = parentTable;
+   this.childColumn = childColumn;
+   this.childTable = childTable;
+   this.position = position;
+   this.deleteRule = deleteRule;
+   this.updateRule = updateRule;
+   this.enableValidateRely = enableRelyValidate;
+  }
+
+  public String getConstraintName() {
+    return constraintName;
+  }
+
+  public void setConstraintName(String fkName) {
+    this.constraintName = fkName;
+  }
+
+  public int getConstraintType() {
+    return constraintType;
+  }
+
+  public void setConstraintType(int ct) {
+    this.constraintType = ct;
+  }
+
+  public int getPosition() {
+    return position;
+  }
+
+  public void setPosition(int po) {
+    this.position = po;
+  }
+
+  public Integer getDeleteRule() {
+    return deleteRule;
+  }
+
+  public void setDeleteRule(Integer de) {
+    this.deleteRule = de;
+  }
+
+  public int getEnableValidateRely() {
+    return enableValidateRely;
+  }
+
+  public void setEnableValidateRely(int enableValidateRely) {
+    this.enableValidateRely = enableValidateRely;
+  }
+
+  public Integer getUpdateRule() {
+    return updateRule;
+  }
+
+  public void setUpdateRule(Integer ur) {
+    this.updateRule = ur;
+  }
+
+  public MTable getChildTable() {
+    return childTable;
+  }
+
+  public void setChildTable(MTable ft) {
+    this.childTable = ft;
+  }
+
+  public MTable getParentTable() {
+    return parentTable;
+  }
+
+  public void setParentTable(MTable pt) {
+    this.parentTable = pt;
+  }
+
+  public MColumnDescriptor getChildColumn() {
+    return childColumn;
+  }
+
+  public void setChildColumn(MColumnDescriptor cc) {
+    this.childColumn = cc;
+  }
+
+  public MColumnDescriptor getParentColumn() {
+    return parentColumn;
+  }
+
+  public void setParentColumn(MColumnDescriptor pc) {
+    this.parentColumn = pc;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/model/package.jdo
----------------------------------------------------------------------
diff --git a/metastore/src/model/package.jdo b/metastore/src/model/package.jdo
index 7385a13..b40df39 100644
--- a/metastore/src/model/package.jdo
+++ b/metastore/src/model/package.jdo
@@ -184,6 +184,39 @@
       </field>
     </class>
 
+    <class name="MConstraint" identity-type="application" table="KEY_CONSTRAINTS" detachable="true" objectid-class="MConstraint$PK">
+      <field name="constraintName" primary-key="true">
+        <column name="CONSTRAINT_NAME"/>
+      </field>
+      <field name="position" primary-key="true">
+        <column name="POSITION"/>
+      </field>
+      <field name="childColumn">
+        <column name="CHILD_CD_ID"/>
+      </field>
+      <field name="childTable">
+        <column name="CHILD_TBL_ID"/>
+      </field>
+      <field name="parentColumn">
+        <column name="PARENT_CD_ID"/>
+      </field>
+      <field name="parentTable">
+        <column name="PARENT_TBL_ID"/>
+      </field>
+      <field name="constraintType">
+        <column name="CONSTRAINT_TYPE"/>
+      </field>
+      <field name="deleteRule">
+	<column name="DELETE_RULE"/>
+      </field>
+      <field name="updateRule">
+        <column name="UPDATE_RULE"/>
+      </field>
+      <field name="enableValidateRely">
+        <column name="ENABLE_VALIDATE_RELY"/>
+      </field>
+    </class>
+
     <class name="MSerDeInfo" identity-type="datastore" table="SERDES" detachable="true">
       <datastore-identity>
         <column name="SERDE_ID"/>

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 94ca835..86e7bea 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -49,6 +49,8 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -820,4 +822,26 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   public long getChangeVersion(String topic) throws MetaException {
     return 0;
   }
+
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+    String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public void createTableWithConstraints(Table tbl,
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 b108f95..5b32f00 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -50,6 +50,8 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -836,6 +838,28 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   public long getChangeVersion(String topic) throws MetaException {
     return 0;
   }
+
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+    String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public void createTableWithConstraints(Table tbl,
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 9a2322f..4dfa83d 100755
--- a/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
+++ b/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
@@ -51,6 +51,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   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 create_table_with_constraints(Table tbl,  primaryKeys,  foreignKeys)')
   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)')
@@ -110,6 +111,8 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   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('  PrimaryKeysResponse get_primary_keys(PrimaryKeysRequest request)')
+  print('  ForeignKeysResponse get_foreign_keys(ForeignKeysRequest request)')
   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)')
@@ -407,6 +410,12 @@ elif cmd == 'create_table_with_environment_context':
     sys.exit(1)
   pp.pprint(client.create_table_with_environment_context(eval(args[0]),eval(args[1]),))
 
+elif cmd == 'create_table_with_constraints':
+  if len(args) != 3:
+    print('create_table_with_constraints requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.create_table_with_constraints(eval(args[0]),eval(args[1]),eval(args[2]),))
+
 elif cmd == 'drop_table':
   if len(args) != 3:
     print('drop_table requires 3 args')
@@ -761,6 +770,18 @@ elif cmd == 'get_index_names':
     sys.exit(1)
   pp.pprint(client.get_index_names(args[0],args[1],eval(args[2]),))
 
+elif cmd == 'get_primary_keys':
+  if len(args) != 1:
+    print('get_primary_keys requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_primary_keys(eval(args[0]),))
+
+elif cmd == 'get_foreign_keys':
+  if len(args) != 1:
+    print('get_foreign_keys requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_foreign_keys(eval(args[0]),))
+
 elif cmd == 'update_table_column_statistics':
   if len(args) != 1:
     print('update_table_column_statistics requires 1 args')


[11/30] hive git commit: HIVE-13507. Improved logging for ptest. (Siddharth Seth, reviewed by Thejas M Nair)

Posted by jd...@apache.org.
HIVE-13507. Improved logging for ptest. (Siddharth Seth, reviewed by Thejas M Nair)


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

Branch: refs/heads/llap
Commit: fcc2e79511bb1a0db70f4a17ede6ade5e73b1f42
Parents: c3fee30
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 18 11:05:59 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Apr 18 11:05:59 2016 -0700

----------------------------------------------------------------------
 .../hive/ptest/execution/ExecutionPhase.java    |  2 +
 .../hive/ptest/execution/HostExecutor.java      | 48 ++++++++++++++++++--
 .../hive/ptest/execution/LocalCommand.java      | 18 +++++++-
 .../apache/hive/ptest/execution/PrepPhase.java  |  1 +
 .../apache/hive/ptest/execution/conf/Host.java  |  3 ++
 5 files changed, 65 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fcc2e795/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
index 3026ea0..6063afc 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
@@ -86,6 +86,8 @@ public class ExecutionPhase extends Phase {
         isolatedWorkQueue.add(batch);
       }
     }
+    logger.info("ParallelWorkQueueSize={}, IsolatedWorkQueueSize={}", parallelWorkQueue.size(),
+        isolatedWorkQueue.size());
     try {
       int expectedNumHosts = hostExecutors.size();
       initalizeHosts();

http://git-wip-us.apache.org/repos/asf/hive/blob/fcc2e795/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
index b05d2c2..735b261 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
@@ -29,6 +29,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Stopwatch;
 import org.apache.hive.ptest.execution.conf.Host;
 import org.apache.hive.ptest.execution.conf.TestBatch;
 import org.apache.hive.ptest.execution.ssh.RSyncCommand;
@@ -65,6 +66,8 @@ class HostExecutor {
   private final File mFailedTestLogDir;
   private final long mNumPollSeconds;
   private volatile boolean mShutdown;
+  private int numParallelBatchesProcessed = 0;
+  private int numIsolatedBatchesProcessed = 0;
   
   HostExecutor(Host host, String privateKey, ListeningExecutorService executor,
       SSHCommandExecutor sshCommandExecutor,
@@ -100,7 +103,18 @@ class HostExecutor {
     return mExecutor.submit(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        executeTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults);
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        mLogger.info("Starting SubmitTests on host {}", getHost());
+        try {
+          executeTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults);
+        } finally {
+          stopwatch.stop();
+          mLogger.info("Finishing submitTests on host: {}. ElapsedTime(seconds)={}," +
+              " NumParallelBatchesProcessed={}, NumIsolatedBatchesProcessed={}",
+              new Object[]{getHost().toString(),
+                  stopwatch.elapsed(TimeUnit.SECONDS), numParallelBatchesProcessed,
+                  numIsolatedBatchesProcessed});
+        }
         return null;
       }
 
@@ -143,6 +157,7 @@ class HostExecutor {
         @Override
         public Void call() throws Exception {
           TestBatch batch = null;
+          Stopwatch sw = Stopwatch.createUnstarted();
           try {
             do {
               batch = parallelWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
@@ -151,8 +166,16 @@ class HostExecutor {
                 return null;
               }
               if(batch != null) {
-                if(!executeTestBatch(drone, batch, failedTestResults)) {
-                  failedTestResults.add(batch);
+                numParallelBatchesProcessed++;
+                sw.reset().start();
+                try {
+                  if (!executeTestBatch(drone, batch, failedTestResults)) {
+                    failedTestResults.add(batch);
+                  }
+                } finally {
+                  sw.stop();
+                  mLogger.info("Finished processing parallel batch [{}] on host {}. ElapsedTime(seconds)={}",
+                      new Object[]{batch.getName(), getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
                 }
               }
             } while(!mShutdown && !parallelWorkQueue.isEmpty());
@@ -176,12 +199,22 @@ class HostExecutor {
     mLogger.info("Starting isolated execution on " + mHost.getName());
     for(Drone drone : ImmutableList.copyOf(mDrones)) {
       TestBatch batch = null;
+      Stopwatch sw = Stopwatch.createUnstarted();
       try {
         do {
+
           batch = isolatedWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
           if(batch != null) {
-            if(!executeTestBatch(drone, batch, failedTestResults)) {
-              failedTestResults.add(batch);
+            numIsolatedBatchesProcessed++;
+            sw.reset().start();
+            try {
+              if (!executeTestBatch(drone, batch, failedTestResults)) {
+                failedTestResults.add(batch);
+              }
+            } finally {
+              sw.stop();
+              mLogger.info("Finished processing isolated batch [{}] on host {}. ElapsedTime(seconds)={}",
+                  new Object[]{batch.getName(), getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
             }
           }
         } while(!mShutdown && !isolatedWorkQueue.isEmpty());
@@ -215,10 +248,15 @@ class HostExecutor {
     Templates.writeTemplateResult("batch-exec.vm", script, templateVariables);
     copyToDroneFromLocal(drone, script.getAbsolutePath(), "$localDir/$instanceName/scratch/" + scriptName);
     script.delete();
+    Stopwatch sw = Stopwatch.createStarted();
     mLogger.info(drone + " executing " + batch + " with " + command);
     RemoteCommandResult sshResult = new SSHCommand(mSSHCommandExecutor, drone.getPrivateKey(), drone.getUser(),
         drone.getHost(), drone.getInstance(), command, true).
         call();
+    sw.stop();
+    mLogger.info("Completed executing tests for batch [{}] on host {}. ElapsedTime(seconds)={}",
+        new Object[]{batch.getName(),
+            getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
     File batchLogDir = null;
     if(sshResult.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
       throw new AbortDroneException("Drone " + drone.toString() + " exited with " +

http://git-wip-us.apache.org/repos/asf/hive/blob/fcc2e795/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
index ec99656..8e2c5c7 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
@@ -22,17 +22,28 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.base.Stopwatch;
 import org.slf4j.Logger;
 
 public class LocalCommand {
 
+  private static final AtomicInteger localCommandCounter = new AtomicInteger(0);
+
+  private final Logger logger;
   private final Process process;
   private final StreamReader streamReader;
   private Integer exitCode;
+  private final int commandId;
+  private final Stopwatch stopwatch = Stopwatch.createUnstarted();
 
   public LocalCommand(Logger logger, OutputPolicy outputPolicy, String command) throws IOException {
-    logger.info("Starting " + command);
+    this.commandId = localCommandCounter.incrementAndGet();
+    this.logger = logger;
+    logger.info("Starting LocalCommandId={}: {}" + commandId, command);
+    stopwatch.start();
     process = new ProcessBuilder().command(new String[] {"bash", "-c", command}).redirectErrorStream(true).start();
     streamReader = new StreamReader(outputPolicy, process.getInputStream());
     streamReader.setName("StreamReader-[" + command + "]");
@@ -45,10 +56,13 @@ public class LocalCommand {
       if(exitCode == null) {
         exitCode = process.waitFor();
       }
+      stopwatch.stop();
+      logger.info("Finished LocalCommandId={}. ElapsedTime(seconds)={}", commandId, stopwatch.elapsed(
+          TimeUnit.SECONDS));
       return exitCode;
     }
   }
-  
+
   public void kill() {
     synchronized (process) {
       process.destroy();

http://git-wip-us.apache.org/repos/asf/hive/blob/fcc2e795/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
index 825f0c0..8fef413 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
@@ -62,6 +62,7 @@ public class PrepPhase extends Phase {
     // source prep
     start = System.currentTimeMillis();
     File sourcePrepScript = new File(mScratchDir, "source-prep.sh");
+    logger.info("Writing {} from template", sourcePrepScript);
     Templates.writeTemplateResult("source-prep.vm", sourcePrepScript, getTemplateDefaults());
     execLocally("bash " + sourcePrepScript.getPath());
     logger.debug("Deleting " + sourcePrepScript + ": " + sourcePrepScript.delete());

http://git-wip-us.apache.org/repos/asf/hive/blob/fcc2e795/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
index c1216c1..a56824c 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
@@ -47,6 +47,9 @@ public class Host {
   public String[] getLocalDirectories() {
     return localDirectories;
   }
+  public String toShortString() {
+    return name;
+  }
   @Override
   public String toString() {
     return "Host [name=" + name + ", user=" + user + ", threads=" + threads


[06/30] hive git commit: HIVE-13498: cleardanglingscratchdir does not work if scratchdir is not on defaultFs (Daniel Dai, reviewed by Thejas Nair)

Posted by jd...@apache.org.
HIVE-13498: cleardanglingscratchdir does not work if scratchdir is not on defaultFs (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/88cea1c2
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/88cea1c2
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/88cea1c2

Branch: refs/heads/llap
Commit: 88cea1c21d045e384c3deb628b32b93830b70e0c
Parents: d559b34
Author: Daniel Dai <da...@hortonworks.com>
Authored: Fri Apr 15 16:12:31 2016 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Fri Apr 15 16:12:54 2016 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/88cea1c2/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 8c6c46f..344dd34 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -658,7 +658,7 @@ public class SessionState {
     conf.set(HDFS_SESSION_PATH_KEY, hdfsSessionPath.toUri().toString());
     // 5. hold a lock file in HDFS session dir to indicate the it is in use
     if (conf.getBoolVar(HiveConf.ConfVars.HIVE_SCRATCH_DIR_LOCK)) {
-      FileSystem fs = FileSystem.get(conf);
+      FileSystem fs = hdfsSessionPath.getFileSystem(conf);
       hdfsSessionPathLockFile = fs.create(new Path(hdfsSessionPath, LOCK_FILE_NAME), true);
       hdfsSessionPathLockFile.writeUTF("hostname: " + InetAddress.getLocalHost().getHostName() + "\n");
       hdfsSessionPathLockFile.writeUTF("process: " + ManagementFactory.getRuntimeMXBean().getName() + "\n");


[04/30] hive git commit: Revert "HIVE-12159: Create vectorized readers for the complex types (Owen O'Malley, reviewed by Matt McCline)"

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
index 8ee8cd7..8bb32ea 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
@@ -24,7 +24,6 @@ import java.sql.Timestamp;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -36,12 +35,9 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
@@ -60,7 +56,8 @@ import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.orc.TypeDescription;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.orc.impl.BitFieldReader;
 import org.apache.orc.impl.DynamicByteArray;
 import org.apache.orc.impl.InStream;
@@ -78,6 +75,60 @@ import org.apache.orc.impl.StreamName;
  */
 public class TreeReaderFactory {
 
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TreeReaderFactory.class);
+
+  public static class TreeReaderSchema {
+
+    /**
+     * The types in the ORC file.
+     */
+    List<OrcProto.Type> fileTypes;
+
+    /**
+     * The treeReaderSchema that the reader should read as.
+     */
+    List<OrcProto.Type> schemaTypes;
+
+    /**
+     * The subtype of the row STRUCT.  Different than 0 for ACID.
+     */
+    int innerStructSubtype;
+
+    public TreeReaderSchema() {
+      fileTypes = null;
+      schemaTypes = null;
+      innerStructSubtype = -1;
+    }
+
+    public TreeReaderSchema fileTypes(List<OrcProto.Type> fileTypes) {
+      this.fileTypes = fileTypes;
+      return this;
+    }
+
+    public TreeReaderSchema schemaTypes(List<OrcProto.Type> schemaTypes) {
+      this.schemaTypes = schemaTypes;
+      return this;
+    }
+
+    public TreeReaderSchema innerStructSubtype(int innerStructSubtype) {
+      this.innerStructSubtype = innerStructSubtype;
+      return this;
+    }
+
+    public List<OrcProto.Type> getFileTypes() {
+      return fileTypes;
+    }
+
+    public List<OrcProto.Type> getSchemaTypes() {
+      return schemaTypes;
+    }
+
+    public int getInnerStructSubtype() {
+      return innerStructSubtype;
+    }
+  }
+
   public abstract static class TreeReader {
     protected final int columnId;
     protected BitFieldReader present = null;
@@ -179,60 +230,36 @@ public class TreeReaderFactory {
     }
 
     /**
-     * Called at the top level to read into the given batch.
-     * @param batch the batch to read into
-     * @param batchSize the number of rows to read
-     * @throws IOException
-     */
-    public void nextBatch(VectorizedRowBatch batch,
-                          int batchSize) throws IOException {
-      batch.cols[0].reset();
-      batch.cols[0].ensureSize(batchSize, false);
-      nextVector(batch.cols[0], null, batchSize);
-    }
-
-    /**
      * Populates the isNull vector array in the previousVector object based on
      * the present stream values. This function is called from all the child
      * readers, and they all set the values based on isNull field value.
      *
-     * @param previous The columnVector object whose isNull value is populated
-     * @param isNull Whether the each value was null at a higher level. If
-     *               isNull is null, all values are non-null.
+     * @param previousVector The columnVector object whose isNull value is populated
      * @param batchSize      Size of the column vector
+     * @return next column vector
      * @throws IOException
      */
-    public void nextVector(ColumnVector previous,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      if (present != null || isNull != null) {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      ColumnVector result = (ColumnVector) previousVector;
+      if (present != null) {
         // Set noNulls and isNull vector of the ColumnVector based on
         // present stream
-        previous.noNulls = true;
-        boolean allNull = true;
+        result.noNulls = true;
         for (int i = 0; i < batchSize; i++) {
-          if (isNull == null || !isNull[i]) {
-            if (present != null && present.next() != 1) {
-              previous.noNulls = false;
-              previous.isNull[i] = true;
-            } else {
-              previous.isNull[i] = false;
-              allNull = false;
-            }
-          } else {
-            previous.noNulls = false;
-            previous.isNull[i] = true;
+          result.isNull[i] = (present.next() != 1);
+          if (result.noNulls && result.isNull[i]) {
+            result.noNulls = false;
           }
         }
-        previous.isRepeating = !previous.noNulls && allNull;
       } else {
-        // There is no present stream, this means that all the values are
+        // There is not present stream, this means that all the values are
         // present.
-        previous.noNulls = true;
+        result.noNulls = true;
         for (int i = 0; i < batchSize; i++) {
-          previous.isNull[i] = false;
+          result.isNull[i] = false;
         }
       }
+      return previousVector;
     }
 
     public BitFieldReader getPresent() {
@@ -240,46 +267,6 @@ public class TreeReaderFactory {
     }
   }
 
-  public static class NullTreeReader extends TreeReader {
-
-    public NullTreeReader(int columnId) throws IOException {
-      super(columnId);
-    }
-
-    @Override
-    public void startStripe(Map<StreamName, InStream> streams,
-                            OrcProto.StripeFooter footer) {
-      // PASS
-    }
-
-    @Override
-    void skipRows(long rows) {
-      // PASS
-    }
-
-    @Override
-    public void seek(PositionProvider position) {
-      // PASS
-    }
-
-    @Override
-    public void seek(PositionProvider[] position) {
-      // PASS
-    }
-
-    @Override
-    Object next(Object previous) {
-      return null;
-    }
-
-    @Override
-    public void nextVector(ColumnVector vector, boolean[] isNull, int size) {
-      vector.noNulls = false;
-      vector.isNull[0] = true;
-      vector.isRepeating = true;
-    }
-  }
-
   public static class BooleanTreeReader extends TreeReader {
     protected BitFieldReader reader = null;
 
@@ -335,16 +322,20 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      LongColumnVector result = (LongColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final LongColumnVector result;
+      if (previousVector == null) {
+        result = new LongColumnVector();
+      } else {
+        result = (LongColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       // Read value entries based on isNull entries
       reader.nextVector(result, batchSize);
+      return result;
     }
   }
 
@@ -396,16 +387,20 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final LongColumnVector result;
+      if (previousVector == null) {
+        result = new LongColumnVector();
+      } else {
+        result = (LongColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
+      reader.nextVector(result, batchSize);
+      return result;
     }
 
     @Override
@@ -478,16 +473,20 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final LongColumnVector result;
+      if (previousVector == null) {
+        result = new LongColumnVector();
+      } else {
+        result = (LongColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
+      reader.nextVector(result, batchSize);
+      return result;
     }
 
     @Override
@@ -560,16 +559,20 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final LongColumnVector result;
+      if (previousVector == null) {
+        result = new LongColumnVector();
+      } else {
+        result = (LongColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
+      reader.nextVector(result, batchSize);
+      return result;
     }
 
     @Override
@@ -643,16 +646,20 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final LongColumnVector result;
+      if (previousVector == null) {
+        result = new LongColumnVector();
+      } else {
+        result = (LongColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
+      reader.nextVector(result, batchSize);
+      return result;
     }
 
     @Override
@@ -712,13 +719,16 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final DoubleColumnVector result = (DoubleColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final DoubleColumnVector result;
+      if (previousVector == null) {
+        result = new DoubleColumnVector();
+      } else {
+        result = (DoubleColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       final boolean hasNulls = !result.noNulls;
       boolean allNulls = hasNulls;
@@ -758,6 +768,7 @@ public class TreeReaderFactory {
         }
         result.isRepeating = repeating;
       }
+      return result;
     }
 
     @Override
@@ -821,13 +832,16 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final DoubleColumnVector result = (DoubleColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final DoubleColumnVector result;
+      if (previousVector == null) {
+        result = new DoubleColumnVector();
+      } else {
+        result = (DoubleColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       final boolean hasNulls = !result.noNulls;
       boolean allNulls = hasNulls;
@@ -867,6 +881,8 @@ public class TreeReaderFactory {
         }
         result.isRepeating = repeating;
       }
+
+      return result;
     }
 
     @Override
@@ -958,15 +974,19 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final BytesColumnVector result = (BytesColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final BytesColumnVector result;
+      if (previousVector == null) {
+        result = new BytesColumnVector();
+      } else {
+        result = (BytesColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, result, batchSize);
+      return result;
     }
 
     @Override
@@ -991,6 +1011,7 @@ public class TreeReaderFactory {
     private final TimeZone readerTimeZone;
     private TimeZone writerTimeZone;
     private boolean hasSameTZRules;
+    private TimestampWritable scratchTimestampWritable;
 
     TimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       this(columnId, null, null, null, null, skipCorrupt);
@@ -1094,9 +1115,9 @@ public class TreeReaderFactory {
         int newNanos = parseNanos(nanos.next());
         // fix the rounding when we divided by 1000.
         if (millis >= 0) {
-          millis += newNanos / WriterImpl.NANOS_PER_MILLI;
+          millis += newNanos / 1000000;
         } else {
-          millis -= newNanos / WriterImpl.NANOS_PER_MILLI;
+          millis -= newNanos / 1000000;
         }
         long offset = 0;
         // If reader and writer time zones have different rules, adjust the timezone difference
@@ -1123,45 +1144,31 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      TimestampColumnVector result = (TimestampColumnVector) previousVector;
-      super.nextVector(previousVector, isNull, batchSize);
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final TimestampColumnVector result;
+      if (previousVector == null) {
+        result = new TimestampColumnVector();
+      } else {
+        result = (TimestampColumnVector) previousVector;
+      }
 
+      result.reset();
+      if (scratchTimestampWritable == null) {
+        scratchTimestampWritable = new TimestampWritable();
+      }
+      Object obj;
       for (int i = 0; i < batchSize; i++) {
-        if (result.noNulls || !result.isNull[i]) {
-          long millis = data.next() + base_timestamp;
-          int newNanos = parseNanos(nanos.next());
-          if (millis < 0 && newNanos != 0) {
-            millis -= 1;
-          }
-          millis *= WriterImpl.MILLIS_PER_SECOND;
-          long offset = 0;
-          // If reader and writer time zones have different rules, adjust the timezone difference
-          // between reader and writer taking day light savings into account.
-          if (!hasSameTZRules) {
-            offset = writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(millis);
-          }
-          long adjustedMillis = millis + offset;
-          // Sometimes the reader timezone might have changed after adding the adjustedMillis.
-          // To account for that change, check for any difference in reader timezone after
-          // adding adjustedMillis. If so use the new offset (offset at adjustedMillis point of time).
-          if (!hasSameTZRules &&
-              (readerTimeZone.getOffset(millis) != readerTimeZone.getOffset(adjustedMillis))) {
-            long newOffset =
-                writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(adjustedMillis);
-            adjustedMillis = millis + newOffset;
-          }
-          result.time[i] = adjustedMillis;
-          result.nanos[i] = newNanos;
-          if (result.isRepeating && i != 0 &&
-              (result.time[0] != result.time[i] ||
-                  result.nanos[0] != result.nanos[i])) {
-            result.isRepeating = false;
-          }
+        obj = next(scratchTimestampWritable);
+        if (obj == null) {
+          result.noNulls = false;
+          result.isNull[i] = true;
+        } else {
+          TimestampWritable writable = (TimestampWritable) obj;
+          result.set(i, writable.getTimestamp());
         }
       }
+
+      return result;
     }
 
     private static int parseNanos(long serialized) {
@@ -1246,16 +1253,20 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final LongColumnVector result;
+      if (previousVector == null) {
+        result = new LongColumnVector();
+      } else {
+        result = (LongColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
+      reader.nextVector(result, batchSize);
+      return result;
     }
 
     @Override
@@ -1267,7 +1278,7 @@ public class TreeReaderFactory {
   public static class DecimalTreeReader extends TreeReader {
     protected InStream valueStream;
     protected IntegerReader scaleReader = null;
-    private int[] scratchScaleVector;
+    private LongColumnVector scratchScaleVector;
 
     private final int precision;
     private final int scale;
@@ -1282,7 +1293,7 @@ public class TreeReaderFactory {
       super(columnId, present);
       this.precision = precision;
       this.scale = scale;
-      this.scratchScaleVector = new int[VectorizedRowBatch.DEFAULT_SIZE];
+      this.scratchScaleVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
       this.valueStream = valueStream;
       if (scaleStream != null && encoding != null) {
         checkEncoding(encoding);
@@ -1341,34 +1352,46 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final DecimalColumnVector result = (DecimalColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final DecimalColumnVector result;
+      if (previousVector == null) {
+        result = new DecimalColumnVector(precision, scale);
+      } else {
+        result = (DecimalColumnVector) previousVector;
+      }
+
+      // Save the reference for isNull in the scratch vector
+      boolean[] scratchIsNull = scratchScaleVector.isNull;
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
-      if (batchSize > scratchScaleVector.length) {
-        scratchScaleVector = new int[(int) batchSize];
-      }
-      scaleReader.nextVector(result, scratchScaleVector, batchSize);
       // Read value entries based on isNull entries
-      if (result.noNulls) {
-        for (int r=0; r < batchSize; ++r) {
+      if (result.isRepeating) {
+        if (!result.isNull[0]) {
           BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
-          HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]);
-          result.set(r, dec);
+          short scaleInData = (short) scaleReader.next();
+          HiveDecimal dec = HiveDecimal.create(bInt, scaleInData);
+          dec = HiveDecimal.enforcePrecisionScale(dec, precision, scale);
+          result.set(0, dec);
         }
-      } else if (!result.isRepeating || !result.isNull[0]) {
-        for (int r=0; r < batchSize; ++r) {
-          if (!result.isNull[r]) {
+      } else {
+        // result vector has isNull values set, use the same to read scale vector.
+        scratchScaleVector.isNull = result.isNull;
+        scaleReader.nextVector(scratchScaleVector, batchSize);
+        for (int i = 0; i < batchSize; i++) {
+          if (!result.isNull[i]) {
             BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
-            HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]);
-            result.set(r, dec);
+            short scaleInData = (short) scratchScaleVector.vector[i];
+            HiveDecimal dec = HiveDecimal.create(bInt, scaleInData);
+            dec = HiveDecimal.enforcePrecisionScale(dec, precision, scale);
+            result.set(i, dec);
           }
         }
       }
+      // Switch back the null vector.
+      scratchScaleVector.isNull = scratchIsNull;
+      return result;
     }
 
     @Override
@@ -1458,10 +1481,8 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      reader.nextVector(previousVector, isNull, batchSize);
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      return reader.nextVector(previousVector, batchSize);
     }
 
     @Override
@@ -1480,7 +1501,7 @@ public class TreeReaderFactory {
         BytesColumnVector result, final int batchSize) throws IOException {
       // Read lengths
       scratchlcv.isNull = result.isNull;  // Notice we are replacing the isNull vector here...
-      lengths.nextVector(scratchlcv, scratchlcv.vector, batchSize);
+      lengths.nextVector(scratchlcv, batchSize);
       int totalLength = 0;
       if (!scratchlcv.isRepeating) {
         for (int i = 0; i < batchSize; i++) {
@@ -1511,35 +1532,31 @@ public class TreeReaderFactory {
     }
 
     // This method has the common code for reading in bytes into a BytesColumnVector.
-    public static void readOrcByteArrays(InStream stream,
-                                         IntegerReader lengths,
-                                         LongColumnVector scratchlcv,
-                                         BytesColumnVector result,
-                                         int batchSize) throws IOException {
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        byte[] allBytes = commonReadByteArrays(stream, lengths, scratchlcv,
-            result, (int) batchSize);
-
-        // Too expensive to figure out 'repeating' by comparisons.
-        result.isRepeating = false;
-        int offset = 0;
-        if (!scratchlcv.isRepeating) {
-          for (int i = 0; i < batchSize; i++) {
-            if (!scratchlcv.isNull[i]) {
-              result.setRef(i, allBytes, offset, (int) scratchlcv.vector[i]);
-              offset += scratchlcv.vector[i];
-            } else {
-              result.setRef(i, allBytes, 0, 0);
-            }
+    public static void readOrcByteArrays(InStream stream, IntegerReader lengths,
+        LongColumnVector scratchlcv,
+        BytesColumnVector result, final int batchSize) throws IOException {
+
+      byte[] allBytes = commonReadByteArrays(stream, lengths, scratchlcv, result, batchSize);
+
+      // Too expensive to figure out 'repeating' by comparisons.
+      result.isRepeating = false;
+      int offset = 0;
+      if (!scratchlcv.isRepeating) {
+        for (int i = 0; i < batchSize; i++) {
+          if (!scratchlcv.isNull[i]) {
+            result.setRef(i, allBytes, offset, (int) scratchlcv.vector[i]);
+            offset += scratchlcv.vector[i];
+          } else {
+            result.setRef(i, allBytes, 0, 0);
           }
-        } else {
-          for (int i = 0; i < batchSize; i++) {
-            if (!scratchlcv.isNull[i]) {
-              result.setRef(i, allBytes, offset, (int) scratchlcv.vector[0]);
-              offset += scratchlcv.vector[0];
-            } else {
-              result.setRef(i, allBytes, 0, 0);
-            }
+        }
+      } else {
+        for (int i = 0; i < batchSize; i++) {
+          if (!scratchlcv.isNull[i]) {
+            result.setRef(i, allBytes, offset, (int) scratchlcv.vector[0]);
+            offset += scratchlcv.vector[0];
+          } else {
+            result.setRef(i, allBytes, 0, 0);
           }
         }
       }
@@ -1624,16 +1641,19 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final BytesColumnVector result = (BytesColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final BytesColumnVector result;
+      if (previousVector == null) {
+        result = new BytesColumnVector();
+      } else {
+        result = (BytesColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
-      BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv,
-          result, batchSize);
+      BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, result, batchSize);
+      return result;
     }
 
     @Override
@@ -1796,15 +1816,18 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      final BytesColumnVector result = (BytesColumnVector) previousVector;
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final BytesColumnVector result;
       int offset;
       int length;
+      if (previousVector == null) {
+        result = new BytesColumnVector();
+      } else {
+        result = (BytesColumnVector) previousVector;
+      }
 
       // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
+      super.nextVector(result, batchSize);
 
       if (dictionaryBuffer != null) {
 
@@ -1815,8 +1838,7 @@ public class TreeReaderFactory {
 
         // Read string offsets
         scratchlcv.isNull = result.isNull;
-        scratchlcv.ensureSize((int) batchSize, false);
-        reader.nextVector(scratchlcv, scratchlcv.vector, batchSize);
+        reader.nextVector(scratchlcv, batchSize);
         if (!scratchlcv.isRepeating) {
 
           // The vector has non-repeating strings. Iterate thru the batch
@@ -1856,6 +1878,7 @@ public class TreeReaderFactory {
           }
         }
       }
+      return result;
     }
 
     int getDictionaryEntryLength(int entry, int offset) {
@@ -1913,13 +1936,11 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       // Get the vector of strings from StringTreeReader, then make a 2nd pass to
       // adjust down the length (right trim and truncate) if necessary.
-      super.nextVector(previousVector, isNull, batchSize);
-      BytesColumnVector result = (BytesColumnVector) previousVector;
+      BytesColumnVector result = (BytesColumnVector) super.nextVector(previousVector, batchSize);
+
       int adjustedDownLen;
       if (result.isRepeating) {
         if (result.noNulls || !result.isNull[0]) {
@@ -1952,6 +1973,7 @@ public class TreeReaderFactory {
           }
         }
       }
+      return result;
     }
   }
 
@@ -1988,13 +2010,10 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       // Get the vector of strings from StringTreeReader, then make a 2nd pass to
       // adjust down the length (truncate) if necessary.
-      super.nextVector(previousVector, isNull, batchSize);
-      BytesColumnVector result = (BytesColumnVector) previousVector;
+      BytesColumnVector result = (BytesColumnVector) super.nextVector(previousVector, batchSize);
 
       int adjustedDownLen;
       if (result.isRepeating) {
@@ -2026,26 +2045,62 @@ public class TreeReaderFactory {
           }
         }
       }
+      return result;
     }
   }
 
   protected static class StructTreeReader extends TreeReader {
+    private final int readColumnCount;
+    private final int resultColumnCount;
     protected final TreeReader[] fields;
+    private final String[] fieldNames;
 
-    protected StructTreeReader(int columnId,
-                               TypeDescription readerSchema,
-                               SchemaEvolution treeReaderSchema,
-                               boolean[] included,
-                               boolean skipCorrupt) throws IOException {
+    protected StructTreeReader(
+        int columnId,
+        TreeReaderSchema treeReaderSchema,
+        boolean[] included,
+        boolean skipCorrupt) throws IOException {
       super(columnId);
 
-      TypeDescription fileSchema = treeReaderSchema.getFileType(readerSchema);
+      OrcProto.Type fileStructType = treeReaderSchema.getFileTypes().get(columnId);
+
+      OrcProto.Type schemaStructType = treeReaderSchema.getSchemaTypes().get(columnId);
 
-      List<TypeDescription> childrenTypes = readerSchema.getChildren();
-      this.fields = new TreeReader[childrenTypes.size()];
-      for (int i = 0; i < fields.length; ++i) {
-        TypeDescription subtype = childrenTypes.get(i);
-        this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt);
+      readColumnCount = Math.min(fileStructType.getFieldNamesCount(), schemaStructType.getFieldNamesCount());
+
+      if (columnId == treeReaderSchema.getInnerStructSubtype()) {
+        // If there are more result columns than reader columns, we will default those additional
+        // columns to NULL.
+        resultColumnCount = schemaStructType.getFieldNamesCount();
+      } else {
+        resultColumnCount = readColumnCount;
+      }
+
+      this.fields = new TreeReader[readColumnCount];
+      this.fieldNames = new String[readColumnCount];
+
+      if (included == null) {
+        for (int i = 0; i < readColumnCount; ++i) {
+          int subtype = schemaStructType.getSubtypes(i);
+          this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt);
+          // Use the treeReaderSchema evolution name since file/reader types may not have the real column name.
+          this.fieldNames[i] = schemaStructType.getFieldNames(i);
+        }
+      } else {
+        for (int i = 0; i < readColumnCount; ++i) {
+          int subtype = schemaStructType.getSubtypes(i);
+          if (subtype >= included.length) {
+            throw new IOException("subtype " + subtype + " exceeds the included array size " +
+                included.length + " fileTypes " + treeReaderSchema.getFileTypes().toString() +
+                " schemaTypes " + treeReaderSchema.getSchemaTypes().toString() +
+                " innerStructSubtype " + treeReaderSchema.getInnerStructSubtype());
+          }
+          if (included[subtype]) {
+            this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt);
+          }
+          // Use the treeReaderSchema evolution name since file/reader types may not have the real column name.
+          this.fieldNames[i] = schemaStructType.getFieldNames(i);
+        }
       }
     }
 
@@ -2065,52 +2120,65 @@ public class TreeReaderFactory {
       OrcStruct result = null;
       if (valuePresent) {
         if (previous == null) {
-          result = new OrcStruct(fields.length);
+          result = new OrcStruct(resultColumnCount);
         } else {
           result = (OrcStruct) previous;
 
           // If the input format was initialized with a file with a
           // different number of fields, the number of fields needs to
           // be updated to the correct number
-          result.setNumFields(fields.length);
+          if (result.getNumFields() != resultColumnCount) {
+            result.setNumFields(resultColumnCount);
+          }
         }
-        for (int i = 0; i < fields.length; ++i) {
+        for (int i = 0; i < readColumnCount; ++i) {
           if (fields[i] != null) {
             result.setFieldValue(i, fields[i].next(result.getFieldValue(i)));
           }
         }
+        if (resultColumnCount > readColumnCount) {
+          for (int i = readColumnCount; i < resultColumnCount; ++i) {
+            // Default new treeReaderSchema evolution fields to NULL.
+            result.setFieldValue(i, null);
+          }
+        }
       }
       return result;
     }
 
     @Override
-    public void nextBatch(VectorizedRowBatch batch,
-                          int batchSize) throws IOException {
-      for(int i=0; i < fields.length &&
-          (vectorColumnCount == -1 || i < vectorColumnCount); ++i) {
-        batch.cols[i].reset();
-        batch.cols[i].ensureSize((int) batchSize, false);
-        fields[i].nextVector(batch.cols[i], null, batchSize);
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      final ColumnVector[] result;
+      if (previousVector == null) {
+        result = new ColumnVector[readColumnCount];
+      } else {
+        result = (ColumnVector[]) previousVector;
       }
-    }
 
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      super.nextVector(previousVector, isNull, batchSize);
-      StructColumnVector result = (StructColumnVector) previousVector;
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        result.isRepeating = false;
+      // Read all the members of struct as column vectors
+      for (int i = 0; i < readColumnCount; i++) {
+        if (fields[i] != null) {
+          if (result[i] == null) {
+            result[i] = (ColumnVector) fields[i].nextVector(null, batchSize);
+          } else {
+            fields[i].nextVector(result[i], batchSize);
+          }
+        }
+      }
 
-        // Read all the members of struct as column vectors
-        boolean[] mask = result.noNulls ? null : result.isNull;
-        for (int f = 0; f < fields.length; f++) {
-          if (fields[f] != null) {
-            fields[f].nextVector(result.fields[f], mask, batchSize);
+      // Default additional treeReaderSchema evolution fields to NULL.
+      if (vectorColumnCount != -1 && vectorColumnCount > readColumnCount) {
+        for (int i = readColumnCount; i < vectorColumnCount; ++i) {
+          ColumnVector colVector = result[i];
+          if (colVector != null) {
+            colVector.isRepeating = true;
+            colVector.noNulls = false;
+            colVector.isNull[0] = true;
           }
         }
       }
+
+      return result;
     }
 
     @Override
@@ -2140,18 +2208,19 @@ public class TreeReaderFactory {
     protected final TreeReader[] fields;
     protected RunLengthByteReader tags;
 
-    protected UnionTreeReader(int fileColumn,
-                              TypeDescription readerSchema,
-                              SchemaEvolution treeReaderSchema,
-                              boolean[] included,
-                              boolean skipCorrupt) throws IOException {
-      super(fileColumn);
-      List<TypeDescription> childrenTypes = readerSchema.getChildren();
-      int fieldCount = childrenTypes.size();
+    protected UnionTreeReader(int columnId,
+        TreeReaderSchema treeReaderSchema,
+        boolean[] included,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      OrcProto.Type type = treeReaderSchema.getSchemaTypes().get(columnId);
+      int fieldCount = type.getSubtypesCount();
       this.fields = new TreeReader[fieldCount];
       for (int i = 0; i < fieldCount; ++i) {
-        TypeDescription subtype = childrenTypes.get(i);
-        this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt);
+        int subtype = type.getSubtypes(i);
+        if (included == null || included[subtype]) {
+          this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt);
+        }
       }
     }
 
@@ -2183,25 +2252,9 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      UnionColumnVector result = (UnionColumnVector) previousVector;
-      super.nextVector(result, isNull, batchSize);
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        result.isRepeating = false;
-        tags.nextVector(result.noNulls ? null : result.isNull, result.tags,
-            batchSize);
-        boolean[] ignore = new boolean[(int) batchSize];
-        for (int f = 0; f < result.fields.length; ++f) {
-          // build the ignore list for this tag
-          for (int r = 0; r < batchSize; ++r) {
-            ignore[r] = (!result.noNulls && result.isNull[r]) ||
-                result.tags[r] != f;
-          }
-          fields[f].nextVector(result.fields[f], ignore, batchSize);
-        }
-      }
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
+      throw new UnsupportedOperationException(
+          "NextVector is not supported operation for Union type");
     }
 
     @Override
@@ -2235,15 +2288,13 @@ public class TreeReaderFactory {
     protected final TreeReader elementReader;
     protected IntegerReader lengths = null;
 
-    protected ListTreeReader(int fileColumn,
-                             TypeDescription readerSchema,
-                             SchemaEvolution treeReaderSchema,
-                             boolean[] included,
-                             boolean skipCorrupt) throws IOException {
-      super(fileColumn);
-      TypeDescription elementType = readerSchema.getChildren().get(0);
-      elementReader = createTreeReader(elementType, treeReaderSchema, included,
-          skipCorrupt);
+    protected ListTreeReader(int columnId,
+        TreeReaderSchema treeReaderSchema,
+        boolean[] included,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      OrcProto.Type type = treeReaderSchema.getSchemaTypes().get(columnId);
+      elementReader = createTreeReader(type.getSubtypes(0), treeReaderSchema, included, skipCorrupt);
     }
 
     @Override
@@ -2284,27 +2335,9 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previous,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      ListColumnVector result = (ListColumnVector) previous;
-      super.nextVector(result, isNull, batchSize);
-      // if we have some none-null values, then read them
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        lengths.nextVector(result, result.lengths, batchSize);
-        // even with repeating lengths, the list doesn't repeat
-        result.isRepeating = false;
-        // build the offsets vector and figure out how many children to read
-        result.childCount = 0;
-        for (int r = 0; r < batchSize; ++r) {
-          if (result.noNulls || !result.isNull[r]) {
-            result.offsets[r] = result.childCount;
-            result.childCount += result.lengths[r];
-          }
-        }
-        result.child.ensureSize(result.childCount, false);
-        elementReader.nextVector(result.child, null, result.childCount);
-      }
+    public Object nextVector(Object previous, final int batchSize) throws IOException {
+      throw new UnsupportedOperationException(
+          "NextVector is not supported operation for List type");
     }
 
     @Override
@@ -2345,16 +2378,24 @@ public class TreeReaderFactory {
     protected final TreeReader valueReader;
     protected IntegerReader lengths = null;
 
-    protected MapTreeReader(int fileColumn,
-                            TypeDescription readerSchema,
-                            SchemaEvolution treeReaderSchema,
-                            boolean[] included,
-                            boolean skipCorrupt) throws IOException {
-      super(fileColumn);
-      TypeDescription keyType = readerSchema.getChildren().get(0);
-      TypeDescription valueType = readerSchema.getChildren().get(1);
-      keyReader = createTreeReader(keyType, treeReaderSchema, included, skipCorrupt);
-      valueReader = createTreeReader(valueType, treeReaderSchema, included, skipCorrupt);
+    protected MapTreeReader(int columnId,
+        TreeReaderSchema treeReaderSchema,
+        boolean[] included,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      OrcProto.Type type = treeReaderSchema.getSchemaTypes().get(columnId);
+      int keyColumn = type.getSubtypes(0);
+      int valueColumn = type.getSubtypes(1);
+      if (included == null || included[keyColumn]) {
+        keyReader = createTreeReader(keyColumn, treeReaderSchema, included, skipCorrupt);
+      } else {
+        keyReader = null;
+      }
+      if (included == null || included[valueColumn]) {
+        valueReader = createTreeReader(valueColumn, treeReaderSchema, included, skipCorrupt);
+      } else {
+        valueReader = null;
+      }
     }
 
     @Override
@@ -2388,28 +2429,9 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public void nextVector(ColumnVector previous,
-                           boolean[] isNull,
-                           int batchSize) throws IOException {
-      MapColumnVector result = (MapColumnVector) previous;
-      super.nextVector(result, isNull, batchSize);
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        lengths.nextVector(result, result.lengths, batchSize);
-        // even with repeating lengths, the map doesn't repeat
-        result.isRepeating = false;
-        // build the offsets vector and figure out how many children to read
-        result.childCount = 0;
-        for (int r = 0; r < batchSize; ++r) {
-          if (result.noNulls || !result.isNull[r]) {
-            result.offsets[r] = result.childCount;
-            result.childCount += result.lengths[r];
-          }
-        }
-        result.keys.ensureSize(result.childCount, false);
-        result.values.ensureSize(result.childCount, false);
-        keyReader.nextVector(result.keys, null, result.childCount);
-        valueReader.nextVector(result.values, null, result.childCount);
-      }
+    public Object nextVector(Object previous, final int batchSize) throws IOException {
+      throw new UnsupportedOperationException(
+          "NextVector is not supported operation for Map type");
     }
 
     @Override
@@ -2449,61 +2471,61 @@ public class TreeReaderFactory {
     }
   }
 
-  public static TreeReader createTreeReader(TypeDescription readerType,
-                                            SchemaEvolution evolution,
-                                            boolean[] included,
-                                            boolean skipCorrupt
-                                            ) throws IOException {
-    TypeDescription fileType = evolution.getFileType(readerType);
-    if (fileType == null ||
-        (included != null && !included[readerType.getId()])) {
-      return new NullTreeReader(0);
-    }
-    switch (readerType.getCategory()) {
+  public static TreeReader createTreeReader(int columnId,
+      TreeReaderSchema treeReaderSchema,
+      boolean[] included,
+      boolean skipCorrupt
+  ) throws IOException {
+    OrcProto.Type type = treeReaderSchema.getSchemaTypes().get(columnId);
+    switch (type.getKind()) {
       case BOOLEAN:
-        return new BooleanTreeReader(fileType.getId());
+        return new BooleanTreeReader(columnId);
       case BYTE:
-        return new ByteTreeReader(fileType.getId());
+        return new ByteTreeReader(columnId);
       case DOUBLE:
-        return new DoubleTreeReader(fileType.getId());
+        return new DoubleTreeReader(columnId);
       case FLOAT:
-        return new FloatTreeReader(fileType.getId());
+        return new FloatTreeReader(columnId);
       case SHORT:
-        return new ShortTreeReader(fileType.getId());
+        return new ShortTreeReader(columnId);
       case INT:
-        return new IntTreeReader(fileType.getId());
+        return new IntTreeReader(columnId);
       case LONG:
-        return new LongTreeReader(fileType.getId(), skipCorrupt);
+        return new LongTreeReader(columnId, skipCorrupt);
       case STRING:
-        return new StringTreeReader(fileType.getId());
+        return new StringTreeReader(columnId);
       case CHAR:
-        return new CharTreeReader(fileType.getId(), readerType.getMaxLength());
+        if (!type.hasMaximumLength()) {
+          throw new IllegalArgumentException("ORC char type has no length specified");
+        }
+        return new CharTreeReader(columnId, type.getMaximumLength());
       case VARCHAR:
-        return new VarcharTreeReader(fileType.getId(), readerType.getMaxLength());
+        if (!type.hasMaximumLength()) {
+          throw new IllegalArgumentException("ORC varchar type has no length specified");
+        }
+        return new VarcharTreeReader(columnId, type.getMaximumLength());
       case BINARY:
-        return new BinaryTreeReader(fileType.getId());
+        return new BinaryTreeReader(columnId);
       case TIMESTAMP:
-        return new TimestampTreeReader(fileType.getId(), skipCorrupt);
+        return new TimestampTreeReader(columnId, skipCorrupt);
       case DATE:
-        return new DateTreeReader(fileType.getId());
+        return new DateTreeReader(columnId);
       case DECIMAL:
-        return new DecimalTreeReader(fileType.getId(), readerType.getPrecision(),
-            readerType.getScale());
+        int precision =
+            type.hasPrecision() ? type.getPrecision() : HiveDecimal.SYSTEM_DEFAULT_PRECISION;
+        int scale = type.hasScale() ? type.getScale() : HiveDecimal.SYSTEM_DEFAULT_SCALE;
+        return new DecimalTreeReader(columnId, precision, scale);
       case STRUCT:
-        return new StructTreeReader(fileType.getId(), readerType,
-            evolution, included, skipCorrupt);
+        return new StructTreeReader(columnId, treeReaderSchema, included, skipCorrupt);
       case LIST:
-        return new ListTreeReader(fileType.getId(), readerType,
-            evolution, included, skipCorrupt);
+        return new ListTreeReader(columnId, treeReaderSchema, included, skipCorrupt);
       case MAP:
-        return new MapTreeReader(fileType.getId(), readerType, evolution,
-            included, skipCorrupt);
+        return new MapTreeReader(columnId, treeReaderSchema, included, skipCorrupt);
       case UNION:
-        return new UnionTreeReader(fileType.getId(), readerType,
-            evolution, included, skipCorrupt);
+        return new UnionTreeReader(columnId, treeReaderSchema, included, skipCorrupt);
       default:
         throw new IllegalArgumentException("Unsupported type " +
-            readerType.getCategory());
+            type.getKind());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java
index e4d2e6e..816b52d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java
@@ -71,29 +71,14 @@ public class VectorizedOrcInputFormat extends FileInputFormat<NullWritable, Vect
         OrcInputFormat.raiseAcidTablesMustBeReadWithAcidReaderException(conf);
       }
 
-      rbCtx = Utilities.getVectorizedRowBatchCtx(conf);
       /**
        * Do we have schema on read in the configuration variables?
        */
-      List<OrcProto.Type> types = file.getTypes();
-      int dataColumns = rbCtx.getDataColumnCount();
-      TypeDescription schema =
-          OrcInputFormat.getDesiredRowTypeDescr(conf, false, dataColumns);
-      if (schema == null) {
-        schema = file.getSchema();
-        // Even if the user isn't doing schema evolution, cut the schema
-        // to the desired size.
-        if (schema.getCategory() == TypeDescription.Category.STRUCT &&
-            schema.getChildren().size() > dataColumns) {
-          schema = schema.clone();
-          List<TypeDescription> children = schema.getChildren();
-          for(int c = children.size() - 1; c >= dataColumns; --c) {
-            children.remove(c);
-          }
-        }
-      }
-      Reader.Options options = new Reader.Options().schema(schema);
+      TypeDescription schema = OrcInputFormat.getDesiredRowTypeDescr(conf, /* isAcidRead */ false);
 
+      List<OrcProto.Type> types = file.getTypes();
+      Reader.Options options = new Reader.Options();
+      options.schema(schema);
       this.offset = fileSplit.getStart();
       this.length = fileSplit.getLength();
       options.range(offset, length);
@@ -102,6 +87,8 @@ public class VectorizedOrcInputFormat extends FileInputFormat<NullWritable, Vect
 
       this.reader = file.rowsOptions(options);
 
+      rbCtx = Utilities.getVectorizedRowBatchCtx(conf);
+
       columnsToIncludeTruncated = rbCtx.getColumnsToIncludeTruncated(conf);
 
       int partitionColumnCount = rbCtx.getPartitionColumnCount();
@@ -116,6 +103,9 @@ public class VectorizedOrcInputFormat extends FileInputFormat<NullWritable, Vect
     @Override
     public boolean next(NullWritable key, VectorizedRowBatch value) throws IOException {
 
+      if (!reader.hasNext()) {
+        return false;
+      }
       try {
         // Check and update partition cols if necessary. Ideally, this should be done
         // in CreateValue as the partition is constant per split. But since Hive uses
@@ -128,9 +118,7 @@ public class VectorizedOrcInputFormat extends FileInputFormat<NullWritable, Vect
           }
           addPartitionCols = false;
         }
-        if (!reader.nextBatch(value)) {
-          return false;
-        }
+        reader.nextBatch(value);
       } catch (Exception e) {
         throw new RuntimeException(e);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
index 8e52907..70fe803 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
@@ -101,6 +101,8 @@ public class WriterImpl extends org.apache.orc.impl.WriterImpl implements Writer
     }
   }
 
+  private static final long NANOS_PER_MILLI = 1000000;
+
   /**
    * Set the value for a given column value within a batch.
    * @param rowId the row to set

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
index 96af65a..2a82092 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
@@ -51,11 +51,11 @@ public class TestTypeDescription {
             .addField("f4", TypeDescription.createDouble())
             .addField("f5", TypeDescription.createBoolean()))
         .addField("f6", TypeDescription.createChar().withMaxLength(100));
-    assertEquals("struct<f1:uniontype<tinyint,decimal(20,10)>,f2:struct<f3:date,f4:double,f5:boolean>,f6:char(100)>",
+    assertEquals("struct<f1:union<tinyint,decimal(20,10)>,f2:struct<f3:date,f4:double,f5:boolean>,f6:char(100)>",
         struct.toString());
     assertEquals(
         "{\"category\": \"struct\", \"id\": 0, \"max\": 8, \"fields\": [\n" +
-            "  \"f1\": {\"category\": \"uniontype\", \"id\": 1, \"max\": 3, \"children\": [\n" +
+            "  \"f1\": {\"category\": \"union\", \"id\": 1, \"max\": 3, \"children\": [\n" +
             "    {\"category\": \"tinyint\", \"id\": 2, \"max\": 2},\n" +
             "    {\"category\": \"decimal\", \"id\": 3, \"max\": 3, \"precision\": 20, \"scale\": 10}]},\n" +
             "  \"f2\": {\"category\": \"struct\", \"id\": 4, \"max\": 7, \"fields\": [\n" +


[12/30] hive git commit: HIVE-13523: Fix connection leak in ORC RecordReader and refactor for unit testing (Thomas Poepping reviewed by Prasanth Jayachandran)

Posted by jd...@apache.org.
HIVE-13523: Fix connection leak in ORC RecordReader and refactor for unit testing (Thomas Poepping reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/llap
Commit: 72b6bcd78867aa26bf0d220be98d1e7850e0d00e
Parents: fcc2e79
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Mon Apr 18 15:53:34 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Mon Apr 18 15:53:52 2016 -0500

----------------------------------------------------------------------
 orc/src/java/org/apache/orc/DataReader.java     |   6 +-
 .../java/org/apache/orc/DataReaderFactory.java  |   9 ++
 .../org/apache/orc/MetadataReaderFactory.java   |  12 ++
 .../apache/orc/impl/DataReaderProperties.java   |  84 ++++++++++
 .../orc/impl/DefaultMetadataReaderFactory.java  |  14 ++
 .../org/apache/orc/impl/MetadataReader.java     |   5 +-
 .../org/apache/orc/impl/MetadataReaderImpl.java |  17 +-
 .../orc/impl/MetadataReaderProperties.java      |  96 ++++++++++++
 .../orc/impl/TestDataReaderProperties.java      |  69 +++++++++
 .../orc/impl/TestMetadataReaderProperties.java  |  72 +++++++++
 .../ql/io/orc/DefaultDataReaderFactory.java     |  14 ++
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |  40 ++++-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java | 155 ++++++++++++++++---
 .../hive/ql/io/orc/RecordReaderUtils.java       |  17 +-
 .../hive/ql/io/orc/TestRecordReaderImpl.java    | 145 ++++++++++++-----
 15 files changed, 658 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/java/org/apache/orc/DataReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/DataReader.java b/orc/src/java/org/apache/orc/DataReader.java
index 3a5f854..b70f26b 100644
--- a/orc/src/java/org/apache/orc/DataReader.java
+++ b/orc/src/java/org/apache/orc/DataReader.java
@@ -18,20 +18,18 @@
 
 package org.apache.orc;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hive.common.io.DiskRangeList;
 
 /** An abstract data reader that IO formats can use to read bytes from underlying storage. */
-public interface DataReader {
+public interface DataReader extends Closeable {
 
   /** Opens the DataReader, making it ready to use. */
   void open() throws IOException;
 
-  /** Closes the DataReader. */
-  void close() throws IOException;
-
   /** Reads the data.
    *
    * Note that for the cases such as zero-copy read, caller must release the disk ranges

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/java/org/apache/orc/DataReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/DataReaderFactory.java b/orc/src/java/org/apache/orc/DataReaderFactory.java
new file mode 100644
index 0000000..ec3a0e9
--- /dev/null
+++ b/orc/src/java/org/apache/orc/DataReaderFactory.java
@@ -0,0 +1,9 @@
+package org.apache.orc;
+
+import org.apache.orc.impl.DataReaderProperties;
+
+public interface DataReaderFactory {
+
+  DataReader create(DataReaderProperties properties);
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/java/org/apache/orc/MetadataReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/MetadataReaderFactory.java b/orc/src/java/org/apache/orc/MetadataReaderFactory.java
new file mode 100644
index 0000000..64629da
--- /dev/null
+++ b/orc/src/java/org/apache/orc/MetadataReaderFactory.java
@@ -0,0 +1,12 @@
+package org.apache.orc;
+
+import org.apache.orc.impl.MetadataReader;
+import org.apache.orc.impl.MetadataReaderProperties;
+
+import java.io.IOException;
+
+public interface MetadataReaderFactory {
+
+  MetadataReader create(MetadataReaderProperties properties) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/java/org/apache/orc/impl/DataReaderProperties.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/DataReaderProperties.java b/orc/src/java/org/apache/orc/impl/DataReaderProperties.java
new file mode 100644
index 0000000..49f47d6
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/DataReaderProperties.java
@@ -0,0 +1,84 @@
+package org.apache.orc.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.CompressionCodec;
+
+import javax.annotation.Nullable;
+
+public final class DataReaderProperties {
+
+  private final FileSystem fileSystem;
+  private final Path path;
+  private final CompressionCodec codec;
+  private final boolean zeroCopy;
+
+  private DataReaderProperties(Builder builder) {
+    this.fileSystem = builder.fileSystem;
+    this.path = builder.path;
+    this.codec = builder.codec;
+    this.zeroCopy = builder.zeroCopy;
+  }
+
+  public FileSystem getFileSystem() {
+    return fileSystem;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  @Nullable
+  public CompressionCodec getCodec() {
+    return codec;
+  }
+
+  public boolean getZeroCopy() {
+    return zeroCopy;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder {
+
+    private FileSystem fileSystem;
+    private Path path;
+    private CompressionCodec codec;
+    private boolean zeroCopy;
+
+    private Builder() {
+
+    }
+
+    public Builder withFileSystem(FileSystem fileSystem) {
+      this.fileSystem = fileSystem;
+      return this;
+    }
+
+    public Builder withPath(Path path) {
+      this.path = path;
+      return this;
+    }
+
+    public Builder withCodec(CompressionCodec codec) {
+      this.codec = codec;
+      return this;
+    }
+
+    public Builder withZeroCopy(boolean zeroCopy) {
+      this.zeroCopy = zeroCopy;
+      return this;
+    }
+
+    public DataReaderProperties build() {
+      Preconditions.checkNotNull(fileSystem);
+      Preconditions.checkNotNull(path);
+
+      return new DataReaderProperties(this);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/java/org/apache/orc/impl/DefaultMetadataReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/DefaultMetadataReaderFactory.java b/orc/src/java/org/apache/orc/impl/DefaultMetadataReaderFactory.java
new file mode 100644
index 0000000..fc0d141
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/DefaultMetadataReaderFactory.java
@@ -0,0 +1,14 @@
+package org.apache.orc.impl;
+
+import org.apache.orc.MetadataReaderFactory;
+
+import java.io.IOException;
+
+public final class DefaultMetadataReaderFactory implements MetadataReaderFactory {
+
+  @Override
+  public MetadataReader create(MetadataReaderProperties properties) throws IOException {
+    return new MetadataReaderImpl(properties);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/java/org/apache/orc/impl/MetadataReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/MetadataReader.java b/orc/src/java/org/apache/orc/impl/MetadataReader.java
index 670a81a..500239d 100644
--- a/orc/src/java/org/apache/orc/impl/MetadataReader.java
+++ b/orc/src/java/org/apache/orc/impl/MetadataReader.java
@@ -17,18 +17,17 @@
  */
 package org.apache.orc.impl;
 
+import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.orc.OrcProto;
 import org.apache.orc.StripeInformation;
 
-public interface MetadataReader {
+public interface MetadataReader extends Closeable {
   OrcIndex readRowIndex(StripeInformation stripe,
                                       OrcProto.StripeFooter footer,
       boolean[] included, OrcProto.RowIndex[] indexes, boolean[] sargColumns,
       OrcProto.BloomFilterIndex[] bloomFilterIndices) throws IOException;
 
   OrcProto.StripeFooter readStripeFooter(StripeInformation stripe) throws IOException;
-
-  void close() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/java/org/apache/orc/impl/MetadataReaderImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/MetadataReaderImpl.java b/orc/src/java/org/apache/orc/impl/MetadataReaderImpl.java
index d0ded52..c3ea74f 100644
--- a/orc/src/java/org/apache/orc/impl/MetadataReaderImpl.java
+++ b/orc/src/java/org/apache/orc/impl/MetadataReaderImpl.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -37,17 +38,11 @@ public class MetadataReaderImpl implements MetadataReader {
   private final int bufferSize;
   private final int typeCount;
 
-  public MetadataReaderImpl(FileSystem fileSystem, Path path,
-      CompressionCodec codec, int bufferSize, int typeCount) throws IOException {
-    this(fileSystem.open(path), codec, bufferSize, typeCount);
-  }
-
-  public MetadataReaderImpl(FSDataInputStream file,
-      CompressionCodec codec, int bufferSize, int typeCount) {
-    this.file = file;
-    this.codec = codec;
-    this.bufferSize = bufferSize;
-    this.typeCount = typeCount;
+  MetadataReaderImpl(MetadataReaderProperties properties) throws IOException {
+    this.file = properties.getFileSystem().open(properties.getPath());
+    this.codec = properties.getCodec();
+    this.bufferSize = properties.getBufferSize();
+    this.typeCount = properties.getTypeCount();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/java/org/apache/orc/impl/MetadataReaderProperties.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/MetadataReaderProperties.java b/orc/src/java/org/apache/orc/impl/MetadataReaderProperties.java
new file mode 100644
index 0000000..321931c
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/MetadataReaderProperties.java
@@ -0,0 +1,96 @@
+package org.apache.orc.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.CompressionCodec;
+
+import javax.annotation.Nullable;
+
+public final class MetadataReaderProperties {
+
+  private final FileSystem fileSystem;
+  private final Path path;
+  private final CompressionCodec codec;
+  private final int bufferSize;
+  private final int typeCount;
+
+  private MetadataReaderProperties(Builder builder) {
+    this.fileSystem = builder.fileSystem;
+    this.path = builder.path;
+    this.codec = builder.codec;
+    this.bufferSize = builder.bufferSize;
+    this.typeCount = builder.typeCount;
+  }
+
+  public FileSystem getFileSystem() {
+    return fileSystem;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  @Nullable
+  public CompressionCodec getCodec() {
+    return codec;
+  }
+
+  public int getBufferSize() {
+    return bufferSize;
+  }
+
+  public int getTypeCount() {
+    return typeCount;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder {
+
+    private FileSystem fileSystem;
+    private Path path;
+    private CompressionCodec codec;
+    private int bufferSize;
+    private int typeCount;
+
+    private Builder() {
+
+    }
+
+    public Builder withFileSystem(FileSystem fileSystem) {
+      this.fileSystem = fileSystem;
+      return this;
+    }
+
+    public Builder withPath(Path path) {
+      this.path = path;
+      return this;
+    }
+
+    public Builder withCodec(CompressionCodec codec) {
+      this.codec = codec;
+      return this;
+    }
+
+    public Builder withBufferSize(int bufferSize) {
+      this.bufferSize = bufferSize;
+      return this;
+    }
+
+    public Builder withTypeCount(int typeCount) {
+      this.typeCount = typeCount;
+      return this;
+    }
+
+    public MetadataReaderProperties build() {
+      Preconditions.checkNotNull(fileSystem);
+      Preconditions.checkNotNull(path);
+
+      return new MetadataReaderProperties(this);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/test/org/apache/orc/impl/TestDataReaderProperties.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestDataReaderProperties.java b/orc/src/test/org/apache/orc/impl/TestDataReaderProperties.java
new file mode 100644
index 0000000..9ec08f3
--- /dev/null
+++ b/orc/src/test/org/apache/orc/impl/TestDataReaderProperties.java
@@ -0,0 +1,69 @@
+package org.apache.orc.impl;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.CompressionCodec;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.mock;
+
+public class TestDataReaderProperties {
+
+  private FileSystem mockedFileSystem = mock(FileSystem.class);
+  private Path mockedPath = mock(Path.class);
+  private CompressionCodec mockedCodec = mock(CompressionCodec.class);
+  private boolean mockedZeroCopy = false;
+
+  @Test
+  public void testCompleteBuild() {
+    DataReaderProperties properties = DataReaderProperties.builder()
+      .withFileSystem(mockedFileSystem)
+      .withPath(mockedPath)
+      .withCodec(mockedCodec)
+      .withZeroCopy(mockedZeroCopy)
+      .build();
+    assertEquals(mockedFileSystem, properties.getFileSystem());
+    assertEquals(mockedPath, properties.getPath());
+    assertEquals(mockedCodec, properties.getCodec());
+    assertEquals(mockedZeroCopy, properties.getZeroCopy());
+  }
+
+  @Test
+  public void testMissingNonRequiredArgs() {
+    DataReaderProperties properties = DataReaderProperties.builder()
+      .withFileSystem(mockedFileSystem)
+      .withPath(mockedPath)
+      .build();
+    assertEquals(mockedFileSystem, properties.getFileSystem());
+    assertEquals(mockedPath, properties.getPath());
+    assertNull(properties.getCodec());
+    assertFalse(properties.getZeroCopy());
+  }
+
+  @Test(expected = java.lang.NullPointerException.class)
+  public void testEmptyBuild() {
+    DataReaderProperties.builder().build();
+  }
+
+  @Test(expected = java.lang.NullPointerException.class)
+  public void testMissingPath() {
+    DataReaderProperties.builder()
+      .withFileSystem(mockedFileSystem)
+      .withCodec(mockedCodec)
+      .withZeroCopy(mockedZeroCopy)
+      .build();
+  }
+
+  @Test(expected = java.lang.NullPointerException.class)
+  public void testMissingFileSystem() {
+    DataReaderProperties.builder()
+      .withPath(mockedPath)
+      .withCodec(mockedCodec)
+      .withZeroCopy(mockedZeroCopy)
+      .build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/orc/src/test/org/apache/orc/impl/TestMetadataReaderProperties.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestMetadataReaderProperties.java b/orc/src/test/org/apache/orc/impl/TestMetadataReaderProperties.java
new file mode 100644
index 0000000..12e8eb4
--- /dev/null
+++ b/orc/src/test/org/apache/orc/impl/TestMetadataReaderProperties.java
@@ -0,0 +1,72 @@
+package org.apache.orc.impl;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.CompressionCodec;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.mock;
+
+public class TestMetadataReaderProperties {
+
+  private FileSystem mockedFileSystem = mock(FileSystem.class);
+  private Path mockedPath = mock(Path.class);
+  private CompressionCodec mockedCodec = mock(CompressionCodec.class);
+  private int mockedBufferSize = 0;
+  private int mockedTypeCount = 0;
+
+  @Test
+  public void testCompleteBuild() {
+    MetadataReaderProperties properties = MetadataReaderProperties.builder()
+      .withFileSystem(mockedFileSystem)
+      .withPath(mockedPath)
+      .withCodec(mockedCodec)
+      .withBufferSize(mockedBufferSize)
+      .withTypeCount(mockedTypeCount)
+      .build();
+    assertEquals(mockedFileSystem, properties.getFileSystem());
+    assertEquals(mockedPath, properties.getPath());
+    assertEquals(mockedCodec, properties.getCodec());
+    assertEquals(mockedBufferSize, properties.getBufferSize());
+    assertEquals(mockedTypeCount, properties.getTypeCount());
+  }
+
+  @Test
+  public void testMissingNonRequiredArgs() {
+    MetadataReaderProperties properties = MetadataReaderProperties.builder()
+      .withFileSystem(mockedFileSystem)
+      .withPath(mockedPath)
+      .build();
+    assertEquals(mockedFileSystem, properties.getFileSystem());
+    assertEquals(mockedPath, properties.getPath());
+    assertNull(properties.getCodec());
+    assertEquals(0, properties.getBufferSize());
+    assertEquals(0, properties.getTypeCount());
+  }
+
+  @Test(expected = java.lang.NullPointerException.class)
+  public void testEmptyBuild() {
+    MetadataReaderProperties.builder().build();
+  }
+
+  @Test(expected = java.lang.NullPointerException.class)
+  public void testMissingPath() {
+    MetadataReaderProperties.builder()
+      .withFileSystem(mockedFileSystem)
+      .withCodec(mockedCodec)
+      .withBufferSize(mockedBufferSize)
+      .build();
+  }
+
+  @Test(expected = java.lang.NullPointerException.class)
+  public void testMissingFileSystem() {
+    MetadataReaderProperties.builder()
+      .withPath(mockedPath)
+      .withCodec(mockedCodec)
+      .withBufferSize(mockedBufferSize)
+      .build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/DefaultDataReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/DefaultDataReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/DefaultDataReaderFactory.java
new file mode 100644
index 0000000..de3471c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/DefaultDataReaderFactory.java
@@ -0,0 +1,14 @@
+package org.apache.hadoop.hive.ql.io.orc;
+
+import org.apache.orc.DataReader;
+import org.apache.orc.DataReaderFactory;
+import org.apache.orc.impl.DataReaderProperties;
+
+public final class DefaultDataReaderFactory implements DataReaderFactory {
+
+  @Override
+  public DataReader create(DataReaderProperties properties) {
+    return RecordReaderUtils.createDefaultDataReader(properties);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
index a031a92..822ef14 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
@@ -26,6 +26,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.orc.DataReaderFactory;
+import org.apache.orc.MetadataReaderFactory;
 import org.apache.orc.impl.BufferChunk;
 import org.apache.orc.ColumnStatistics;
 import org.apache.orc.impl.ColumnStatisticsImpl;
@@ -33,9 +35,11 @@ import org.apache.orc.CompressionCodec;
 import org.apache.orc.DataReader;
 import org.apache.orc.FileMetaInfo;
 import org.apache.orc.FileMetadata;
+import org.apache.orc.impl.DataReaderProperties;
+import org.apache.orc.impl.DefaultMetadataReaderFactory;
 import org.apache.orc.impl.InStream;
 import org.apache.orc.impl.MetadataReader;
-import org.apache.orc.impl.MetadataReaderImpl;
+import org.apache.orc.impl.MetadataReaderProperties;
 import org.apache.orc.StripeInformation;
 import org.apache.orc.StripeStatistics;
 import org.slf4j.Logger;
@@ -76,7 +80,8 @@ public class ReaderImpl implements Reader {
   private final List<StripeInformation> stripes;
   protected final int rowIndexStride;
   private final long contentLength, numberOfRows;
-
+  private final MetadataReaderFactory metadataReaderFactory = new DefaultMetadataReaderFactory();
+  private final DataReaderFactory dataReaderFactory = new DefaultDataReaderFactory();
 
   private final ObjectInspector inspector;
   private long deserializedSize = -1;
@@ -667,8 +672,20 @@ public class ReaderImpl implements Reader {
       Arrays.fill(include, true);
       options.include(include);
     }
-    return new RecordReaderImpl(this.getStripes(), fileSystem, path,
-        options, types, codec, bufferSize, rowIndexStride, conf);
+
+    return RecordReaderImpl.builder()
+        .withMetadataReaderFactory(metadataReaderFactory)
+        .withDataReaderFactory(dataReaderFactory)
+        .withStripes(this.getStripes())
+        .withFileSystem(fileSystem)
+        .withPath(path)
+        .withOptions(options)
+        .withTypes(types)
+        .withCodec(codec)
+        .withBufferSize(bufferSize)
+        .withStrideRate(rowIndexStride)
+        .withConf(conf)
+        .build();
   }
 
 
@@ -852,7 +869,13 @@ public class ReaderImpl implements Reader {
 
   @Override
   public MetadataReader metadata() throws IOException {
-    return new MetadataReaderImpl(fileSystem, path, codec, bufferSize, types.size());
+    return metadataReaderFactory.create(MetadataReaderProperties.builder()
+      .withBufferSize(bufferSize)
+      .withCodec(codec)
+      .withFileSystem(fileSystem)
+      .withPath(path)
+      .withTypeCount(types.size())
+      .build());
   }
 
   @Override
@@ -867,7 +890,12 @@ public class ReaderImpl implements Reader {
 
   @Override
   public DataReader createDefaultDataReader(boolean useZeroCopy) {
-    return RecordReaderUtils.createDefaultDataReader(fileSystem, path, useZeroCopy, codec);
+    return dataReaderFactory.create(DataReaderProperties.builder()
+      .withFileSystem(fileSystem)
+      .withPath(path)
+      .withCodec(codec)
+      .withZeroCopy(useZeroCopy)
+      .build());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index 3975409..9cfcc0e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -27,8 +27,11 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.commons.lang3.exception.ExceptionUtils;
+import com.google.common.base.Preconditions;
+import com.google.common.io.Closer;
 import org.apache.orc.BooleanColumnStatistics;
+import org.apache.orc.DataReaderFactory;
+import org.apache.orc.MetadataReaderFactory;
 import org.apache.orc.OrcUtils;
 import org.apache.orc.impl.BufferChunk;
 import org.apache.orc.ColumnStatistics;
@@ -38,11 +41,13 @@ import org.apache.orc.DataReader;
 import org.apache.orc.DateColumnStatistics;
 import org.apache.orc.DecimalColumnStatistics;
 import org.apache.orc.DoubleColumnStatistics;
+import org.apache.orc.impl.DataReaderProperties;
+import org.apache.orc.impl.DefaultMetadataReaderFactory;
 import org.apache.orc.impl.InStream;
 import org.apache.orc.IntegerColumnStatistics;
 import org.apache.orc.impl.MetadataReader;
-import org.apache.orc.impl.MetadataReaderImpl;
 import org.apache.orc.OrcConf;
+import org.apache.orc.impl.MetadataReaderProperties;
 import org.apache.orc.impl.OrcIndex;
 import org.apache.orc.impl.PositionProvider;
 import org.apache.orc.impl.StreamName;
@@ -141,17 +146,99 @@ public class RecordReaderImpl implements RecordReader {
     return result;
   }
 
-  protected RecordReaderImpl(List<StripeInformation> stripes,
-                             FileSystem fileSystem,
-                             Path path,
-                             Reader.Options options,
-                             List<OrcProto.Type> types,
-                             CompressionCodec codec,
-                             int bufferSize,
-                             long strideRate,
-                             Configuration conf
-                             ) throws IOException {
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder {
+    private Reader.Options options;
+    private CompressionCodec codec;
+    private List<OrcProto.Type> types;
+    private List<StripeInformation> stripes;
+    private int bufferSize;
+    private FileSystem fileSystem;
+    private Path path;
+    private Configuration conf;
+    private long strideRate;
+    private MetadataReaderFactory metadataReaderFactory = new DefaultMetadataReaderFactory();
+    private DataReaderFactory dataReaderFactory = new DefaultDataReaderFactory();
+
+    private Builder() {
 
+    }
+
+    public Builder withOptions(Reader.Options options) {
+      this.options = options;
+      return this;
+    }
+
+    public Builder withCodec(CompressionCodec codec) {
+      this.codec = codec;
+      return this;
+    }
+
+    public Builder withTypes(List<OrcProto.Type> types) {
+      this.types = types;
+      return this;
+    }
+
+    public Builder withStripes(List<StripeInformation> stripes) {
+      this.stripes = stripes;
+      return this;
+    }
+
+    public Builder withBufferSize(int bufferSize) {
+      this.bufferSize = bufferSize;
+      return this;
+    }
+
+    public Builder withFileSystem(FileSystem fileSystem) {
+      this.fileSystem = fileSystem;
+      return this;
+    }
+
+    public Builder withPath(Path path) {
+      this.path = path;
+      return this;
+    }
+
+    public Builder withConf(Configuration conf) {
+      this.conf = conf;
+      return this;
+    }
+
+    public Builder withStrideRate(long strideRate) {
+      this.strideRate = strideRate;
+      return this;
+    }
+
+    public Builder withMetadataReaderFactory(MetadataReaderFactory metadataReaderFactory) {
+      this.metadataReaderFactory = metadataReaderFactory;
+      return this;
+    }
+
+    public Builder withDataReaderFactory(DataReaderFactory dataReaderFactory) {
+      this.dataReaderFactory = dataReaderFactory;
+      return this;
+    }
+
+    public RecordReaderImpl build() throws IOException {
+      Preconditions.checkNotNull(metadataReaderFactory);
+      Preconditions.checkNotNull(dataReaderFactory);
+      Preconditions.checkNotNull(options);
+      Preconditions.checkNotNull(types);
+      Preconditions.checkNotNull(stripes);
+      Preconditions.checkNotNull(fileSystem);
+      Preconditions.checkNotNull(path);
+      Preconditions.checkNotNull(conf);
+
+      return new RecordReaderImpl(this);
+    }
+  }
+
+  private RecordReaderImpl(Builder builder) throws IOException {
+    Reader.Options options = builder.options;
+    this.types = builder.types;
     TreeReaderFactory.TreeReaderSchema treeReaderSchema;
     if (options.getSchema() == null) {
       if (LOG.isInfoEnabled()) {
@@ -166,18 +253,23 @@ public class RecordReaderImpl implements RecordReader {
       List<OrcProto.Type> schemaTypes = OrcUtils.getOrcTypes(options.getSchema());
       treeReaderSchema = SchemaEvolution.validateAndCreate(types, schemaTypes);
     }
-    this.path = path;
-    this.codec = codec;
-    this.types = types;
-    this.bufferSize = bufferSize;
+    this.path = builder.path;
+    this.codec = builder.codec;
+    this.bufferSize = builder.bufferSize;
     this.included = options.getInclude();
-    this.conf = conf;
-    this.rowIndexStride = strideRate;
-    this.metadata = new MetadataReaderImpl(fileSystem, path, codec, bufferSize, types.size());
+    this.conf = builder.conf;
+    this.rowIndexStride = builder.strideRate;
+    this.metadata = builder.metadataReaderFactory.create(MetadataReaderProperties.builder()
+        .withFileSystem(builder.fileSystem)
+        .withPath(path)
+        .withCodec(codec)
+        .withBufferSize(bufferSize)
+        .withTypeCount(types.size())
+        .build());
     SearchArgument sarg = options.getSearchArgument();
-    if (sarg != null && strideRate != 0) {
+    if (sarg != null && builder.strideRate != 0) {
       sargApp = new SargApplier(
-          sarg, options.getColumnNames(), strideRate, types, included.length);
+          sarg, options.getColumnNames(), builder.strideRate, types, included.length);
     } else {
       sargApp = null;
     }
@@ -185,7 +277,7 @@ public class RecordReaderImpl implements RecordReader {
     long skippedRows = 0;
     long offset = options.getOffset();
     long maxOffset = options.getMaxOffset();
-    for(StripeInformation stripe: stripes) {
+    for(StripeInformation stripe: builder.stripes) {
       long stripeStart = stripe.getOffset();
       if (offset > stripeStart) {
         skippedRows += stripe.getNumberOfRows();
@@ -200,7 +292,12 @@ public class RecordReaderImpl implements RecordReader {
       zeroCopy = OrcConf.USE_ZEROCOPY.getBoolean(conf);
     }
     // TODO: we could change the ctor to pass this externally
-    this.dataReader = RecordReaderUtils.createDefaultDataReader(fileSystem, path, zeroCopy, codec);
+    this.dataReader = builder.dataReaderFactory.create(DataReaderProperties.builder()
+      .withFileSystem(builder.fileSystem)
+      .withCodec(codec)
+      .withPath(path)
+      .withZeroCopy(zeroCopy)
+      .build());
     this.dataReader.open();
 
     firstRow = skippedRows;
@@ -1119,8 +1216,16 @@ public class RecordReaderImpl implements RecordReader {
 
   @Override
   public void close() throws IOException {
-    clearStreams();
-    dataReader.close();
+    Closer closer = Closer.create();
+    try {
+      closer.register(metadata);
+      closer.register(dataReader);
+      clearStreams();
+    } catch (IOException e) {
+      throw closer.rethrow(e);
+    } finally {
+      closer.close();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java
index 8a73948..177721d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.shims.HadoopShims.ZeroCopyReaderShim;
 import org.apache.orc.impl.BufferChunk;
 import org.apache.orc.CompressionCodec;
 import org.apache.orc.DataReader;
+import org.apache.orc.impl.DataReaderProperties;
 import org.apache.orc.impl.DirectDecompressionCodec;
 import org.apache.orc.OrcProto;
 
@@ -60,12 +61,11 @@ public class RecordReaderUtils {
     private boolean useZeroCopy;
     private CompressionCodec codec;
 
-    public DefaultDataReader(
-        FileSystem fs, Path path, boolean useZeroCopy, CompressionCodec codec) {
-      this.fs = fs;
-      this.path = path;
-      this.useZeroCopy = useZeroCopy;
-      this.codec = codec;
+    private DefaultDataReader(DataReaderProperties properties) {
+      this.fs = properties.getFileSystem();
+      this.path = properties.getPath();
+      this.useZeroCopy = properties.getZeroCopy();
+      this.codec = properties.getCodec();
     }
 
     @Override
@@ -108,9 +108,8 @@ public class RecordReaderUtils {
 
   }
 
-  static DataReader createDefaultDataReader(
-      FileSystem fs, Path path, boolean useZeroCopy, CompressionCodec codec) {
-    return new DefaultDataReader(fs, path, useZeroCopy, codec);
+  static DataReader createDefaultDataReader(DataReaderProperties properties) {
+    return new DefaultDataReader(properties);
   }
 
   public static boolean[] findPresentStreamsByColumn(

http://git-wip-us.apache.org/repos/asf/hive/blob/72b6bcd7/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
index 6803abd..cc7182f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
@@ -21,11 +21,18 @@ package org.apache.hadoop.hive.ql.io.orc;
 import static junit.framework.Assert.assertEquals;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.io.InputStream;
 import java.sql.Timestamp;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -46,9 +53,17 @@ import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.orc.ColumnStatistics;
+import org.apache.orc.CompressionCodec;
+import org.apache.orc.DataReader;
+import org.apache.orc.DataReaderFactory;
+import org.apache.orc.MetadataReaderFactory;
+import org.apache.orc.StripeInformation;
 import org.apache.orc.impl.ColumnStatisticsImpl;
 import org.apache.orc.OrcProto;
 
+import org.apache.orc.impl.DataReaderProperties;
+import org.apache.orc.impl.MetadataReader;
+import org.apache.orc.impl.MetadataReaderProperties;
 import org.junit.Test;
 import org.mockito.MockSettings;
 import org.mockito.Mockito;
@@ -148,16 +163,16 @@ public class TestRecordReaderImpl {
     footer.writeTo(buffer);
     ps.writeTo(buffer);
     buffer.write(ps.getSerializedSize());
-    FileSystem fs = Mockito.mock(FileSystem.class, settings);
+    FileSystem fs = mock(FileSystem.class, settings);
     FSDataInputStream file =
         new FSDataInputStream(new BufferInStream(buffer.getData(),
             buffer.getLength()));
     Path p = new Path("/dir/file.orc");
-    Mockito.when(fs.open(p)).thenReturn(file);
+    when(fs.open(p)).thenReturn(file);
     OrcFile.ReaderOptions options = OrcFile.readerOptions(conf);
     options.filesystem(fs);
     options.maxLength(buffer.getLength());
-    Mockito.when(fs.getFileStatus(p))
+    when(fs.getFileStatus(p))
         .thenReturn(new FileStatus(10, false, 3, 3000, 0, p));
     Reader reader = OrcFile.createReader(p, options);
   }
@@ -165,21 +180,21 @@ public class TestRecordReaderImpl {
   @Test
   public void testCompareToRangeInt() throws Exception {
     assertEquals(Location.BEFORE,
-        RecordReaderImpl.compareToRange(19L, 20L, 40L));
+      RecordReaderImpl.compareToRange(19L, 20L, 40L));
     assertEquals(Location.AFTER,
-        RecordReaderImpl.compareToRange(41L, 20L, 40L));
+      RecordReaderImpl.compareToRange(41L, 20L, 40L));
     assertEquals(Location.MIN,
         RecordReaderImpl.compareToRange(20L, 20L, 40L));
     assertEquals(Location.MIDDLE,
         RecordReaderImpl.compareToRange(21L, 20L, 40L));
     assertEquals(Location.MAX,
-        RecordReaderImpl.compareToRange(40L, 20L, 40L));
+      RecordReaderImpl.compareToRange(40L, 20L, 40L));
     assertEquals(Location.BEFORE,
-        RecordReaderImpl.compareToRange(0L, 1L, 1L));
+      RecordReaderImpl.compareToRange(0L, 1L, 1L));
     assertEquals(Location.MIN,
-        RecordReaderImpl.compareToRange(1L, 1L, 1L));
+      RecordReaderImpl.compareToRange(1L, 1L, 1L));
     assertEquals(Location.AFTER,
-        RecordReaderImpl.compareToRange(2L, 1L, 1L));
+      RecordReaderImpl.compareToRange(2L, 1L, 1L));
   }
 
   @Test
@@ -205,43 +220,43 @@ public class TestRecordReaderImpl {
   @Test
   public void testCompareToCharNeedConvert() throws Exception {
     assertEquals(Location.BEFORE,
-        RecordReaderImpl.compareToRange("apple", "hello", "world"));
+      RecordReaderImpl.compareToRange("apple", "hello", "world"));
     assertEquals(Location.AFTER,
-        RecordReaderImpl.compareToRange("zombie", "hello", "world"));
+      RecordReaderImpl.compareToRange("zombie", "hello", "world"));
     assertEquals(Location.MIN,
         RecordReaderImpl.compareToRange("hello", "hello", "world"));
     assertEquals(Location.MIDDLE,
         RecordReaderImpl.compareToRange("pilot", "hello", "world"));
     assertEquals(Location.MAX,
-        RecordReaderImpl.compareToRange("world", "hello", "world"));
+      RecordReaderImpl.compareToRange("world", "hello", "world"));
     assertEquals(Location.BEFORE,
-        RecordReaderImpl.compareToRange("apple", "hello", "hello"));
+      RecordReaderImpl.compareToRange("apple", "hello", "hello"));
     assertEquals(Location.MIN,
-        RecordReaderImpl.compareToRange("hello", "hello", "hello"));
+      RecordReaderImpl.compareToRange("hello", "hello", "hello"));
     assertEquals(Location.AFTER,
-        RecordReaderImpl.compareToRange("zombie", "hello", "hello"));
+      RecordReaderImpl.compareToRange("zombie", "hello", "hello"));
   }
 
   @Test
   public void testGetMin() throws Exception {
     assertEquals(10L, RecordReaderImpl.getMin(
-        ColumnStatisticsImpl.deserialize(createIntStats(10L, 100L))));
+      ColumnStatisticsImpl.deserialize(createIntStats(10L, 100L))));
     assertEquals(10.0d, RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
-        OrcProto.ColumnStatistics.newBuilder()
-            .setDoubleStatistics(OrcProto.DoubleStatistics.newBuilder()
-                .setMinimum(10.0d).setMaximum(100.0d).build()).build())));
+      OrcProto.ColumnStatistics.newBuilder()
+        .setDoubleStatistics(OrcProto.DoubleStatistics.newBuilder()
+          .setMinimum(10.0d).setMaximum(100.0d).build()).build())));
     assertEquals(null, RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
-        OrcProto.ColumnStatistics.newBuilder()
-            .setStringStatistics(OrcProto.StringStatistics.newBuilder().build())
-            .build())));
+      OrcProto.ColumnStatistics.newBuilder()
+        .setStringStatistics(OrcProto.StringStatistics.newBuilder().build())
+        .build())));
     assertEquals("a", RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
-        OrcProto.ColumnStatistics.newBuilder()
-            .setStringStatistics(OrcProto.StringStatistics.newBuilder()
-                .setMinimum("a").setMaximum("b").build()).build())));
+      OrcProto.ColumnStatistics.newBuilder()
+        .setStringStatistics(OrcProto.StringStatistics.newBuilder()
+          .setMinimum("a").setMaximum("b").build()).build())));
     assertEquals("hello", RecordReaderImpl.getMin(ColumnStatisticsImpl
-        .deserialize(createStringStats("hello", "world"))));
+      .deserialize(createStringStats("hello", "world"))));
     assertEquals(HiveDecimal.create("111.1"), RecordReaderImpl.getMin(ColumnStatisticsImpl
-        .deserialize(createDecimalStats("111.1", "112.1"))));
+      .deserialize(createDecimalStats("111.1", "112.1"))));
   }
 
   private static OrcProto.ColumnStatistics createIntStats(Long min,
@@ -262,7 +277,7 @@ public class TestRecordReaderImpl {
     OrcProto.BucketStatistics.Builder boolStats = OrcProto.BucketStatistics.newBuilder();
     boolStats.addCount(trueCount);
     return OrcProto.ColumnStatistics.newBuilder().setNumberOfValues(n).setBucketStatistics(
-        boolStats.build()).build();
+      boolStats.build()).build();
   }
 
   private static OrcProto.ColumnStatistics createIntStats(int min, int max) {
@@ -341,9 +356,9 @@ public class TestRecordReaderImpl {
             .setStringStatistics(OrcProto.StringStatistics.newBuilder()
                 .setMinimum("a").setMaximum("b").build()).build())));
     assertEquals("world", RecordReaderImpl.getMax(ColumnStatisticsImpl
-        .deserialize(createStringStats("hello", "world"))));
+      .deserialize(createStringStats("hello", "world"))));
     assertEquals(HiveDecimal.create("112.1"), RecordReaderImpl.getMax(ColumnStatisticsImpl
-        .deserialize(createDecimalStats("111.1", "112.1"))));
+      .deserialize(createDecimalStats("111.1", "112.1"))));
   }
 
   @Test
@@ -365,15 +380,15 @@ public class TestRecordReaderImpl {
     pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", false, null);
     assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
+      RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
     assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
+      RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
   }
 
   @Test
   public void testPredEvalWithIntStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
+      PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
 
@@ -402,7 +417,7 @@ public class TestRecordReaderImpl {
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
         PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
     assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
+      RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
   }
 
   @Test
@@ -620,7 +635,7 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10 * 24L * 60L * 60L * 1000L,
-            100 * 24L * 60L * 60L * 1000L), pred, null));
+          100 * 24L * 60L * 60L * 1000L), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
         PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
@@ -738,9 +753,9 @@ public class TestRecordReaderImpl {
     assertEquals(TruthValue.NO_NULL,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 5L), pred, null));
     assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(30L, 40L), pred, null));
+      RecordReaderImpl.evaluatePredicateProto(createIntStats(30L, 40L), pred, null));
     assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(5L, 15L), pred, null));
+      RecordReaderImpl.evaluatePredicateProto(createIntStats(5L, 15L), pred, null));
     assertEquals(TruthValue.YES_NO_NULL,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 25L), pred, null));
     assertEquals(TruthValue.YES_NO_NULL,
@@ -876,10 +891,10 @@ public class TestRecordReaderImpl {
     assertEquals(TruthValue.YES_NO_NULL, // before & min
         RecordReaderImpl.evaluatePredicateProto(createStringStats("f", "g", true), pred, null));
     assertEquals(TruthValue.YES_NO_NULL, // before & middle
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("e", "g", true), pred, null));
+      RecordReaderImpl.evaluatePredicateProto(createStringStats("e", "g", true), pred, null));
 
     assertEquals(TruthValue.YES_NULL, // min & after
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "e", true), pred, null));
+      RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "e", true), pred, null));
     assertEquals(TruthValue.YES_NULL, // min & max
         RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "f", true), pred, null));
     assertEquals(TruthValue.YES_NO_NULL, // min & middle
@@ -1623,4 +1638,56 @@ public class TestRecordReaderImpl {
     bf.addString(HiveDecimal.create(15).toString());
     assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
   }
+
+  @Test
+  public void testClose() throws Exception {
+    DataReader mockedDataReader = mock(DataReader.class);
+    MetadataReader mockedMetadataReader = mock(MetadataReader.class);
+
+    closeMockedRecordReader(mockedDataReader, mockedMetadataReader);
+
+    verify(mockedDataReader, atLeastOnce()).close();
+    verify(mockedMetadataReader, atLeastOnce()).close();
+  }
+
+  @Test
+  public void testCloseWithException() throws Exception {
+    DataReader mockedDataReader = mock(DataReader.class);
+    MetadataReader mockedMetadataReader = mock(MetadataReader.class);
+    doThrow(IOException.class).when(mockedDataReader).close();
+
+    try {
+      closeMockedRecordReader(mockedDataReader, mockedMetadataReader);
+      fail("Exception should have been thrown when Record Reader was closed");
+    } catch (IOException expected) {
+
+    }
+
+    verify(mockedMetadataReader, atLeastOnce()).close();
+    verify(mockedDataReader, atLeastOnce()).close();
+  }
+
+  private void closeMockedRecordReader(DataReader mockedDataReader,
+                                       MetadataReader mockedMetadataReader) throws IOException {
+    DataReaderFactory mockedDataReaderFactory = mock(DataReaderFactory.class);
+    MetadataReaderFactory mockedMetadataReaderFactory = mock(MetadataReaderFactory.class);
+    when(mockedDataReaderFactory.create(any(DataReaderProperties.class))).thenReturn(mockedDataReader);
+    when(mockedMetadataReaderFactory.create(any(MetadataReaderProperties.class))).thenReturn(mockedMetadataReader);
+
+    RecordReader recordReader = RecordReaderImpl.builder()
+      .withBufferSize(0)
+      .withCodec(mock(CompressionCodec.class))
+      .withConf(mock(Configuration.class))
+      .withFileSystem(mock(FileSystem.class))
+      .withOptions(mock(Reader.Options.class))
+      .withPath(mock(Path.class))
+      .withStrideRate(0)
+      .withStripes(Collections.singletonList(mock(StripeInformation.class)))
+      .withTypes(Collections.singletonList(OrcProto.Type.getDefaultInstance()))
+      .withDataReaderFactory(mockedDataReaderFactory)
+      .withMetadataReaderFactory(mockedMetadataReaderFactory)
+      .build();
+
+    recordReader.close();
+  }
 }


[20/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java
new file mode 100644
index 0000000..546528c
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java
@@ -0,0 +1,1103 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * 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.3)")
+public class SQLPrimaryKey implements org.apache.thrift.TBase<SQLPrimaryKey, SQLPrimaryKey._Fields>, java.io.Serializable, Cloneable, Comparable<SQLPrimaryKey> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLPrimaryKey");
+
+  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField KEY_SEQ_FIELD_DESC = new org.apache.thrift.protocol.TField("key_seq", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField PK_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("pk_name", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)6);
+  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SQLPrimaryKeyStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SQLPrimaryKeyTupleSchemeFactory());
+  }
+
+  private String table_db; // required
+  private String table_name; // required
+  private String column_name; // required
+  private int key_seq; // required
+  private String pk_name; // required
+  private boolean enable_cstr; // required
+  private boolean validate_cstr; // required
+  private boolean rely_cstr; // 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 {
+    TABLE_DB((short)1, "table_db"),
+    TABLE_NAME((short)2, "table_name"),
+    COLUMN_NAME((short)3, "column_name"),
+    KEY_SEQ((short)4, "key_seq"),
+    PK_NAME((short)5, "pk_name"),
+    ENABLE_CSTR((short)6, "enable_cstr"),
+    VALIDATE_CSTR((short)7, "validate_cstr"),
+    RELY_CSTR((short)8, "rely_cstr");
+
+    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: // TABLE_DB
+          return TABLE_DB;
+        case 2: // TABLE_NAME
+          return TABLE_NAME;
+        case 3: // COLUMN_NAME
+          return COLUMN_NAME;
+        case 4: // KEY_SEQ
+          return KEY_SEQ;
+        case 5: // PK_NAME
+          return PK_NAME;
+        case 6: // ENABLE_CSTR
+          return ENABLE_CSTR;
+        case 7: // VALIDATE_CSTR
+          return VALIDATE_CSTR;
+        case 8: // RELY_CSTR
+          return RELY_CSTR;
+        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 __KEY_SEQ_ISSET_ID = 0;
+  private static final int __ENABLE_CSTR_ISSET_ID = 1;
+  private static final int __VALIDATE_CSTR_ISSET_ID = 2;
+  private static final int __RELY_CSTR_ISSET_ID = 3;
+  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.TABLE_DB, new org.apache.thrift.meta_data.FieldMetaData("table_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("column_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.KEY_SEQ, new org.apache.thrift.meta_data.FieldMetaData("key_seq", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.PK_NAME, new org.apache.thrift.meta_data.FieldMetaData("pk_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ENABLE_CSTR, new org.apache.thrift.meta_data.FieldMetaData("enable_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.VALIDATE_CSTR, new org.apache.thrift.meta_data.FieldMetaData("validate_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.RELY_CSTR, new org.apache.thrift.meta_data.FieldMetaData("rely_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SQLPrimaryKey.class, metaDataMap);
+  }
+
+  public SQLPrimaryKey() {
+  }
+
+  public SQLPrimaryKey(
+    String table_db,
+    String table_name,
+    String column_name,
+    int key_seq,
+    String pk_name,
+    boolean enable_cstr,
+    boolean validate_cstr,
+    boolean rely_cstr)
+  {
+    this();
+    this.table_db = table_db;
+    this.table_name = table_name;
+    this.column_name = column_name;
+    this.key_seq = key_seq;
+    setKey_seqIsSet(true);
+    this.pk_name = pk_name;
+    this.enable_cstr = enable_cstr;
+    setEnable_cstrIsSet(true);
+    this.validate_cstr = validate_cstr;
+    setValidate_cstrIsSet(true);
+    this.rely_cstr = rely_cstr;
+    setRely_cstrIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SQLPrimaryKey(SQLPrimaryKey other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetTable_db()) {
+      this.table_db = other.table_db;
+    }
+    if (other.isSetTable_name()) {
+      this.table_name = other.table_name;
+    }
+    if (other.isSetColumn_name()) {
+      this.column_name = other.column_name;
+    }
+    this.key_seq = other.key_seq;
+    if (other.isSetPk_name()) {
+      this.pk_name = other.pk_name;
+    }
+    this.enable_cstr = other.enable_cstr;
+    this.validate_cstr = other.validate_cstr;
+    this.rely_cstr = other.rely_cstr;
+  }
+
+  public SQLPrimaryKey deepCopy() {
+    return new SQLPrimaryKey(this);
+  }
+
+  @Override
+  public void clear() {
+    this.table_db = null;
+    this.table_name = null;
+    this.column_name = null;
+    setKey_seqIsSet(false);
+    this.key_seq = 0;
+    this.pk_name = null;
+    setEnable_cstrIsSet(false);
+    this.enable_cstr = false;
+    setValidate_cstrIsSet(false);
+    this.validate_cstr = false;
+    setRely_cstrIsSet(false);
+    this.rely_cstr = false;
+  }
+
+  public String getTable_db() {
+    return this.table_db;
+  }
+
+  public void setTable_db(String table_db) {
+    this.table_db = table_db;
+  }
+
+  public void unsetTable_db() {
+    this.table_db = null;
+  }
+
+  /** Returns true if field table_db is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable_db() {
+    return this.table_db != null;
+  }
+
+  public void setTable_dbIsSet(boolean value) {
+    if (!value) {
+      this.table_db = null;
+    }
+  }
+
+  public String getTable_name() {
+    return this.table_name;
+  }
+
+  public void setTable_name(String table_name) {
+    this.table_name = table_name;
+  }
+
+  public void unsetTable_name() {
+    this.table_name = null;
+  }
+
+  /** Returns true if field table_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable_name() {
+    return this.table_name != null;
+  }
+
+  public void setTable_nameIsSet(boolean value) {
+    if (!value) {
+      this.table_name = null;
+    }
+  }
+
+  public String getColumn_name() {
+    return this.column_name;
+  }
+
+  public void setColumn_name(String column_name) {
+    this.column_name = column_name;
+  }
+
+  public void unsetColumn_name() {
+    this.column_name = null;
+  }
+
+  /** Returns true if field column_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetColumn_name() {
+    return this.column_name != null;
+  }
+
+  public void setColumn_nameIsSet(boolean value) {
+    if (!value) {
+      this.column_name = null;
+    }
+  }
+
+  public int getKey_seq() {
+    return this.key_seq;
+  }
+
+  public void setKey_seq(int key_seq) {
+    this.key_seq = key_seq;
+    setKey_seqIsSet(true);
+  }
+
+  public void unsetKey_seq() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __KEY_SEQ_ISSET_ID);
+  }
+
+  /** Returns true if field key_seq is set (has been assigned a value) and false otherwise */
+  public boolean isSetKey_seq() {
+    return EncodingUtils.testBit(__isset_bitfield, __KEY_SEQ_ISSET_ID);
+  }
+
+  public void setKey_seqIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __KEY_SEQ_ISSET_ID, value);
+  }
+
+  public String getPk_name() {
+    return this.pk_name;
+  }
+
+  public void setPk_name(String pk_name) {
+    this.pk_name = pk_name;
+  }
+
+  public void unsetPk_name() {
+    this.pk_name = null;
+  }
+
+  /** Returns true if field pk_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetPk_name() {
+    return this.pk_name != null;
+  }
+
+  public void setPk_nameIsSet(boolean value) {
+    if (!value) {
+      this.pk_name = null;
+    }
+  }
+
+  public boolean isEnable_cstr() {
+    return this.enable_cstr;
+  }
+
+  public void setEnable_cstr(boolean enable_cstr) {
+    this.enable_cstr = enable_cstr;
+    setEnable_cstrIsSet(true);
+  }
+
+  public void unsetEnable_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field enable_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetEnable_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID);
+  }
+
+  public void setEnable_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID, value);
+  }
+
+  public boolean isValidate_cstr() {
+    return this.validate_cstr;
+  }
+
+  public void setValidate_cstr(boolean validate_cstr) {
+    this.validate_cstr = validate_cstr;
+    setValidate_cstrIsSet(true);
+  }
+
+  public void unsetValidate_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field validate_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetValidate_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID);
+  }
+
+  public void setValidate_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID, value);
+  }
+
+  public boolean isRely_cstr() {
+    return this.rely_cstr;
+  }
+
+  public void setRely_cstr(boolean rely_cstr) {
+    this.rely_cstr = rely_cstr;
+    setRely_cstrIsSet(true);
+  }
+
+  public void unsetRely_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RELY_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field rely_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetRely_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __RELY_CSTR_ISSET_ID);
+  }
+
+  public void setRely_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RELY_CSTR_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TABLE_DB:
+      if (value == null) {
+        unsetTable_db();
+      } else {
+        setTable_db((String)value);
+      }
+      break;
+
+    case TABLE_NAME:
+      if (value == null) {
+        unsetTable_name();
+      } else {
+        setTable_name((String)value);
+      }
+      break;
+
+    case COLUMN_NAME:
+      if (value == null) {
+        unsetColumn_name();
+      } else {
+        setColumn_name((String)value);
+      }
+      break;
+
+    case KEY_SEQ:
+      if (value == null) {
+        unsetKey_seq();
+      } else {
+        setKey_seq((Integer)value);
+      }
+      break;
+
+    case PK_NAME:
+      if (value == null) {
+        unsetPk_name();
+      } else {
+        setPk_name((String)value);
+      }
+      break;
+
+    case ENABLE_CSTR:
+      if (value == null) {
+        unsetEnable_cstr();
+      } else {
+        setEnable_cstr((Boolean)value);
+      }
+      break;
+
+    case VALIDATE_CSTR:
+      if (value == null) {
+        unsetValidate_cstr();
+      } else {
+        setValidate_cstr((Boolean)value);
+      }
+      break;
+
+    case RELY_CSTR:
+      if (value == null) {
+        unsetRely_cstr();
+      } else {
+        setRely_cstr((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TABLE_DB:
+      return getTable_db();
+
+    case TABLE_NAME:
+      return getTable_name();
+
+    case COLUMN_NAME:
+      return getColumn_name();
+
+    case KEY_SEQ:
+      return getKey_seq();
+
+    case PK_NAME:
+      return getPk_name();
+
+    case ENABLE_CSTR:
+      return isEnable_cstr();
+
+    case VALIDATE_CSTR:
+      return isValidate_cstr();
+
+    case RELY_CSTR:
+      return isRely_cstr();
+
+    }
+    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 TABLE_DB:
+      return isSetTable_db();
+    case TABLE_NAME:
+      return isSetTable_name();
+    case COLUMN_NAME:
+      return isSetColumn_name();
+    case KEY_SEQ:
+      return isSetKey_seq();
+    case PK_NAME:
+      return isSetPk_name();
+    case ENABLE_CSTR:
+      return isSetEnable_cstr();
+    case VALIDATE_CSTR:
+      return isSetValidate_cstr();
+    case RELY_CSTR:
+      return isSetRely_cstr();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SQLPrimaryKey)
+      return this.equals((SQLPrimaryKey)that);
+    return false;
+  }
+
+  public boolean equals(SQLPrimaryKey that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_table_db = true && this.isSetTable_db();
+    boolean that_present_table_db = true && that.isSetTable_db();
+    if (this_present_table_db || that_present_table_db) {
+      if (!(this_present_table_db && that_present_table_db))
+        return false;
+      if (!this.table_db.equals(that.table_db))
+        return false;
+    }
+
+    boolean this_present_table_name = true && this.isSetTable_name();
+    boolean that_present_table_name = true && that.isSetTable_name();
+    if (this_present_table_name || that_present_table_name) {
+      if (!(this_present_table_name && that_present_table_name))
+        return false;
+      if (!this.table_name.equals(that.table_name))
+        return false;
+    }
+
+    boolean this_present_column_name = true && this.isSetColumn_name();
+    boolean that_present_column_name = true && that.isSetColumn_name();
+    if (this_present_column_name || that_present_column_name) {
+      if (!(this_present_column_name && that_present_column_name))
+        return false;
+      if (!this.column_name.equals(that.column_name))
+        return false;
+    }
+
+    boolean this_present_key_seq = true;
+    boolean that_present_key_seq = true;
+    if (this_present_key_seq || that_present_key_seq) {
+      if (!(this_present_key_seq && that_present_key_seq))
+        return false;
+      if (this.key_seq != that.key_seq)
+        return false;
+    }
+
+    boolean this_present_pk_name = true && this.isSetPk_name();
+    boolean that_present_pk_name = true && that.isSetPk_name();
+    if (this_present_pk_name || that_present_pk_name) {
+      if (!(this_present_pk_name && that_present_pk_name))
+        return false;
+      if (!this.pk_name.equals(that.pk_name))
+        return false;
+    }
+
+    boolean this_present_enable_cstr = true;
+    boolean that_present_enable_cstr = true;
+    if (this_present_enable_cstr || that_present_enable_cstr) {
+      if (!(this_present_enable_cstr && that_present_enable_cstr))
+        return false;
+      if (this.enable_cstr != that.enable_cstr)
+        return false;
+    }
+
+    boolean this_present_validate_cstr = true;
+    boolean that_present_validate_cstr = true;
+    if (this_present_validate_cstr || that_present_validate_cstr) {
+      if (!(this_present_validate_cstr && that_present_validate_cstr))
+        return false;
+      if (this.validate_cstr != that.validate_cstr)
+        return false;
+    }
+
+    boolean this_present_rely_cstr = true;
+    boolean that_present_rely_cstr = true;
+    if (this_present_rely_cstr || that_present_rely_cstr) {
+      if (!(this_present_rely_cstr && that_present_rely_cstr))
+        return false;
+      if (this.rely_cstr != that.rely_cstr)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_table_db = true && (isSetTable_db());
+    list.add(present_table_db);
+    if (present_table_db)
+      list.add(table_db);
+
+    boolean present_table_name = true && (isSetTable_name());
+    list.add(present_table_name);
+    if (present_table_name)
+      list.add(table_name);
+
+    boolean present_column_name = true && (isSetColumn_name());
+    list.add(present_column_name);
+    if (present_column_name)
+      list.add(column_name);
+
+    boolean present_key_seq = true;
+    list.add(present_key_seq);
+    if (present_key_seq)
+      list.add(key_seq);
+
+    boolean present_pk_name = true && (isSetPk_name());
+    list.add(present_pk_name);
+    if (present_pk_name)
+      list.add(pk_name);
+
+    boolean present_enable_cstr = true;
+    list.add(present_enable_cstr);
+    if (present_enable_cstr)
+      list.add(enable_cstr);
+
+    boolean present_validate_cstr = true;
+    list.add(present_validate_cstr);
+    if (present_validate_cstr)
+      list.add(validate_cstr);
+
+    boolean present_rely_cstr = true;
+    list.add(present_rely_cstr);
+    if (present_rely_cstr)
+      list.add(rely_cstr);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SQLPrimaryKey other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetTable_db()).compareTo(other.isSetTable_db());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable_db()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table_db, other.table_db);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTable_name()).compareTo(other.isSetTable_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table_name, other.table_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetColumn_name()).compareTo(other.isSetColumn_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetColumn_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column_name, other.column_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetKey_seq()).compareTo(other.isSetKey_seq());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetKey_seq()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.key_seq, other.key_seq);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPk_name()).compareTo(other.isSetPk_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPk_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pk_name, other.pk_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetEnable_cstr()).compareTo(other.isSetEnable_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEnable_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enable_cstr, other.enable_cstr);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetValidate_cstr()).compareTo(other.isSetValidate_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetValidate_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.validate_cstr, other.validate_cstr);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetRely_cstr()).compareTo(other.isSetRely_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetRely_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rely_cstr, other.rely_cstr);
+      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("SQLPrimaryKey(");
+    boolean first = true;
+
+    sb.append("table_db:");
+    if (this.table_db == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.table_db);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("table_name:");
+    if (this.table_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.table_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("column_name:");
+    if (this.column_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.column_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("key_seq:");
+    sb.append(this.key_seq);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("pk_name:");
+    if (this.pk_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pk_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("enable_cstr:");
+    sb.append(this.enable_cstr);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("validate_cstr:");
+    sb.append(this.validate_cstr);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("rely_cstr:");
+    sb.append(this.rely_cstr);
+    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 {
+      // 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 SQLPrimaryKeyStandardSchemeFactory implements SchemeFactory {
+    public SQLPrimaryKeyStandardScheme getScheme() {
+      return new SQLPrimaryKeyStandardScheme();
+    }
+  }
+
+  private static class SQLPrimaryKeyStandardScheme extends StandardScheme<SQLPrimaryKey> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SQLPrimaryKey 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: // TABLE_DB
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table_db = iprot.readString();
+              struct.setTable_dbIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TABLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table_name = iprot.readString();
+              struct.setTable_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // COLUMN_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.column_name = iprot.readString();
+              struct.setColumn_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // KEY_SEQ
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.key_seq = iprot.readI32();
+              struct.setKey_seqIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // PK_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.pk_name = iprot.readString();
+              struct.setPk_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // ENABLE_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.enable_cstr = iprot.readBool();
+              struct.setEnable_cstrIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // VALIDATE_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.validate_cstr = iprot.readBool();
+              struct.setValidate_cstrIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // RELY_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.rely_cstr = iprot.readBool();
+              struct.setRely_cstrIsSet(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, SQLPrimaryKey struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.table_db != null) {
+        oprot.writeFieldBegin(TABLE_DB_FIELD_DESC);
+        oprot.writeString(struct.table_db);
+        oprot.writeFieldEnd();
+      }
+      if (struct.table_name != null) {
+        oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+        oprot.writeString(struct.table_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.column_name != null) {
+        oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC);
+        oprot.writeString(struct.column_name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(KEY_SEQ_FIELD_DESC);
+      oprot.writeI32(struct.key_seq);
+      oprot.writeFieldEnd();
+      if (struct.pk_name != null) {
+        oprot.writeFieldBegin(PK_NAME_FIELD_DESC);
+        oprot.writeString(struct.pk_name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(ENABLE_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.enable_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(VALIDATE_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.validate_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(RELY_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.rely_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SQLPrimaryKeyTupleSchemeFactory implements SchemeFactory {
+    public SQLPrimaryKeyTupleScheme getScheme() {
+      return new SQLPrimaryKeyTupleScheme();
+    }
+  }
+
+  private static class SQLPrimaryKeyTupleScheme extends TupleScheme<SQLPrimaryKey> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SQLPrimaryKey struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetTable_db()) {
+        optionals.set(0);
+      }
+      if (struct.isSetTable_name()) {
+        optionals.set(1);
+      }
+      if (struct.isSetColumn_name()) {
+        optionals.set(2);
+      }
+      if (struct.isSetKey_seq()) {
+        optionals.set(3);
+      }
+      if (struct.isSetPk_name()) {
+        optionals.set(4);
+      }
+      if (struct.isSetEnable_cstr()) {
+        optionals.set(5);
+      }
+      if (struct.isSetValidate_cstr()) {
+        optionals.set(6);
+      }
+      if (struct.isSetRely_cstr()) {
+        optionals.set(7);
+      }
+      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetTable_db()) {
+        oprot.writeString(struct.table_db);
+      }
+      if (struct.isSetTable_name()) {
+        oprot.writeString(struct.table_name);
+      }
+      if (struct.isSetColumn_name()) {
+        oprot.writeString(struct.column_name);
+      }
+      if (struct.isSetKey_seq()) {
+        oprot.writeI32(struct.key_seq);
+      }
+      if (struct.isSetPk_name()) {
+        oprot.writeString(struct.pk_name);
+      }
+      if (struct.isSetEnable_cstr()) {
+        oprot.writeBool(struct.enable_cstr);
+      }
+      if (struct.isSetValidate_cstr()) {
+        oprot.writeBool(struct.validate_cstr);
+      }
+      if (struct.isSetRely_cstr()) {
+        oprot.writeBool(struct.rely_cstr);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SQLPrimaryKey struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(8);
+      if (incoming.get(0)) {
+        struct.table_db = iprot.readString();
+        struct.setTable_dbIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.table_name = iprot.readString();
+        struct.setTable_nameIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.column_name = iprot.readString();
+        struct.setColumn_nameIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.key_seq = iprot.readI32();
+        struct.setKey_seqIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.pk_name = iprot.readString();
+        struct.setPk_nameIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.enable_cstr = iprot.readBool();
+        struct.setEnable_cstrIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.validate_cstr = iprot.readBool();
+        struct.setValidate_cstrIsSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.rely_cstr = iprot.readBool();
+        struct.setRely_cstrIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 afa832c..4df2199 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
@@ -354,14 +354,14 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list484 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list484.size);
-                ShowCompactResponseElement _elem485;
-                for (int _i486 = 0; _i486 < _list484.size; ++_i486)
+                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list500.size);
+                ShowCompactResponseElement _elem501;
+                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
                 {
-                  _elem485 = new ShowCompactResponseElement();
-                  _elem485.read(iprot);
-                  struct.compacts.add(_elem485);
+                  _elem501 = new ShowCompactResponseElement();
+                  _elem501.read(iprot);
+                  struct.compacts.add(_elem501);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter487 : struct.compacts)
+          for (ShowCompactResponseElement _iter503 : struct.compacts)
           {
-            _iter487.write(oprot);
+            _iter503.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter488 : struct.compacts)
+        for (ShowCompactResponseElement _iter504 : struct.compacts)
         {
-          _iter488.write(oprot);
+          _iter504.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list489 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list489.size);
-        ShowCompactResponseElement _elem490;
-        for (int _i491 = 0; _i491 < _list489.size; ++_i491)
+        org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list505.size);
+        ShowCompactResponseElement _elem506;
+        for (int _i507 = 0; _i507 < _list505.size; ++_i507)
         {
-          _elem490 = new ShowCompactResponseElement();
-          _elem490.read(iprot);
-          struct.compacts.add(_elem490);
+          _elem506 = new ShowCompactResponseElement();
+          _elem506.read(iprot);
+          struct.compacts.add(_elem506);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 b9b7f3c..11944db 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
@@ -350,14 +350,14 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list460 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list460.size);
-                ShowLocksResponseElement _elem461;
-                for (int _i462 = 0; _i462 < _list460.size; ++_i462)
+                org.apache.thrift.protocol.TList _list476 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list476.size);
+                ShowLocksResponseElement _elem477;
+                for (int _i478 = 0; _i478 < _list476.size; ++_i478)
                 {
-                  _elem461 = new ShowLocksResponseElement();
-                  _elem461.read(iprot);
-                  struct.locks.add(_elem461);
+                  _elem477 = new ShowLocksResponseElement();
+                  _elem477.read(iprot);
+                  struct.locks.add(_elem477);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter463 : struct.locks)
+          for (ShowLocksResponseElement _iter479 : struct.locks)
           {
-            _iter463.write(oprot);
+            _iter479.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter464 : struct.locks)
+          for (ShowLocksResponseElement _iter480 : struct.locks)
           {
-            _iter464.write(oprot);
+            _iter480.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list465.size);
-          ShowLocksResponseElement _elem466;
-          for (int _i467 = 0; _i467 < _list465.size; ++_i467)
+          org.apache.thrift.protocol.TList _list481 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list481.size);
+          ShowLocksResponseElement _elem482;
+          for (int _i483 = 0; _i483 < _list481.size; ++_i483)
           {
-            _elem466 = new ShowLocksResponseElement();
-            _elem466.read(iprot);
-            struct.locks.add(_elem466);
+            _elem482 = new ShowLocksResponseElement();
+            _elem482.read(iprot);
+            struct.locks.add(_elem482);
           }
         }
         struct.setLocksIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 d0daee5..feed244 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
@@ -537,13 +537,13 @@ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequ
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list356 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list356.size);
-                String _elem357;
-                for (int _i358 = 0; _i358 < _list356.size; ++_i358)
+                org.apache.thrift.protocol.TList _list372 = iprot.readListBegin();
+                struct.colNames = new ArrayList<String>(_list372.size);
+                String _elem373;
+                for (int _i374 = 0; _i374 < _list372.size; ++_i374)
                 {
-                  _elem357 = iprot.readString();
-                  struct.colNames.add(_elem357);
+                  _elem373 = iprot.readString();
+                  struct.colNames.add(_elem373);
                 }
                 iprot.readListEnd();
               }
@@ -579,9 +579,9 @@ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequ
         oprot.writeFieldBegin(COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.colNames.size()));
-          for (String _iter359 : struct.colNames)
+          for (String _iter375 : struct.colNames)
           {
-            oprot.writeString(_iter359);
+            oprot.writeString(_iter375);
           }
           oprot.writeListEnd();
         }
@@ -608,9 +608,9 @@ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequ
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter360 : struct.colNames)
+        for (String _iter376 : struct.colNames)
         {
-          oprot.writeString(_iter360);
+          oprot.writeString(_iter376);
         }
       }
     }
@@ -623,13 +623,13 @@ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequ
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list361 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list361.size);
-        String _elem362;
-        for (int _i363 = 0; _i363 < _list361.size; ++_i363)
+        org.apache.thrift.protocol.TList _list377 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.colNames = new ArrayList<String>(_list377.size);
+        String _elem378;
+        for (int _i379 = 0; _i379 < _list377.size; ++_i379)
         {
-          _elem362 = iprot.readString();
-          struct.colNames.add(_elem362);
+          _elem378 = iprot.readString();
+          struct.colNames.add(_elem378);
         }
       }
       struct.setColNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 78d4250..97cd816 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
@@ -354,14 +354,14 @@ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResul
           case 1: // TABLE_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list330 = iprot.readListBegin();
-                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list330.size);
-                ColumnStatisticsObj _elem331;
-                for (int _i332 = 0; _i332 < _list330.size; ++_i332)
+                org.apache.thrift.protocol.TList _list346 = iprot.readListBegin();
+                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list346.size);
+                ColumnStatisticsObj _elem347;
+                for (int _i348 = 0; _i348 < _list346.size; ++_i348)
                 {
-                  _elem331 = new ColumnStatisticsObj();
-                  _elem331.read(iprot);
-                  struct.tableStats.add(_elem331);
+                  _elem347 = new ColumnStatisticsObj();
+                  _elem347.read(iprot);
+                  struct.tableStats.add(_elem347);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResul
         oprot.writeFieldBegin(TABLE_STATS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tableStats.size()));
-          for (ColumnStatisticsObj _iter333 : struct.tableStats)
+          for (ColumnStatisticsObj _iter349 : struct.tableStats)
           {
-            _iter333.write(oprot);
+            _iter349.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResul
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tableStats.size());
-        for (ColumnStatisticsObj _iter334 : struct.tableStats)
+        for (ColumnStatisticsObj _iter350 : struct.tableStats)
         {
-          _iter334.write(oprot);
+          _iter350.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResul
     public void read(org.apache.thrift.protocol.TProtocol prot, TableStatsResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list335 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list335.size);
-        ColumnStatisticsObj _elem336;
-        for (int _i337 = 0; _i337 < _list335.size; ++_i337)
+        org.apache.thrift.protocol.TList _list351 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list351.size);
+        ColumnStatisticsObj _elem352;
+        for (int _i353 = 0; _i353 < _list351.size; ++_i353)
         {
-          _elem336 = new ColumnStatisticsObj();
-          _elem336.read(iprot);
-          struct.tableStats.add(_elem336);
+          _elem352 = new ColumnStatisticsObj();
+          _elem352.read(iprot);
+          struct.tableStats.add(_elem352);
         }
       }
       struct.setTableStatsIsSet(true);


[15/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 ac8d8a4..119a5f1 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -156,6 +156,15 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys):
+    """
+    Parameters:
+     - tbl
+     - primaryKeys
+     - foreignKeys
+    """
+    pass
+
   def drop_table(self, dbname, name, deleteData):
     """
     Parameters:
@@ -695,6 +704,20 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_primary_keys(self, request):
+    """
+    Parameters:
+     - request
+    """
+    pass
+
+  def get_foreign_keys(self, request):
+    """
+    Parameters:
+     - request
+    """
+    pass
+
   def update_table_column_statistics(self, stats_obj):
     """
     Parameters:
@@ -1811,6 +1834,47 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o4
     return
 
+  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys):
+    """
+    Parameters:
+     - tbl
+     - primaryKeys
+     - foreignKeys
+    """
+    self.send_create_table_with_constraints(tbl, primaryKeys, foreignKeys)
+    self.recv_create_table_with_constraints()
+
+  def send_create_table_with_constraints(self, tbl, primaryKeys, foreignKeys):
+    self._oprot.writeMessageBegin('create_table_with_constraints', TMessageType.CALL, self._seqid)
+    args = create_table_with_constraints_args()
+    args.tbl = tbl
+    args.primaryKeys = primaryKeys
+    args.foreignKeys = foreignKeys
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_create_table_with_constraints(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = create_table_with_constraints_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    if result.o3 is not None:
+      raise result.o3
+    if result.o4 is not None:
+      raise result.o4
+    return
+
   def drop_table(self, dbname, name, deleteData):
     """
     Parameters:
@@ -4134,6 +4198,76 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o2
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_index_names failed: unknown result")
 
+  def get_primary_keys(self, request):
+    """
+    Parameters:
+     - request
+    """
+    self.send_get_primary_keys(request)
+    return self.recv_get_primary_keys()
+
+  def send_get_primary_keys(self, request):
+    self._oprot.writeMessageBegin('get_primary_keys', TMessageType.CALL, self._seqid)
+    args = get_primary_keys_args()
+    args.request = request
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_primary_keys(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_primary_keys_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_primary_keys failed: unknown result")
+
+  def get_foreign_keys(self, request):
+    """
+    Parameters:
+     - request
+    """
+    self.send_get_foreign_keys(request)
+    return self.recv_get_foreign_keys()
+
+  def send_get_foreign_keys(self, request):
+    self._oprot.writeMessageBegin('get_foreign_keys', TMessageType.CALL, self._seqid)
+    args = get_foreign_keys_args()
+    args.request = request
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_foreign_keys(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_foreign_keys_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_foreign_keys failed: unknown result")
+
   def update_table_column_statistics(self, stats_obj):
     """
     Parameters:
@@ -6364,6 +6498,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_schema_with_environment_context"] = Processor.process_get_schema_with_environment_context
     self._processMap["create_table"] = Processor.process_create_table
     self._processMap["create_table_with_environment_context"] = Processor.process_create_table_with_environment_context
+    self._processMap["create_table_with_constraints"] = Processor.process_create_table_with_constraints
     self._processMap["drop_table"] = Processor.process_drop_table
     self._processMap["drop_table_with_environment_context"] = Processor.process_drop_table_with_environment_context
     self._processMap["get_tables"] = Processor.process_get_tables
@@ -6423,6 +6558,8 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_index_by_name"] = Processor.process_get_index_by_name
     self._processMap["get_indexes"] = Processor.process_get_indexes
     self._processMap["get_index_names"] = Processor.process_get_index_names
+    self._processMap["get_primary_keys"] = Processor.process_get_primary_keys
+    self._processMap["get_foreign_keys"] = Processor.process_get_foreign_keys
     self._processMap["update_table_column_statistics"] = Processor.process_update_table_column_statistics
     self._processMap["update_partition_column_statistics"] = Processor.process_update_partition_column_statistics
     self._processMap["get_table_column_statistics"] = Processor.process_get_table_column_statistics
@@ -6973,6 +7110,37 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_create_table_with_constraints(self, seqid, iprot, oprot):
+    args = create_table_with_constraints_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = create_table_with_constraints_result()
+    try:
+      self._handler.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyExistsException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except InvalidObjectException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except MetaException as o3:
+      msg_type = TMessageType.REPLY
+      result.o3 = o3
+    except NoSuchObjectException as o4:
+      msg_type = TMessageType.REPLY
+      result.o4 = o4
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("create_table_with_constraints", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_drop_table(self, seqid, iprot, oprot):
     args = drop_table_args()
     args.read(iprot)
@@ -8493,6 +8661,56 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_primary_keys(self, seqid, iprot, oprot):
+    args = get_primary_keys_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_primary_keys_result()
+    try:
+      result.success = self._handler.get_primary_keys(args.request)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except NoSuchObjectException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_primary_keys", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_foreign_keys(self, seqid, iprot, oprot):
+    args = get_foreign_keys_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_foreign_keys_result()
+    try:
+      result.success = self._handler.get_foreign_keys(args.request)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except NoSuchObjectException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_foreign_keys", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_update_table_column_statistics(self, seqid, iprot, oprot):
     args = update_table_column_statistics_args()
     args.read(iprot)
@@ -10879,10 +11097,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype525, _size522) = iprot.readListBegin()
-          for _i526 in xrange(_size522):
-            _elem527 = iprot.readString()
-            self.success.append(_elem527)
+          (_etype539, _size536) = iprot.readListBegin()
+          for _i540 in xrange(_size536):
+            _elem541 = iprot.readString()
+            self.success.append(_elem541)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10905,8 +11123,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 iter528 in self.success:
-        oprot.writeString(iter528)
+      for iter542 in self.success:
+        oprot.writeString(iter542)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11011,10 +11229,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype532, _size529) = iprot.readListBegin()
-          for _i533 in xrange(_size529):
-            _elem534 = iprot.readString()
-            self.success.append(_elem534)
+          (_etype546, _size543) = iprot.readListBegin()
+          for _i547 in xrange(_size543):
+            _elem548 = iprot.readString()
+            self.success.append(_elem548)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11037,8 +11255,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 iter535 in self.success:
-        oprot.writeString(iter535)
+      for iter549 in self.success:
+        oprot.writeString(iter549)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11808,12 +12026,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype537, _vtype538, _size536 ) = iprot.readMapBegin()
-          for _i540 in xrange(_size536):
-            _key541 = iprot.readString()
-            _val542 = Type()
-            _val542.read(iprot)
-            self.success[_key541] = _val542
+          (_ktype551, _vtype552, _size550 ) = iprot.readMapBegin()
+          for _i554 in xrange(_size550):
+            _key555 = iprot.readString()
+            _val556 = Type()
+            _val556.read(iprot)
+            self.success[_key555] = _val556
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -11836,9 +12054,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 kiter543,viter544 in self.success.items():
-        oprot.writeString(kiter543)
-        viter544.write(oprot)
+      for kiter557,viter558 in self.success.items():
+        oprot.writeString(kiter557)
+        viter558.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -11981,11 +12199,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype548, _size545) = iprot.readListBegin()
-          for _i549 in xrange(_size545):
-            _elem550 = FieldSchema()
-            _elem550.read(iprot)
-            self.success.append(_elem550)
+          (_etype562, _size559) = iprot.readListBegin()
+          for _i563 in xrange(_size559):
+            _elem564 = FieldSchema()
+            _elem564.read(iprot)
+            self.success.append(_elem564)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12020,8 +12238,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 iter551 in self.success:
-        iter551.write(oprot)
+      for iter565 in self.success:
+        iter565.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12188,11 +12406,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype555, _size552) = iprot.readListBegin()
-          for _i556 in xrange(_size552):
-            _elem557 = FieldSchema()
-            _elem557.read(iprot)
-            self.success.append(_elem557)
+          (_etype569, _size566) = iprot.readListBegin()
+          for _i570 in xrange(_size566):
+            _elem571 = FieldSchema()
+            _elem571.read(iprot)
+            self.success.append(_elem571)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12227,8 +12445,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 iter558 in self.success:
-        iter558.write(oprot)
+      for iter572 in self.success:
+        iter572.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12381,11 +12599,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype562, _size559) = iprot.readListBegin()
-          for _i563 in xrange(_size559):
-            _elem564 = FieldSchema()
-            _elem564.read(iprot)
-            self.success.append(_elem564)
+          (_etype576, _size573) = iprot.readListBegin()
+          for _i577 in xrange(_size573):
+            _elem578 = FieldSchema()
+            _elem578.read(iprot)
+            self.success.append(_elem578)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12420,8 +12638,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 iter565 in self.success:
-        iter565.write(oprot)
+      for iter579 in self.success:
+        iter579.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12588,11 +12806,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype569, _size566) = iprot.readListBegin()
-          for _i570 in xrange(_size566):
-            _elem571 = FieldSchema()
-            _elem571.read(iprot)
-            self.success.append(_elem571)
+          (_etype583, _size580) = iprot.readListBegin()
+          for _i584 in xrange(_size580):
+            _elem585 = FieldSchema()
+            _elem585.read(iprot)
+            self.success.append(_elem585)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12627,8 +12845,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 iter572 in self.success:
-        iter572.write(oprot)
+      for iter586 in self.success:
+        iter586.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13031,25 +13249,25 @@ class create_table_with_environment_context_result:
   def __ne__(self, other):
     return not (self == other)
 
-class drop_table_args:
+class create_table_with_constraints_args:
   """
   Attributes:
-   - dbname
-   - name
-   - deleteData
+   - tbl
+   - primaryKeys
+   - foreignKeys
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'dbname', None, None, ), # 1
-    (2, TType.STRING, 'name', None, None, ), # 2
-    (3, TType.BOOL, 'deleteData', None, None, ), # 3
+    (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1
+    (2, TType.LIST, 'primaryKeys', (TType.STRUCT,(SQLPrimaryKey, SQLPrimaryKey.thrift_spec)), None, ), # 2
+    (3, TType.LIST, 'foreignKeys', (TType.STRUCT,(SQLForeignKey, SQLForeignKey.thrift_spec)), None, ), # 3
   )
 
-  def __init__(self, dbname=None, name=None, deleteData=None,):
-    self.dbname = dbname
-    self.name = name
-    self.deleteData = deleteData
+  def __init__(self, tbl=None, primaryKeys=None, foreignKeys=None,):
+    self.tbl = tbl
+    self.primaryKeys = primaryKeys
+    self.foreignKeys = foreignKeys
 
   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:
@@ -13061,18 +13279,31 @@ class drop_table_args:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.STRING:
-          self.dbname = iprot.readString()
+        if ftype == TType.STRUCT:
+          self.tbl = Table()
+          self.tbl.read(iprot)
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.STRING:
-          self.name = iprot.readString()
+        if ftype == TType.LIST:
+          self.primaryKeys = []
+          (_etype590, _size587) = iprot.readListBegin()
+          for _i591 in xrange(_size587):
+            _elem592 = SQLPrimaryKey()
+            _elem592.read(iprot)
+            self.primaryKeys.append(_elem592)
+          iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
-        if ftype == TType.BOOL:
-          self.deleteData = iprot.readBool()
+        if ftype == TType.LIST:
+          self.foreignKeys = []
+          (_etype596, _size593) = iprot.readListBegin()
+          for _i597 in xrange(_size593):
+            _elem598 = SQLForeignKey()
+            _elem598.read(iprot)
+            self.foreignKeys.append(_elem598)
+          iprot.readListEnd()
         else:
           iprot.skip(ftype)
       else:
@@ -13084,18 +13315,24 @@ class drop_table_args:
     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('drop_table_args')
-    if self.dbname is not None:
-      oprot.writeFieldBegin('dbname', TType.STRING, 1)
-      oprot.writeString(self.dbname)
+    oprot.writeStructBegin('create_table_with_constraints_args')
+    if self.tbl is not None:
+      oprot.writeFieldBegin('tbl', TType.STRUCT, 1)
+      self.tbl.write(oprot)
       oprot.writeFieldEnd()
-    if self.name is not None:
-      oprot.writeFieldBegin('name', TType.STRING, 2)
-      oprot.writeString(self.name)
+    if self.primaryKeys is not None:
+      oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
+      oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
+      for iter599 in self.primaryKeys:
+        iter599.write(oprot)
+      oprot.writeListEnd()
       oprot.writeFieldEnd()
-    if self.deleteData is not None:
-      oprot.writeFieldBegin('deleteData', TType.BOOL, 3)
-      oprot.writeBool(self.deleteData)
+    if self.foreignKeys is not None:
+      oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
+      oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
+      for iter600 in self.foreignKeys:
+        iter600.write(oprot)
+      oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -13106,9 +13343,9 @@ class drop_table_args:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.dbname)
-    value = (value * 31) ^ hash(self.name)
-    value = (value * 31) ^ hash(self.deleteData)
+    value = (value * 31) ^ hash(self.tbl)
+    value = (value * 31) ^ hash(self.primaryKeys)
+    value = (value * 31) ^ hash(self.foreignKeys)
     return value
 
   def __repr__(self):
@@ -13122,18 +13359,217 @@ class drop_table_args:
   def __ne__(self, other):
     return not (self == other)
 
-class drop_table_result:
+class create_table_with_constraints_result:
   """
   Attributes:
    - o1
+   - o2
    - o3
+   - o4
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
-    (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2
-  )
+    (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3
+    (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4
+  )
+
+  def __init__(self, o1=None, o2=None, o3=None, o4=None,):
+    self.o1 = o1
+    self.o2 = o2
+    self.o3 = o3
+    self.o4 = o4
+
+  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.o1 = AlreadyExistsException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = InvalidObjectException()
+          self.o2.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.o3 = MetaException()
+          self.o3.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRUCT:
+          self.o4 = NoSuchObjectException()
+          self.o4.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('create_table_with_constraints_result')
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o3 is not None:
+      oprot.writeFieldBegin('o3', TType.STRUCT, 3)
+      self.o3.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o4 is not None:
+      oprot.writeFieldBegin('o4', TType.STRUCT, 4)
+      self.o4.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    value = (value * 31) ^ hash(self.o3)
+    value = (value * 31) ^ hash(self.o4)
+    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 drop_table_args:
+  """
+  Attributes:
+   - dbname
+   - name
+   - deleteData
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'dbname', None, None, ), # 1
+    (2, TType.STRING, 'name', None, None, ), # 2
+    (3, TType.BOOL, 'deleteData', None, None, ), # 3
+  )
+
+  def __init__(self, dbname=None, name=None, deleteData=None,):
+    self.dbname = dbname
+    self.name = name
+    self.deleteData = deleteData
+
+  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.dbname = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.BOOL:
+          self.deleteData = 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('drop_table_args')
+    if self.dbname is not None:
+      oprot.writeFieldBegin('dbname', TType.STRING, 1)
+      oprot.writeString(self.dbname)
+      oprot.writeFieldEnd()
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 2)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    if self.deleteData is not None:
+      oprot.writeFieldBegin('deleteData', TType.BOOL, 3)
+      oprot.writeBool(self.deleteData)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.dbname)
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.deleteData)
+    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 drop_table_result:
+  """
+  Attributes:
+   - o1
+   - o3
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2
+  )
 
   def __init__(self, o1=None, o3=None,):
     self.o1 = o1
@@ -13493,10 +13929,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype576, _size573) = iprot.readListBegin()
-          for _i577 in xrange(_size573):
-            _elem578 = iprot.readString()
-            self.success.append(_elem578)
+          (_etype604, _size601) = iprot.readListBegin()
+          for _i605 in xrange(_size601):
+            _elem606 = iprot.readString()
+            self.success.append(_elem606)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13519,8 +13955,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 iter579 in self.success:
-        oprot.writeString(iter579)
+      for iter607 in self.success:
+        oprot.writeString(iter607)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13593,10 +14029,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype583, _size580) = iprot.readListBegin()
-          for _i584 in xrange(_size580):
-            _elem585 = iprot.readString()
-            self.tbl_types.append(_elem585)
+          (_etype611, _size608) = iprot.readListBegin()
+          for _i612 in xrange(_size608):
+            _elem613 = iprot.readString()
+            self.tbl_types.append(_elem613)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13621,8 +14057,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter586 in self.tbl_types:
-        oprot.writeString(iter586)
+      for iter614 in self.tbl_types:
+        oprot.writeString(iter614)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13678,11 +14114,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype590, _size587) = iprot.readListBegin()
-          for _i591 in xrange(_size587):
-            _elem592 = TableMeta()
-            _elem592.read(iprot)
-            self.success.append(_elem592)
+          (_etype618, _size615) = iprot.readListBegin()
+          for _i619 in xrange(_size615):
+            _elem620 = TableMeta()
+            _elem620.read(iprot)
+            self.success.append(_elem620)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13705,8 +14141,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter593 in self.success:
-        iter593.write(oprot)
+      for iter621 in self.success:
+        iter621.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13830,10 +14266,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype597, _size594) = iprot.readListBegin()
-          for _i598 in xrange(_size594):
-            _elem599 = iprot.readString()
-            self.success.append(_elem599)
+          (_etype625, _size622) = iprot.readListBegin()
+          for _i626 in xrange(_size622):
+            _elem627 = iprot.readString()
+            self.success.append(_elem627)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13856,8 +14292,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 iter600 in self.success:
-        oprot.writeString(iter600)
+      for iter628 in self.success:
+        oprot.writeString(iter628)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14093,10 +14529,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype604, _size601) = iprot.readListBegin()
-          for _i605 in xrange(_size601):
-            _elem606 = iprot.readString()
-            self.tbl_names.append(_elem606)
+          (_etype632, _size629) = iprot.readListBegin()
+          for _i633 in xrange(_size629):
+            _elem634 = iprot.readString()
+            self.tbl_names.append(_elem634)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14117,8 +14553,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 iter607 in self.tbl_names:
-        oprot.writeString(iter607)
+      for iter635 in self.tbl_names:
+        oprot.writeString(iter635)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14179,11 +14615,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype611, _size608) = iprot.readListBegin()
-          for _i612 in xrange(_size608):
-            _elem613 = Table()
-            _elem613.read(iprot)
-            self.success.append(_elem613)
+          (_etype639, _size636) = iprot.readListBegin()
+          for _i640 in xrange(_size636):
+            _elem641 = Table()
+            _elem641.read(iprot)
+            self.success.append(_elem641)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14218,8 +14654,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 iter614 in self.success:
-        iter614.write(oprot)
+      for iter642 in self.success:
+        iter642.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14385,10 +14821,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype618, _size615) = iprot.readListBegin()
-          for _i619 in xrange(_size615):
-            _elem620 = iprot.readString()
-            self.success.append(_elem620)
+          (_etype646, _size643) = iprot.readListBegin()
+          for _i647 in xrange(_size643):
+            _elem648 = iprot.readString()
+            self.success.append(_elem648)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14423,8 +14859,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 iter621 in self.success:
-        oprot.writeString(iter621)
+      for iter649 in self.success:
+        oprot.writeString(iter649)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15394,11 +15830,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype625, _size622) = iprot.readListBegin()
-          for _i626 in xrange(_size622):
-            _elem627 = Partition()
-            _elem627.read(iprot)
-            self.new_parts.append(_elem627)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = Partition()
+            _elem655.read(iprot)
+            self.new_parts.append(_elem655)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15415,8 +15851,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 iter628 in self.new_parts:
-        iter628.write(oprot)
+      for iter656 in self.new_parts:
+        iter656.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15574,11 +16010,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype632, _size629) = iprot.readListBegin()
-          for _i633 in xrange(_size629):
-            _elem634 = PartitionSpec()
-            _elem634.read(iprot)
-            self.new_parts.append(_elem634)
+          (_etype660, _size657) = iprot.readListBegin()
+          for _i661 in xrange(_size657):
+            _elem662 = PartitionSpec()
+            _elem662.read(iprot)
+            self.new_parts.append(_elem662)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15595,8 +16031,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 iter635 in self.new_parts:
-        iter635.write(oprot)
+      for iter663 in self.new_parts:
+        iter663.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15770,10 +16206,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype639, _size636) = iprot.readListBegin()
-          for _i640 in xrange(_size636):
-            _elem641 = iprot.readString()
-            self.part_vals.append(_elem641)
+          (_etype667, _size664) = iprot.readListBegin()
+          for _i668 in xrange(_size664):
+            _elem669 = iprot.readString()
+            self.part_vals.append(_elem669)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15798,8 +16234,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 iter642 in self.part_vals:
-        oprot.writeString(iter642)
+      for iter670 in self.part_vals:
+        oprot.writeString(iter670)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16152,10 +16588,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype646, _size643) = iprot.readListBegin()
-          for _i647 in xrange(_size643):
-            _elem648 = iprot.readString()
-            self.part_vals.append(_elem648)
+          (_etype674, _size671) = iprot.readListBegin()
+          for _i675 in xrange(_size671):
+            _elem676 = iprot.readString()
+            self.part_vals.append(_elem676)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16186,8 +16622,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 iter649 in self.part_vals:
-        oprot.writeString(iter649)
+      for iter677 in self.part_vals:
+        oprot.writeString(iter677)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -16782,10 +17218,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype653, _size650) = iprot.readListBegin()
-          for _i654 in xrange(_size650):
-            _elem655 = iprot.readString()
-            self.part_vals.append(_elem655)
+          (_etype681, _size678) = iprot.readListBegin()
+          for _i682 in xrange(_size678):
+            _elem683 = iprot.readString()
+            self.part_vals.append(_elem683)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16815,8 +17251,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 iter656 in self.part_vals:
-        oprot.writeString(iter656)
+      for iter684 in self.part_vals:
+        oprot.writeString(iter684)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -16989,10 +17425,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype660, _size657) = iprot.readListBegin()
-          for _i661 in xrange(_size657):
-            _elem662 = iprot.readString()
-            self.part_vals.append(_elem662)
+          (_etype688, _size685) = iprot.readListBegin()
+          for _i689 in xrange(_size685):
+            _elem690 = iprot.readString()
+            self.part_vals.append(_elem690)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17028,8 +17464,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 iter663 in self.part_vals:
-        oprot.writeString(iter663)
+      for iter691 in self.part_vals:
+        oprot.writeString(iter691)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -17766,10 +18202,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype667, _size664) = iprot.readListBegin()
-          for _i668 in xrange(_size664):
-            _elem669 = iprot.readString()
-            self.part_vals.append(_elem669)
+          (_etype695, _size692) = iprot.readListBegin()
+          for _i696 in xrange(_size692):
+            _elem697 = iprot.readString()
+            self.part_vals.append(_elem697)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17794,8 +18230,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 iter670 in self.part_vals:
-        oprot.writeString(iter670)
+      for iter698 in self.part_vals:
+        oprot.writeString(iter698)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17954,11 +18390,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype672, _vtype673, _size671 ) = iprot.readMapBegin()
-          for _i675 in xrange(_size671):
-            _key676 = iprot.readString()
-            _val677 = iprot.readString()
-            self.partitionSpecs[_key676] = _val677
+          (_ktype700, _vtype701, _size699 ) = iprot.readMapBegin()
+          for _i703 in xrange(_size699):
+            _key704 = iprot.readString()
+            _val705 = iprot.readString()
+            self.partitionSpecs[_key704] = _val705
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17995,9 +18431,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 kiter678,viter679 in self.partitionSpecs.items():
-        oprot.writeString(kiter678)
-        oprot.writeString(viter679)
+      for kiter706,viter707 in self.partitionSpecs.items():
+        oprot.writeString(kiter706)
+        oprot.writeString(viter707)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -18202,11 +18638,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype681, _vtype682, _size680 ) = iprot.readMapBegin()
-          for _i684 in xrange(_size680):
-            _key685 = iprot.readString()
-            _val686 = iprot.readString()
-            self.partitionSpecs[_key685] = _val686
+          (_ktype709, _vtype710, _size708 ) = iprot.readMapBegin()
+          for _i712 in xrange(_size708):
+            _key713 = iprot.readString()
+            _val714 = iprot.readString()
+            self.partitionSpecs[_key713] = _val714
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -18243,9 +18679,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter687,viter688 in self.partitionSpecs.items():
-        oprot.writeString(kiter687)
-        oprot.writeString(viter688)
+      for kiter715,viter716 in self.partitionSpecs.items():
+        oprot.writeString(kiter715)
+        oprot.writeString(viter716)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -18328,11 +18764,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype692, _size689) = iprot.readListBegin()
-          for _i693 in xrange(_size689):
-            _elem694 = Partition()
-            _elem694.read(iprot)
-            self.success.append(_elem694)
+          (_etype720, _size717) = iprot.readListBegin()
+          for _i721 in xrange(_size717):
+            _elem722 = Partition()
+            _elem722.read(iprot)
+            self.success.append(_elem722)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18373,8 +18809,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter695 in self.success:
-        iter695.write(oprot)
+      for iter723 in self.success:
+        iter723.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18468,10 +18904,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype699, _size696) = iprot.readListBegin()
-          for _i700 in xrange(_size696):
-            _elem701 = iprot.readString()
-            self.part_vals.append(_elem701)
+          (_etype727, _size724) = iprot.readListBegin()
+          for _i728 in xrange(_size724):
+            _elem729 = iprot.readString()
+            self.part_vals.append(_elem729)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18483,10 +18919,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype705, _size702) = iprot.readListBegin()
-          for _i706 in xrange(_size702):
-            _elem707 = iprot.readString()
-            self.group_names.append(_elem707)
+          (_etype733, _size730) = iprot.readListBegin()
+          for _i734 in xrange(_size730):
+            _elem735 = iprot.readString()
+            self.group_names.append(_elem735)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18511,8 +18947,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 iter708 in self.part_vals:
-        oprot.writeString(iter708)
+      for iter736 in self.part_vals:
+        oprot.writeString(iter736)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -18522,8 +18958,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 iter709 in self.group_names:
-        oprot.writeString(iter709)
+      for iter737 in self.group_names:
+        oprot.writeString(iter737)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18952,11 +19388,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype713, _size710) = iprot.readListBegin()
-          for _i714 in xrange(_size710):
-            _elem715 = Partition()
-            _elem715.read(iprot)
-            self.success.append(_elem715)
+          (_etype741, _size738) = iprot.readListBegin()
+          for _i742 in xrange(_size738):
+            _elem743 = Partition()
+            _elem743.read(iprot)
+            self.success.append(_elem743)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18985,8 +19421,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 iter716 in self.success:
-        iter716.write(oprot)
+      for iter744 in self.success:
+        iter744.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19080,10 +19516,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype720, _size717) = iprot.readListBegin()
-          for _i721 in xrange(_size717):
-            _elem722 = iprot.readString()
-            self.group_names.append(_elem722)
+          (_etype748, _size745) = iprot.readListBegin()
+          for _i749 in xrange(_size745):
+            _elem750 = iprot.readString()
+            self.group_names.append(_elem750)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19116,8 +19552,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 iter723 in self.group_names:
-        oprot.writeString(iter723)
+      for iter751 in self.group_names:
+        oprot.writeString(iter751)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19178,11 +19614,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype727, _size724) = iprot.readListBegin()
-          for _i728 in xrange(_size724):
-            _elem729 = Partition()
-            _elem729.read(iprot)
-            self.success.append(_elem729)
+          (_etype755, _size752) = iprot.readListBegin()
+          for _i756 in xrange(_size752):
+            _elem757 = Partition()
+            _elem757.read(iprot)
+            self.success.append(_elem757)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19211,8 +19647,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 iter730 in self.success:
-        iter730.write(oprot)
+      for iter758 in self.success:
+        iter758.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19370,11 +19806,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype734, _size731) = iprot.readListBegin()
-          for _i735 in xrange(_size731):
-            _elem736 = PartitionSpec()
-            _elem736.read(iprot)
-            self.success.append(_elem736)
+          (_etype762, _size759) = iprot.readListBegin()
+          for _i763 in xrange(_size759):
+            _elem764 = PartitionSpec()
+            _elem764.read(iprot)
+            self.success.append(_elem764)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19403,8 +19839,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 iter737 in self.success:
-        iter737.write(oprot)
+      for iter765 in self.success:
+        iter765.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19559,10 +19995,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype741, _size738) = iprot.readListBegin()
-          for _i742 in xrange(_size738):
-            _elem743 = iprot.readString()
-            self.success.append(_elem743)
+          (_etype769, _size766) = iprot.readListBegin()
+          for _i770 in xrange(_size766):
+            _elem771 = iprot.readString()
+            self.success.append(_elem771)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19585,8 +20021,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 iter744 in self.success:
-        oprot.writeString(iter744)
+      for iter772 in self.success:
+        oprot.writeString(iter772)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -19662,10 +20098,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype748, _size745) = iprot.readListBegin()
-          for _i749 in xrange(_size745):
-            _elem750 = iprot.readString()
-            self.part_vals.append(_elem750)
+          (_etype776, _size773) = iprot.readListBegin()
+          for _i777 in xrange(_size773):
+            _elem778 = iprot.readString()
+            self.part_vals.append(_elem778)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19695,8 +20131,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 iter751 in self.part_vals:
-        oprot.writeString(iter751)
+      for iter779 in self.part_vals:
+        oprot.writeString(iter779)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -19760,11 +20196,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype755, _size752) = iprot.readListBegin()
-          for _i756 in xrange(_size752):
-            _elem757 = Partition()
-            _elem757.read(iprot)
-            self.success.append(_elem757)
+          (_etype783, _size780) = iprot.readListBegin()
+          for _i784 in xrange(_size780):
+            _elem785 = Partition()
+            _elem785.read(iprot)
+            self.success.append(_elem785)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19793,8 +20229,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 iter758 in self.success:
-        iter758.write(oprot)
+      for iter786 in self.success:
+        iter786.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19881,10 +20317,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype762, _size759) = iprot.readListBegin()
-          for _i763 in xrange(_size759):
-            _elem764 = iprot.readString()
-            self.part_vals.append(_elem764)
+          (_etype790, _size787) = iprot.readListBegin()
+          for _i791 in xrange(_size787):
+            _elem792 = iprot.readString()
+            self.part_vals.append(_elem792)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19901,10 +20337,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype768, _size765) = iprot.readListBegin()
-          for _i769 in xrange(_size765):
-            _elem770 = iprot.readString()
-            self.group_names.append(_elem770)
+          (_etype796, _size793) = iprot.readListBegin()
+          for _i797 in xrange(_size793):
+            _elem798 = iprot.readString()
+            self.group_names.append(_elem798)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19929,8 +20365,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 iter771 in self.part_vals:
-        oprot.writeString(iter771)
+      for iter799 in self.part_vals:
+        oprot.writeString(iter799)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -19944,8 +20380,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 iter772 in self.group_names:
-        oprot.writeString(iter772)
+      for iter800 in self.group_names:
+        oprot.writeString(iter800)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20007,11 +20443,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = Partition()
-            _elem778.read(iprot)
-            self.success.append(_elem778)
+          (_etype804, _size801) = iprot.readListBegin()
+          for _i805 in xrange(_size801):
+            _elem806 = Partition()
+            _elem806.read(iprot)
+            self.success.append(_elem806)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20040,8 +20476,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 iter779 in self.success:
-        iter779.write(oprot)
+      for iter807 in self.success:
+        iter807.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20122,10 +20558,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype783, _size780) = iprot.readListBegin()
-          for _i784 in xrange(_size780):
-            _elem785 = iprot.readString()
-            self.part_vals.append(_elem785)
+          (_etype811, _size808) = iprot.readListBegin()
+          for _i812 in xrange(_size808):
+            _elem813 = iprot.readString()
+            self.part_vals.append(_elem813)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20155,8 +20591,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 iter786 in self.part_vals:
-        oprot.writeString(iter786)
+      for iter814 in self.part_vals:
+        oprot.writeString(iter814)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -20220,10 +20656,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype790, _size787) = iprot.readListBegin()
-          for _i791 in xrange(_size787):
-            _elem792 = iprot.readString()
-            self.success.append(_elem792)
+          (_etype818, _size815) = iprot.readListBegin()
+          for _i819 in xrange(_size815):
+            _elem820 = iprot.readString()
+            self.success.append(_elem820)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20252,8 +20688,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 iter793 in self.success:
-        oprot.writeString(iter793)
+      for iter821 in self.success:
+        oprot.writeString(iter821)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20424,11 +20860,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype797, _size794) = iprot.readListBegin()
-          for _i798 in xrange(_size794):
-            _elem799 = Partition()
-            _elem799.read(iprot)
-            self.success.append(_elem799)
+          (_etype825, _size822) = iprot.readListBegin()
+          for _i826 in xrange(_size822):
+            _elem827 = Partition()
+            _elem827.read(iprot)
+            self.success.append(_elem827)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20457,8 +20893,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 iter800 in self.success:
-        iter800.write(oprot)
+      for iter828 in self.success:
+        iter828.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20629,11 +21065,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype804, _size801) = iprot.readListBegin()
-          for _i805 in xrange(_size801):
-            _elem806 = PartitionSpec()
-            _elem806.read(iprot)
-            self.success.append(_elem806)
+          (_etype832, _size829) = iprot.readListBegin()
+          for _i833 in xrange(_size829):
+            _elem834 = PartitionSpec()
+            _elem834.read(iprot)
+            self.success.append(_elem834)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20662,8 +21098,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 iter807 in self.success:
-        iter807.write(oprot)
+      for iter835 in self.success:
+        iter835.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21083,10 +21519,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype811, _size808) = iprot.readListBegin()
-          for _i812 in xrange(_size808):
-            _elem813 = iprot.readString()
-            self.names.append(_elem813)
+          (_etype839, _size836) = iprot.readListBegin()
+          for _i840 in xrange(_size836):
+            _elem841 = iprot.readString()
+            self.names.append(_elem841)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21111,8 +21547,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 iter814 in self.names:
-        oprot.writeString(iter814)
+      for iter842 in self.names:
+        oprot.writeString(iter842)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21171,11 +21607,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype818, _size815) = iprot.readListBegin()
-          for _i819 in xrange(_size815):
-            _elem820 = Partition()
-            _elem820.read(iprot)
-            self.success.append(_elem820)
+          (_etype846, _size843) = iprot.readListBegin()
+          for _i847 in xrange(_size843):
+            _elem848 = Partition()
+            _elem848.read(iprot)
+            self.success.append(_elem848)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21204,8 +21640,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 iter821 in self.success:
-        iter821.write(oprot)
+      for iter849 in self.success:
+        iter849.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21455,11 +21891,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype825, _size822) = iprot.readListBegin()
-          for _i826 in xrange(_size822):
-            _elem827 = Partition()
-            _elem827.read(iprot)
-            self.new_parts.append(_elem827)
+          (_etype853, _size850) = iprot.readListBegin()
+          for _i854 in xrange(_size850):
+            _elem855 = Partition()
+            _elem855.read(iprot)
+            self.new_parts.append(_elem855)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21484,8 +21920,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 iter828 in self.new_parts:
-        iter828.write(oprot)
+      for iter856 in self.new_parts:
+        iter856.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21638,11 +22074,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype832, _size829) = iprot.readListBegin()
-          for _i833 in xrange(_size829):
-            _elem834 = Partition()
-            _elem834.read(iprot)
-            self.new_parts.append(_elem834)
+          (_etype860, _size857) = iprot.readListBegin()
+          for _i861 in xrange(_size857):
+            _elem862 = Partition()
+            _elem862.read(iprot)
+            self.new_parts.append(_elem862)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21673,8 +22109,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter835 in self.new_parts:
-        iter835.write(oprot)
+      for iter863 in self.new_parts:
+        iter863.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -22018,10 +22454,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype839, _size836) = iprot.readListBegin()
-          for _i840 in xrange(_size836):
-            _elem841 = iprot.readString()
-            self.part_vals.append(_elem841)
+          (_etype867, _size864) = iprot.readListBegin()
+          for _i868 in xrange(_size864):
+            _elem869 = iprot.readString()
+            self.part_vals.append(_elem869)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22052,8 +22488,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 iter842 in self.part_vals:
-        oprot.writeString(iter842)
+      for iter870 in self.part_vals:
+        oprot.writeString(iter870)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -22195,10 +22631,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype846, _size843) = iprot.readListBegin()
-          for _i847 in xrange(_size843):
-            _elem848 = iprot.readString()
-            self.part_vals.append(_elem848)
+          (_etype874, _size871) = iprot.readListBegin()
+          for _i875 in xrange(_size871):
+            _elem876 = iprot.readString()
+            self.part_vals.append(_elem876)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22220,8 +22656,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 iter849 in self.part_vals:
-        oprot.writeString(iter849)
+      for iter877 in self.part_vals:
+        oprot.writeString(iter877)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -22579,10 +23015,10 @@ class partition_name_to_vals_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)
+          (_etype881, _size878) = iprot.readListBegin()
+          for _i882 in xrange(_size878):
+            _elem883 = iprot.readString()
+            self.success.append(_elem883)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22605,8 +23041,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 iter856 in self.success:
-        oprot.writeString(iter856)
+      for iter884 in self.success:
+        oprot.writeString(iter884)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22730,11 +23166,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype858, _vtype859, _size857 ) = iprot.readMapBegin()
-          for _i861 in xrange(_size857):
-            _key862 = iprot.readString()
-            _val863 = iprot.readString()
-            self.success[_key862] = _val863
+          (_ktype886, _vtype887, _size885 ) = iprot.readMapBegin()
+          for _i889 in xrange(_size885):
+            _key890 = iprot.readString()
+            _val891 = iprot.readString()
+            self.success[_key890] = _val891
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -22757,9 +23193,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 kiter864,viter865 in self.success.items():
-        oprot.writeString(kiter864)
-        oprot.writeString(viter865)
+      for kiter892,viter893 in self.success.items():
+        oprot.writeString(kiter892)
+        oprot.writeString(viter893)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22835,11 +23271,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype867, _vtype868, _size866 ) = iprot.readMapBegin()
-          for _i870 in xrange(_size866):
-            _key871 = iprot.readString()
-            _val872 = iprot.readString()
-            self.part_vals[_key871] = _val872
+          (_ktype895, _vtype896, _size894 ) = iprot.readMapBegin()
+          for _i898 in xrange(_size894):
+            _key899 = iprot.readString()
+            _val900 = iprot.readString()
+            self.part_vals[_key899] = _val900
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -22869,9 +23305,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 kiter873,viter874 in self.part_vals.items():
-        oprot.writeString(kiter873)
-        oprot.writeString(viter874)
+      for kiter901,viter902 in self.part_vals.items():
+        oprot.writeString(kiter901)
+        oprot.writeString(viter902)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -23085,11 +23521,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype876, _vtype877, _size875 ) = iprot.readMapBegin()
-          for _i879 in xrange(_size875):
-            _key880 = iprot.readString()
-            _val881 = iprot.readString()
-            self.part_vals[_key880] = _val881
+          (_ktype904, _vtype905, _size903 ) = iprot.readMapBegin()
+          for _i907 in xrange(_size903):
+            _key908 = iprot.readString()
+            _val909 = iprot.readString()
+            self.part_vals[_key908] = _val909
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23119,9 +23555,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 kiter882,viter883 in self.part_vals.items():
-        oprot.writeString(kiter882)
-        oprot.writeString(viter883)
+      for kiter910,viter911 in self.part_vals.items():
+        oprot.writeString(kiter910)
+        oprot.writeString(viter911)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -23923,7 +24359,383 @@ class get_index_by_name_args:
     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_index_by_name_args')
+    oprot.writeStructBegin('get_index_by_name_args')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
+      oprot.writeFieldEnd()
+    if self.tbl_name is not None:
+      oprot.writeFieldBegin('tbl_name', TType.STRING, 2)
+      oprot.writeString(self.tbl_name)
+      oprot.writeFieldEnd()
+    if self.index_name is not None:
+      oprot.writeFieldBegin('index_name', TType.STRING, 3)
+      oprot.writeString(self.index_name)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.tbl_name)
+    value = (value * 31) ^ hash(self.index_name)
+    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_index_by_name_result:
+  """
+  Attributes:
+   - success
+   - o1
+   - o2
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, success=None, o1=None, o2=None,):
+    self.success = success
+    self.o1 = o1
+    self.o2 = o2
+
+  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 = Index()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = NoSuchObjectException()
+          self.o2.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_index_by_name_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    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_indexes_args:
+  """
+  Attributes:
+   - db_name
+   - tbl_name
+   - max_indexes
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'tbl_name', None, None, ), # 2
+    (3, TType.I16, 'max_indexes', None, -1, ), # 3
+  )
+
+  def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],):
+    self.db_name = db_name
+    self.tbl_name = tbl_name
+    self.max_indexes = max_indexes
+
+  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.db_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tbl_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I16:
+          self.max_indexes = iprot.readI16()
+        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_indexes_args')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
+      oprot.writeFieldEnd()
+    if self.tbl_name is not None:
+      oprot.writeFieldBegin('tbl_name', TType.STRING, 2)
+      oprot.writeString(self.tbl_name)
+      oprot.writeFieldEnd()
+    if self.max_indexes is not None:
+      oprot.writeFieldBegin('max_indexes', TType.I16, 3)
+      oprot.writeI16(self.max_indexes)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.tbl_name)
+    value = (value * 31) ^ hash(self.max_indexes)
+    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_indexes_result:
+  """
+  Attributes:
+   - success
+   - o1
+   - o2
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRUCT,(Index, Index.thrift_spec)), None, ), # 0
+    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, success=None, o1=None, o2=None,):
+    self.success = success
+    self.o1 = o1
+    self.o2 = o2
+
+  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.LIST:
+          self.success = []
+          (_etype915, _size912) = iprot.readListBegin()
+          for _i916 in xrange(_size912):
+            _elem917 = Index()
+            _elem917.read(iprot)
+            self.success.append(_elem917)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = NoSuchObjectException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = MetaException()
+          self.o2.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_indexes_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRUCT, len(self.success))
+      for iter918 in self.success:
+        iter918.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    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_index_names_args:
+  """
+  Attributes:
+   - db_name
+   - tbl_name
+   - max_indexes
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'tbl_name', None, None, ), # 2
+    (3, TType.I16, 'max_indexes', None, -1, ), # 3
+  )
+
+  def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],):
+    self.db_name = db_name
+    self.tbl_name = tbl_name
+    self.max_indexes = max_indexes
+
+  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.db_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tbl_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I16:
+          self.max_indexes = iprot.readI16()
+        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_index_names_args')
     if self.db_name is not None:
       oprot.writeFieldBegin('db_name', TType.STRING, 1)
       oprot.writeString(self.db_name)
@@ -23932,9 +24744,9 @@ class get_index_by_name_args:
       oprot.writeFieldBegin('tbl_name', TType.STRING, 2)
       oprot.writeString(self.tbl_name)
       oprot.writeFieldEnd()
-    if self.index_name is not None:
-      oprot.writeFieldBegin('index_name', TType.STRING, 3)
-      oprot.writeString(self.index_name)
+    if self.max_indexes is not None:
+      oprot.writeFieldBegin('max_indexes', TType.I16, 3)
+      oprot.writeI16(self.max_indexes)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ 

<TRUNCATED>

[17/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 488a920..e2fa963 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -369,70 +369,109 @@ class FieldSchema {
 
 }
 
-class Type {
+class SQLPrimaryKey {
   static $_TSPEC;
 
   /**
    * @var string
    */
-  public $name = null;
+  public $table_db = null;
   /**
    * @var string
    */
-  public $type1 = null;
+  public $table_name = null;
   /**
    * @var string
    */
-  public $type2 = null;
+  public $column_name = null;
   /**
-   * @var \metastore\FieldSchema[]
+   * @var int
    */
-  public $fields = null;
+  public $key_seq = null;
+  /**
+   * @var string
+   */
+  public $pk_name = null;
+  /**
+   * @var bool
+   */
+  public $enable_cstr = null;
+  /**
+   * @var bool
+   */
+  public $validate_cstr = null;
+  /**
+   * @var bool
+   */
+  public $rely_cstr = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'name',
+          'var' => 'table_db',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'type1',
+          'var' => 'table_name',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'type2',
+          'var' => 'column_name',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'fields',
-          'type' => TType::LST,
-          'etype' => TType::STRUCT,
-          'elem' => array(
-            'type' => TType::STRUCT,
-            'class' => '\metastore\FieldSchema',
-            ),
+          'var' => 'key_seq',
+          'type' => TType::I32,
+          ),
+        5 => array(
+          'var' => 'pk_name',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'enable_cstr',
+          'type' => TType::BOOL,
+          ),
+        7 => array(
+          'var' => 'validate_cstr',
+          'type' => TType::BOOL,
+          ),
+        8 => array(
+          'var' => 'rely_cstr',
+          'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['name'])) {
-        $this->name = $vals['name'];
+      if (isset($vals['table_db'])) {
+        $this->table_db = $vals['table_db'];
       }
-      if (isset($vals['type1'])) {
-        $this->type1 = $vals['type1'];
+      if (isset($vals['table_name'])) {
+        $this->table_name = $vals['table_name'];
       }
-      if (isset($vals['type2'])) {
-        $this->type2 = $vals['type2'];
+      if (isset($vals['column_name'])) {
+        $this->column_name = $vals['column_name'];
       }
-      if (isset($vals['fields'])) {
-        $this->fields = $vals['fields'];
+      if (isset($vals['key_seq'])) {
+        $this->key_seq = $vals['key_seq'];
+      }
+      if (isset($vals['pk_name'])) {
+        $this->pk_name = $vals['pk_name'];
+      }
+      if (isset($vals['enable_cstr'])) {
+        $this->enable_cstr = $vals['enable_cstr'];
+      }
+      if (isset($vals['validate_cstr'])) {
+        $this->validate_cstr = $vals['validate_cstr'];
+      }
+      if (isset($vals['rely_cstr'])) {
+        $this->rely_cstr = $vals['rely_cstr'];
       }
     }
   }
 
   public function getName() {
-    return 'Type';
+    return 'SQLPrimaryKey';
   }
 
   public function read($input)
@@ -452,39 +491,56 @@ class Type {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->name);
+            $xfer += $input->readString($this->table_db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->type1);
+            $xfer += $input->readString($this->table_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->type2);
+            $xfer += $input->readString($this->column_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
-          if ($ftype == TType::LST) {
-            $this->fields = array();
-            $_size0 = 0;
-            $_etype3 = 0;
-            $xfer += $input->readListBegin($_etype3, $_size0);
-            for ($_i4 = 0; $_i4 < $_size0; ++$_i4)
-            {
-              $elem5 = null;
-              $elem5 = new \metastore\FieldSchema();
-              $xfer += $elem5->read($input);
-              $this->fields []= $elem5;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->key_seq);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->pk_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->enable_cstr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->validate_cstr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 8:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->rely_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -501,37 +557,45 @@ class Type {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('Type');
-    if ($this->name !== null) {
-      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
-      $xfer += $output->writeString($this->name);
+    $xfer += $output->writeStructBegin('SQLPrimaryKey');
+    if ($this->table_db !== null) {
+      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 1);
+      $xfer += $output->writeString($this->table_db);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->type1 !== null) {
-      $xfer += $output->writeFieldBegin('type1', TType::STRING, 2);
-      $xfer += $output->writeString($this->type1);
+    if ($this->table_name !== null) {
+      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2);
+      $xfer += $output->writeString($this->table_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->type2 !== null) {
-      $xfer += $output->writeFieldBegin('type2', TType::STRING, 3);
-      $xfer += $output->writeString($this->type2);
+    if ($this->column_name !== null) {
+      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 3);
+      $xfer += $output->writeString($this->column_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->fields !== null) {
-      if (!is_array($this->fields)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('fields', TType::LST, 4);
-      {
-        $output->writeListBegin(TType::STRUCT, count($this->fields));
-        {
-          foreach ($this->fields as $iter6)
-          {
-            $xfer += $iter6->write($output);
-          }
-        }
-        $output->writeListEnd();
-      }
+    if ($this->key_seq !== null) {
+      $xfer += $output->writeFieldBegin('key_seq', TType::I32, 4);
+      $xfer += $output->writeI32($this->key_seq);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->pk_name !== null) {
+      $xfer += $output->writeFieldBegin('pk_name', TType::STRING, 5);
+      $xfer += $output->writeString($this->pk_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->enable_cstr !== null) {
+      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 6);
+      $xfer += $output->writeBool($this->enable_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->validate_cstr !== null) {
+      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 7);
+      $xfer += $output->writeBool($this->validate_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->rely_cstr !== null) {
+      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 8);
+      $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -541,80 +605,175 @@ class Type {
 
 }
 
-class HiveObjectRef {
+class SQLForeignKey {
   static $_TSPEC;
 
   /**
-   * @var int
+   * @var string
    */
-  public $objectType = null;
+  public $pktable_db = null;
   /**
    * @var string
    */
-  public $dbName = null;
+  public $pktable_name = null;
   /**
    * @var string
    */
-  public $objectName = null;
+  public $pkcolumn_name = null;
   /**
-   * @var string[]
+   * @var string
    */
-  public $partValues = null;
+  public $fktable_db = null;
   /**
    * @var string
    */
-  public $columnName = null;
+  public $fktable_name = null;
+  /**
+   * @var string
+   */
+  public $fkcolumn_name = null;
+  /**
+   * @var int
+   */
+  public $key_seq = null;
+  /**
+   * @var int
+   */
+  public $update_rule = null;
+  /**
+   * @var int
+   */
+  public $delete_rule = null;
+  /**
+   * @var string
+   */
+  public $fk_name = null;
+  /**
+   * @var string
+   */
+  public $pk_name = null;
+  /**
+   * @var bool
+   */
+  public $enable_cstr = null;
+  /**
+   * @var bool
+   */
+  public $validate_cstr = null;
+  /**
+   * @var bool
+   */
+  public $rely_cstr = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'objectType',
-          'type' => TType::I32,
+          'var' => 'pktable_db',
+          'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'dbName',
+          'var' => 'pktable_name',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'objectName',
+          'var' => 'pkcolumn_name',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'partValues',
-          'type' => TType::LST,
-          'etype' => TType::STRING,
-          'elem' => array(
-            'type' => TType::STRING,
-            ),
+          'var' => 'fktable_db',
+          'type' => TType::STRING,
           ),
         5 => array(
-          'var' => 'columnName',
+          'var' => 'fktable_name',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'fkcolumn_name',
+          'type' => TType::STRING,
+          ),
+        7 => array(
+          'var' => 'key_seq',
+          'type' => TType::I32,
+          ),
+        8 => array(
+          'var' => 'update_rule',
+          'type' => TType::I32,
+          ),
+        9 => array(
+          'var' => 'delete_rule',
+          'type' => TType::I32,
+          ),
+        10 => array(
+          'var' => 'fk_name',
+          'type' => TType::STRING,
+          ),
+        11 => array(
+          'var' => 'pk_name',
           'type' => TType::STRING,
           ),
+        12 => array(
+          'var' => 'enable_cstr',
+          'type' => TType::BOOL,
+          ),
+        13 => array(
+          'var' => 'validate_cstr',
+          'type' => TType::BOOL,
+          ),
+        14 => array(
+          'var' => 'rely_cstr',
+          'type' => TType::BOOL,
+          ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['objectType'])) {
-        $this->objectType = $vals['objectType'];
+      if (isset($vals['pktable_db'])) {
+        $this->pktable_db = $vals['pktable_db'];
       }
-      if (isset($vals['dbName'])) {
-        $this->dbName = $vals['dbName'];
+      if (isset($vals['pktable_name'])) {
+        $this->pktable_name = $vals['pktable_name'];
       }
-      if (isset($vals['objectName'])) {
-        $this->objectName = $vals['objectName'];
+      if (isset($vals['pkcolumn_name'])) {
+        $this->pkcolumn_name = $vals['pkcolumn_name'];
       }
-      if (isset($vals['partValues'])) {
-        $this->partValues = $vals['partValues'];
+      if (isset($vals['fktable_db'])) {
+        $this->fktable_db = $vals['fktable_db'];
       }
-      if (isset($vals['columnName'])) {
-        $this->columnName = $vals['columnName'];
+      if (isset($vals['fktable_name'])) {
+        $this->fktable_name = $vals['fktable_name'];
+      }
+      if (isset($vals['fkcolumn_name'])) {
+        $this->fkcolumn_name = $vals['fkcolumn_name'];
+      }
+      if (isset($vals['key_seq'])) {
+        $this->key_seq = $vals['key_seq'];
+      }
+      if (isset($vals['update_rule'])) {
+        $this->update_rule = $vals['update_rule'];
+      }
+      if (isset($vals['delete_rule'])) {
+        $this->delete_rule = $vals['delete_rule'];
+      }
+      if (isset($vals['fk_name'])) {
+        $this->fk_name = $vals['fk_name'];
+      }
+      if (isset($vals['pk_name'])) {
+        $this->pk_name = $vals['pk_name'];
+      }
+      if (isset($vals['enable_cstr'])) {
+        $this->enable_cstr = $vals['enable_cstr'];
+      }
+      if (isset($vals['validate_cstr'])) {
+        $this->validate_cstr = $vals['validate_cstr'];
+      }
+      if (isset($vals['rely_cstr'])) {
+        $this->rely_cstr = $vals['rely_cstr'];
       }
     }
   }
 
   public function getName() {
-    return 'HiveObjectRef';
+    return 'SQLForeignKey';
   }
 
   public function read($input)
@@ -633,98 +792,184 @@ class HiveObjectRef {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->objectType);
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->pktable_db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbName);
+            $xfer += $input->readString($this->pktable_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->objectName);
+            $xfer += $input->readString($this->pkcolumn_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
-          if ($ftype == TType::LST) {
-            $this->partValues = array();
-            $_size7 = 0;
-            $_etype10 = 0;
-            $xfer += $input->readListBegin($_etype10, $_size7);
-            for ($_i11 = 0; $_i11 < $_size7; ++$_i11)
-            {
-              $elem12 = null;
-              $xfer += $input->readString($elem12);
-              $this->partValues []= $elem12;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->fktable_db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 5:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->columnName);
+            $xfer += $input->readString($this->fktable_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        default:
-          $xfer += $input->skip($ftype);
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->fkcolumn_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
           break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
+        case 7:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->key_seq);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 8:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->update_rule);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 9:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->delete_rule);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 10:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->fk_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 11:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->pk_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 12:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->enable_cstr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 13:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->validate_cstr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 14:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->rely_cstr);
+          } 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('HiveObjectRef');
-    if ($this->objectType !== null) {
-      $xfer += $output->writeFieldBegin('objectType', TType::I32, 1);
-      $xfer += $output->writeI32($this->objectType);
+    $xfer += $output->writeStructBegin('SQLForeignKey');
+    if ($this->pktable_db !== null) {
+      $xfer += $output->writeFieldBegin('pktable_db', TType::STRING, 1);
+      $xfer += $output->writeString($this->pktable_db);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->dbName !== null) {
-      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 2);
-      $xfer += $output->writeString($this->dbName);
+    if ($this->pktable_name !== null) {
+      $xfer += $output->writeFieldBegin('pktable_name', TType::STRING, 2);
+      $xfer += $output->writeString($this->pktable_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->objectName !== null) {
-      $xfer += $output->writeFieldBegin('objectName', TType::STRING, 3);
-      $xfer += $output->writeString($this->objectName);
+    if ($this->pkcolumn_name !== null) {
+      $xfer += $output->writeFieldBegin('pkcolumn_name', TType::STRING, 3);
+      $xfer += $output->writeString($this->pkcolumn_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->partValues !== null) {
-      if (!is_array($this->partValues)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('partValues', TType::LST, 4);
-      {
-        $output->writeListBegin(TType::STRING, count($this->partValues));
-        {
-          foreach ($this->partValues as $iter13)
-          {
-            $xfer += $output->writeString($iter13);
-          }
-        }
-        $output->writeListEnd();
-      }
+    if ($this->fktable_db !== null) {
+      $xfer += $output->writeFieldBegin('fktable_db', TType::STRING, 4);
+      $xfer += $output->writeString($this->fktable_db);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->columnName !== null) {
-      $xfer += $output->writeFieldBegin('columnName', TType::STRING, 5);
-      $xfer += $output->writeString($this->columnName);
+    if ($this->fktable_name !== null) {
+      $xfer += $output->writeFieldBegin('fktable_name', TType::STRING, 5);
+      $xfer += $output->writeString($this->fktable_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->fkcolumn_name !== null) {
+      $xfer += $output->writeFieldBegin('fkcolumn_name', TType::STRING, 6);
+      $xfer += $output->writeString($this->fkcolumn_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->key_seq !== null) {
+      $xfer += $output->writeFieldBegin('key_seq', TType::I32, 7);
+      $xfer += $output->writeI32($this->key_seq);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->update_rule !== null) {
+      $xfer += $output->writeFieldBegin('update_rule', TType::I32, 8);
+      $xfer += $output->writeI32($this->update_rule);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->delete_rule !== null) {
+      $xfer += $output->writeFieldBegin('delete_rule', TType::I32, 9);
+      $xfer += $output->writeI32($this->delete_rule);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->fk_name !== null) {
+      $xfer += $output->writeFieldBegin('fk_name', TType::STRING, 10);
+      $xfer += $output->writeString($this->fk_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->pk_name !== null) {
+      $xfer += $output->writeFieldBegin('pk_name', TType::STRING, 11);
+      $xfer += $output->writeString($this->pk_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->enable_cstr !== null) {
+      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 12);
+      $xfer += $output->writeBool($this->enable_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->validate_cstr !== null) {
+      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 13);
+      $xfer += $output->writeBool($this->validate_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->rely_cstr !== null) {
+      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 14);
+      $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -734,76 +979,70 @@ class HiveObjectRef {
 
 }
 
-class PrivilegeGrantInfo {
+class Type {
   static $_TSPEC;
 
   /**
    * @var string
    */
-  public $privilege = null;
-  /**
-   * @var int
-   */
-  public $createTime = null;
+  public $name = null;
   /**
    * @var string
    */
-  public $grantor = null;
+  public $type1 = null;
   /**
-   * @var int
+   * @var string
    */
-  public $grantorType = null;
+  public $type2 = null;
   /**
-   * @var bool
+   * @var \metastore\FieldSchema[]
    */
-  public $grantOption = null;
+  public $fields = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'privilege',
+          'var' => 'name',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'createTime',
-          'type' => TType::I32,
+          'var' => 'type1',
+          'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'grantor',
+          'var' => 'type2',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'grantorType',
-          'type' => TType::I32,
-          ),
-        5 => array(
-          'var' => 'grantOption',
-          'type' => TType::BOOL,
+          'var' => 'fields',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\FieldSchema',
+            ),
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['privilege'])) {
-        $this->privilege = $vals['privilege'];
-      }
-      if (isset($vals['createTime'])) {
-        $this->createTime = $vals['createTime'];
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
       }
-      if (isset($vals['grantor'])) {
-        $this->grantor = $vals['grantor'];
+      if (isset($vals['type1'])) {
+        $this->type1 = $vals['type1'];
       }
-      if (isset($vals['grantorType'])) {
-        $this->grantorType = $vals['grantorType'];
+      if (isset($vals['type2'])) {
+        $this->type2 = $vals['type2'];
       }
-      if (isset($vals['grantOption'])) {
-        $this->grantOption = $vals['grantOption'];
+      if (isset($vals['fields'])) {
+        $this->fields = $vals['fields'];
       }
     }
   }
 
   public function getName() {
-    return 'PrivilegeGrantInfo';
+    return 'Type';
   }
 
   public function read($input)
@@ -823,35 +1062,39 @@ class PrivilegeGrantInfo {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->privilege);
+            $xfer += $input->readString($this->name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->createTime);
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->type1);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->grantor);
+            $xfer += $input->readString($this->type2);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->grantorType);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 5:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->grantOption);
+          if ($ftype == TType::LST) {
+            $this->fields = array();
+            $_size0 = 0;
+            $_etype3 = 0;
+            $xfer += $input->readListBegin($_etype3, $_size0);
+            for ($_i4 = 0; $_i4 < $_size0; ++$_i4)
+            {
+              $elem5 = null;
+              $elem5 = new \metastore\FieldSchema();
+              $xfer += $elem5->read($input);
+              $this->fields []= $elem5;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -868,30 +1111,37 @@ class PrivilegeGrantInfo {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('PrivilegeGrantInfo');
-    if ($this->privilege !== null) {
-      $xfer += $output->writeFieldBegin('privilege', TType::STRING, 1);
-      $xfer += $output->writeString($this->privilege);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->createTime !== null) {
-      $xfer += $output->writeFieldBegin('createTime', TType::I32, 2);
-      $xfer += $output->writeI32($this->createTime);
+    $xfer += $output->writeStructBegin('Type');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->grantor !== null) {
-      $xfer += $output->writeFieldBegin('grantor', TType::STRING, 3);
-      $xfer += $output->writeString($this->grantor);
+    if ($this->type1 !== null) {
+      $xfer += $output->writeFieldBegin('type1', TType::STRING, 2);
+      $xfer += $output->writeString($this->type1);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->grantorType !== null) {
-      $xfer += $output->writeFieldBegin('grantorType', TType::I32, 4);
-      $xfer += $output->writeI32($this->grantorType);
+    if ($this->type2 !== null) {
+      $xfer += $output->writeFieldBegin('type2', TType::STRING, 3);
+      $xfer += $output->writeString($this->type2);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->grantOption !== null) {
-      $xfer += $output->writeFieldBegin('grantOption', TType::BOOL, 5);
-      $xfer += $output->writeBool($this->grantOption);
+    if ($this->fields !== null) {
+      if (!is_array($this->fields)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fields', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->fields));
+        {
+          foreach ($this->fields as $iter6)
+          {
+            $xfer += $iter6->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -901,67 +1151,80 @@ class PrivilegeGrantInfo {
 
 }
 
-class HiveObjectPrivilege {
+class HiveObjectRef {
   static $_TSPEC;
 
   /**
-   * @var \metastore\HiveObjectRef
+   * @var int
    */
-  public $hiveObject = null;
+  public $objectType = null;
   /**
    * @var string
    */
-  public $principalName = null;
+  public $dbName = null;
   /**
-   * @var int
+   * @var string
    */
-  public $principalType = null;
+  public $objectName = null;
   /**
-   * @var \metastore\PrivilegeGrantInfo
+   * @var string[]
    */
-  public $grantInfo = null;
+  public $partValues = null;
+  /**
+   * @var string
+   */
+  public $columnName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'hiveObject',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\HiveObjectRef',
+          'var' => 'objectType',
+          'type' => TType::I32,
           ),
         2 => array(
-          'var' => 'principalName',
+          'var' => 'dbName',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'principalType',
-          'type' => TType::I32,
+          'var' => 'objectName',
+          'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'grantInfo',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\PrivilegeGrantInfo',
+          'var' => 'partValues',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        5 => array(
+          'var' => 'columnName',
+          'type' => TType::STRING,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['hiveObject'])) {
-        $this->hiveObject = $vals['hiveObject'];
+      if (isset($vals['objectType'])) {
+        $this->objectType = $vals['objectType'];
       }
-      if (isset($vals['principalName'])) {
-        $this->principalName = $vals['principalName'];
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
       }
-      if (isset($vals['principalType'])) {
-        $this->principalType = $vals['principalType'];
+      if (isset($vals['objectName'])) {
+        $this->objectName = $vals['objectName'];
       }
-      if (isset($vals['grantInfo'])) {
-        $this->grantInfo = $vals['grantInfo'];
+      if (isset($vals['partValues'])) {
+        $this->partValues = $vals['partValues'];
+      }
+      if (isset($vals['columnName'])) {
+        $this->columnName = $vals['columnName'];
       }
     }
   }
 
   public function getName() {
-    return 'HiveObjectPrivilege';
+    return 'HiveObjectRef';
   }
 
   public function read($input)
@@ -980,31 +1243,46 @@ class HiveObjectPrivilege {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->hiveObject = new \metastore\HiveObjectRef();
-            $xfer += $this->hiveObject->read($input);
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->objectType);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->principalName);
+            $xfer += $input->readString($this->dbName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->principalType);
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->objectName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
-          if ($ftype == TType::STRUCT) {
-            $this->grantInfo = new \metastore\PrivilegeGrantInfo();
-            $xfer += $this->grantInfo->read($input);
+          if ($ftype == TType::LST) {
+            $this->partValues = array();
+            $_size7 = 0;
+            $_etype10 = 0;
+            $xfer += $input->readListBegin($_etype10, $_size7);
+            for ($_i11 = 0; $_i11 < $_size7; ++$_i11)
+            {
+              $elem12 = null;
+              $xfer += $input->readString($elem12);
+              $this->partValues []= $elem12;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->columnName);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1021,31 +1299,42 @@ class HiveObjectPrivilege {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('HiveObjectPrivilege');
-    if ($this->hiveObject !== null) {
-      if (!is_object($this->hiveObject)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('hiveObject', TType::STRUCT, 1);
-      $xfer += $this->hiveObject->write($output);
+    $xfer += $output->writeStructBegin('HiveObjectRef');
+    if ($this->objectType !== null) {
+      $xfer += $output->writeFieldBegin('objectType', TType::I32, 1);
+      $xfer += $output->writeI32($this->objectType);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->principalName !== null) {
-      $xfer += $output->writeFieldBegin('principalName', TType::STRING, 2);
-      $xfer += $output->writeString($this->principalName);
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 2);
+      $xfer += $output->writeString($this->dbName);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->principalType !== null) {
-      $xfer += $output->writeFieldBegin('principalType', TType::I32, 3);
-      $xfer += $output->writeI32($this->principalType);
+    if ($this->objectName !== null) {
+      $xfer += $output->writeFieldBegin('objectName', TType::STRING, 3);
+      $xfer += $output->writeString($this->objectName);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->grantInfo !== null) {
-      if (!is_object($this->grantInfo)) {
+    if ($this->partValues !== null) {
+      if (!is_array($this->partValues)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('grantInfo', TType::STRUCT, 4);
-      $xfer += $this->grantInfo->write($output);
+      $xfer += $output->writeFieldBegin('partValues', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::STRING, count($this->partValues));
+        {
+          foreach ($this->partValues as $iter13)
+          {
+            $xfer += $output->writeString($iter13);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->columnName !== null) {
+      $xfer += $output->writeFieldBegin('columnName', TType::STRING, 5);
+      $xfer += $output->writeString($this->columnName);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -1055,37 +1344,76 @@ class HiveObjectPrivilege {
 
 }
 
-class PrivilegeBag {
+class PrivilegeGrantInfo {
   static $_TSPEC;
 
   /**
-   * @var \metastore\HiveObjectPrivilege[]
+   * @var string
    */
-  public $privileges = null;
+  public $privilege = null;
+  /**
+   * @var int
+   */
+  public $createTime = null;
+  /**
+   * @var string
+   */
+  public $grantor = null;
+  /**
+   * @var int
+   */
+  public $grantorType = null;
+  /**
+   * @var bool
+   */
+  public $grantOption = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'privileges',
-          'type' => TType::LST,
-          'etype' => TType::STRUCT,
-          'elem' => array(
-            'type' => TType::STRUCT,
-            'class' => '\metastore\HiveObjectPrivilege',
-            ),
+          'var' => 'privilege',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'createTime',
+          'type' => TType::I32,
+          ),
+        3 => array(
+          'var' => 'grantor',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'grantorType',
+          'type' => TType::I32,
+          ),
+        5 => array(
+          'var' => 'grantOption',
+          'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['privileges'])) {
-        $this->privileges = $vals['privileges'];
+      if (isset($vals['privilege'])) {
+        $this->privilege = $vals['privilege'];
+      }
+      if (isset($vals['createTime'])) {
+        $this->createTime = $vals['createTime'];
+      }
+      if (isset($vals['grantor'])) {
+        $this->grantor = $vals['grantor'];
+      }
+      if (isset($vals['grantorType'])) {
+        $this->grantorType = $vals['grantorType'];
+      }
+      if (isset($vals['grantOption'])) {
+        $this->grantOption = $vals['grantOption'];
       }
     }
   }
 
   public function getName() {
-    return 'PrivilegeBag';
+    return 'PrivilegeGrantInfo';
   }
 
   public function read($input)
@@ -1104,19 +1432,36 @@ class PrivilegeBag {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::LST) {
-            $this->privileges = array();
-            $_size14 = 0;
-            $_etype17 = 0;
-            $xfer += $input->readListBegin($_etype17, $_size14);
-            for ($_i18 = 0; $_i18 < $_size14; ++$_i18)
-            {
-              $elem19 = null;
-              $elem19 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem19->read($input);
-              $this->privileges []= $elem19;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->privilege);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->createTime);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->grantor);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->grantorType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->grantOption);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1133,118 +1478,100 @@ class PrivilegeBag {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('PrivilegeBag');
-    if ($this->privileges !== null) {
-      if (!is_array($this->privileges)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('privileges', TType::LST, 1);
-      {
-        $output->writeListBegin(TType::STRUCT, count($this->privileges));
-        {
-          foreach ($this->privileges as $iter20)
-          {
-            $xfer += $iter20->write($output);
-          }
-        }
-        $output->writeListEnd();
-      }
+    $xfer += $output->writeStructBegin('PrivilegeGrantInfo');
+    if ($this->privilege !== null) {
+      $xfer += $output->writeFieldBegin('privilege', TType::STRING, 1);
+      $xfer += $output->writeString($this->privilege);
       $xfer += $output->writeFieldEnd();
     }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class PrincipalPrivilegeSet {
+    if ($this->createTime !== null) {
+      $xfer += $output->writeFieldBegin('createTime', TType::I32, 2);
+      $xfer += $output->writeI32($this->createTime);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantor !== null) {
+      $xfer += $output->writeFieldBegin('grantor', TType::STRING, 3);
+      $xfer += $output->writeString($this->grantor);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantorType !== null) {
+      $xfer += $output->writeFieldBegin('grantorType', TType::I32, 4);
+      $xfer += $output->writeI32($this->grantorType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantOption !== null) {
+      $xfer += $output->writeFieldBegin('grantOption', TType::BOOL, 5);
+      $xfer += $output->writeBool($this->grantOption);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class HiveObjectPrivilege {
   static $_TSPEC;
 
   /**
-   * @var array
+   * @var \metastore\HiveObjectRef
    */
-  public $userPrivileges = null;
+  public $hiveObject = null;
   /**
-   * @var array
+   * @var string
    */
-  public $groupPrivileges = null;
+  public $principalName = null;
   /**
-   * @var array
+   * @var int
    */
-  public $rolePrivileges = null;
+  public $principalType = null;
+  /**
+   * @var \metastore\PrivilegeGrantInfo
+   */
+  public $grantInfo = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'userPrivileges',
-          'type' => TType::MAP,
-          'ktype' => TType::STRING,
-          'vtype' => TType::LST,
-          'key' => array(
-            'type' => TType::STRING,
-          ),
-          'val' => array(
-            'type' => TType::LST,
-            'etype' => TType::STRUCT,
-            'elem' => array(
-              'type' => TType::STRUCT,
-              'class' => '\metastore\PrivilegeGrantInfo',
-              ),
-            ),
+          'var' => 'hiveObject',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\HiveObjectRef',
           ),
         2 => array(
-          'var' => 'groupPrivileges',
-          'type' => TType::MAP,
-          'ktype' => TType::STRING,
-          'vtype' => TType::LST,
-          'key' => array(
-            'type' => TType::STRING,
-          ),
-          'val' => array(
-            'type' => TType::LST,
-            'etype' => TType::STRUCT,
-            'elem' => array(
-              'type' => TType::STRUCT,
-              'class' => '\metastore\PrivilegeGrantInfo',
-              ),
-            ),
+          'var' => 'principalName',
+          'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'rolePrivileges',
-          'type' => TType::MAP,
-          'ktype' => TType::STRING,
-          'vtype' => TType::LST,
-          'key' => array(
-            'type' => TType::STRING,
+          'var' => 'principalType',
+          'type' => TType::I32,
           ),
-          'val' => array(
-            'type' => TType::LST,
-            'etype' => TType::STRUCT,
-            'elem' => array(
-              'type' => TType::STRUCT,
-              'class' => '\metastore\PrivilegeGrantInfo',
-              ),
-            ),
+        4 => array(
+          'var' => 'grantInfo',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\PrivilegeGrantInfo',
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['userPrivileges'])) {
-        $this->userPrivileges = $vals['userPrivileges'];
+      if (isset($vals['hiveObject'])) {
+        $this->hiveObject = $vals['hiveObject'];
       }
-      if (isset($vals['groupPrivileges'])) {
-        $this->groupPrivileges = $vals['groupPrivileges'];
+      if (isset($vals['principalName'])) {
+        $this->principalName = $vals['principalName'];
       }
-      if (isset($vals['rolePrivileges'])) {
-        $this->rolePrivileges = $vals['rolePrivileges'];
+      if (isset($vals['principalType'])) {
+        $this->principalType = $vals['principalType'];
+      }
+      if (isset($vals['grantInfo'])) {
+        $this->grantInfo = $vals['grantInfo'];
       }
     }
   }
 
   public function getName() {
-    return 'PrincipalPrivilegeSet';
+    return 'HiveObjectPrivilege';
   }
 
   public function read($input)
@@ -1263,94 +1590,31 @@ class PrincipalPrivilegeSet {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::MAP) {
-            $this->userPrivileges = array();
-            $_size21 = 0;
-            $_ktype22 = 0;
-            $_vtype23 = 0;
-            $xfer += $input->readMapBegin($_ktype22, $_vtype23, $_size21);
-            for ($_i25 = 0; $_i25 < $_size21; ++$_i25)
-            {
-              $key26 = '';
-              $val27 = array();
-              $xfer += $input->readString($key26);
-              $val27 = array();
-              $_size28 = 0;
-              $_etype31 = 0;
-              $xfer += $input->readListBegin($_etype31, $_size28);
-              for ($_i32 = 0; $_i32 < $_size28; ++$_i32)
-              {
-                $elem33 = null;
-                $elem33 = new \metastore\PrivilegeGrantInfo();
-                $xfer += $elem33->read($input);
-                $val27 []= $elem33;
-              }
-              $xfer += $input->readListEnd();
-              $this->userPrivileges[$key26] = $val27;
-            }
-            $xfer += $input->readMapEnd();
+          if ($ftype == TType::STRUCT) {
+            $this->hiveObject = new \metastore\HiveObjectRef();
+            $xfer += $this->hiveObject->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::MAP) {
-            $this->groupPrivileges = array();
-            $_size34 = 0;
-            $_ktype35 = 0;
-            $_vtype36 = 0;
-            $xfer += $input->readMapBegin($_ktype35, $_vtype36, $_size34);
-            for ($_i38 = 0; $_i38 < $_size34; ++$_i38)
-            {
-              $key39 = '';
-              $val40 = array();
-              $xfer += $input->readString($key39);
-              $val40 = array();
-              $_size41 = 0;
-              $_etype44 = 0;
-              $xfer += $input->readListBegin($_etype44, $_size41);
-              for ($_i45 = 0; $_i45 < $_size41; ++$_i45)
-              {
-                $elem46 = null;
-                $elem46 = new \metastore\PrivilegeGrantInfo();
-                $xfer += $elem46->read($input);
-                $val40 []= $elem46;
-              }
-              $xfer += $input->readListEnd();
-              $this->groupPrivileges[$key39] = $val40;
-            }
-            $xfer += $input->readMapEnd();
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->principalName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
-          if ($ftype == TType::MAP) {
-            $this->rolePrivileges = array();
-            $_size47 = 0;
-            $_ktype48 = 0;
-            $_vtype49 = 0;
-            $xfer += $input->readMapBegin($_ktype48, $_vtype49, $_size47);
-            for ($_i51 = 0; $_i51 < $_size47; ++$_i51)
-            {
-              $key52 = '';
-              $val53 = array();
-              $xfer += $input->readString($key52);
-              $val53 = array();
-              $_size54 = 0;
-              $_etype57 = 0;
-              $xfer += $input->readListBegin($_etype57, $_size54);
-              for ($_i58 = 0; $_i58 < $_size54; ++$_i58)
-              {
-                $elem59 = null;
-                $elem59 = new \metastore\PrivilegeGrantInfo();
-                $xfer += $elem59->read($input);
-                $val53 []= $elem59;
-              }
-              $xfer += $input->readListEnd();
-              $this->rolePrivileges[$key52] = $val53;
-            }
-            $xfer += $input->readMapEnd();
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->principalType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRUCT) {
+            $this->grantInfo = new \metastore\PrivilegeGrantInfo();
+            $xfer += $this->grantInfo->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1367,86 +1631,31 @@ class PrincipalPrivilegeSet {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('PrincipalPrivilegeSet');
-    if ($this->userPrivileges !== null) {
-      if (!is_array($this->userPrivileges)) {
+    $xfer += $output->writeStructBegin('HiveObjectPrivilege');
+    if ($this->hiveObject !== null) {
+      if (!is_object($this->hiveObject)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('userPrivileges', TType::MAP, 1);
-      {
-        $output->writeMapBegin(TType::STRING, TType::LST, count($this->userPrivileges));
-        {
-          foreach ($this->userPrivileges as $kiter60 => $viter61)
-          {
-            $xfer += $output->writeString($kiter60);
-            {
-              $output->writeListBegin(TType::STRUCT, count($viter61));
-              {
-                foreach ($viter61 as $iter62)
-                {
-                  $xfer += $iter62->write($output);
-                }
-              }
-              $output->writeListEnd();
-            }
-          }
-        }
-        $output->writeMapEnd();
-      }
+      $xfer += $output->writeFieldBegin('hiveObject', TType::STRUCT, 1);
+      $xfer += $this->hiveObject->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->groupPrivileges !== null) {
-      if (!is_array($this->groupPrivileges)) {
+    if ($this->principalName !== null) {
+      $xfer += $output->writeFieldBegin('principalName', TType::STRING, 2);
+      $xfer += $output->writeString($this->principalName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->principalType !== null) {
+      $xfer += $output->writeFieldBegin('principalType', TType::I32, 3);
+      $xfer += $output->writeI32($this->principalType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantInfo !== null) {
+      if (!is_object($this->grantInfo)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('groupPrivileges', TType::MAP, 2);
-      {
-        $output->writeMapBegin(TType::STRING, TType::LST, count($this->groupPrivileges));
-        {
-          foreach ($this->groupPrivileges as $kiter63 => $viter64)
-          {
-            $xfer += $output->writeString($kiter63);
-            {
-              $output->writeListBegin(TType::STRUCT, count($viter64));
-              {
-                foreach ($viter64 as $iter65)
-                {
-                  $xfer += $iter65->write($output);
-                }
-              }
-              $output->writeListEnd();
-            }
-          }
-        }
-        $output->writeMapEnd();
-      }
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->rolePrivileges !== null) {
-      if (!is_array($this->rolePrivileges)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('rolePrivileges', TType::MAP, 3);
-      {
-        $output->writeMapBegin(TType::STRING, TType::LST, count($this->rolePrivileges));
-        {
-          foreach ($this->rolePrivileges as $kiter66 => $viter67)
-          {
-            $xfer += $output->writeString($kiter66);
-            {
-              $output->writeListBegin(TType::STRUCT, count($viter67));
-              {
-                foreach ($viter67 as $iter68)
-                {
-                  $xfer += $iter68->write($output);
-                }
-              }
-              $output->writeListEnd();
-            }
-          }
-        }
-        $output->writeMapEnd();
-      }
+      $xfer += $output->writeFieldBegin('grantInfo', TType::STRUCT, 4);
+      $xfer += $this->grantInfo->write($output);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -1456,55 +1665,37 @@ class PrincipalPrivilegeSet {
 
 }
 
-class GrantRevokePrivilegeRequest {
+class PrivilegeBag {
   static $_TSPEC;
 
   /**
-   * @var int
-   */
-  public $requestType = null;
-  /**
-   * @var \metastore\PrivilegeBag
+   * @var \metastore\HiveObjectPrivilege[]
    */
   public $privileges = null;
-  /**
-   * @var bool
-   */
-  public $revokeGrantOption = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'requestType',
-          'type' => TType::I32,
-          ),
-        2 => array(
           'var' => 'privileges',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\PrivilegeBag',
-          ),
-        3 => array(
-          'var' => 'revokeGrantOption',
-          'type' => TType::BOOL,
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\HiveObjectPrivilege',
+            ),
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['requestType'])) {
-        $this->requestType = $vals['requestType'];
-      }
       if (isset($vals['privileges'])) {
         $this->privileges = $vals['privileges'];
       }
-      if (isset($vals['revokeGrantOption'])) {
-        $this->revokeGrantOption = $vals['revokeGrantOption'];
-      }
     }
   }
 
   public function getName() {
-    return 'GrantRevokePrivilegeRequest';
+    return 'PrivilegeBag';
   }
 
   public function read($input)
@@ -1523,23 +1714,19 @@ class GrantRevokePrivilegeRequest {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->requestType);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::STRUCT) {
-            $this->privileges = new \metastore\PrivilegeBag();
-            $xfer += $this->privileges->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 3:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->revokeGrantOption);
+          if ($ftype == TType::LST) {
+            $this->privileges = array();
+            $_size14 = 0;
+            $_etype17 = 0;
+            $xfer += $input->readListBegin($_etype17, $_size14);
+            for ($_i18 = 0; $_i18 < $_size14; ++$_i18)
+            {
+              $elem19 = null;
+              $elem19 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem19->read($input);
+              $this->privileges []= $elem19;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1556,23 +1743,22 @@ class GrantRevokePrivilegeRequest {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('GrantRevokePrivilegeRequest');
-    if ($this->requestType !== null) {
-      $xfer += $output->writeFieldBegin('requestType', TType::I32, 1);
-      $xfer += $output->writeI32($this->requestType);
-      $xfer += $output->writeFieldEnd();
-    }
+    $xfer += $output->writeStructBegin('PrivilegeBag');
     if ($this->privileges !== null) {
-      if (!is_object($this->privileges)) {
+      if (!is_array($this->privileges)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 2);
-      $xfer += $this->privileges->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->revokeGrantOption !== null) {
-      $xfer += $output->writeFieldBegin('revokeGrantOption', TType::BOOL, 3);
-      $xfer += $output->writeBool($this->revokeGrantOption);
+      $xfer += $output->writeFieldBegin('privileges', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->privileges));
+        {
+          foreach ($this->privileges as $iter20)
+          {
+            $xfer += $iter20->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -1582,32 +1768,93 @@ class GrantRevokePrivilegeRequest {
 
 }
 
-class GrantRevokePrivilegeResponse {
+class PrincipalPrivilegeSet {
   static $_TSPEC;
 
   /**
-   * @var bool
+   * @var array
    */
-  public $success = null;
+  public $userPrivileges = null;
+  /**
+   * @var array
+   */
+  public $groupPrivileges = null;
+  /**
+   * @var array
+   */
+  public $rolePrivileges = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'success',
-          'type' => TType::BOOL,
+          'var' => 'userPrivileges',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::LST,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::LST,
+            'etype' => TType::STRUCT,
+            'elem' => array(
+              'type' => TType::STRUCT,
+              'class' => '\metastore\PrivilegeGrantInfo',
+              ),
+            ),
+          ),
+        2 => array(
+          'var' => 'groupPrivileges',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::LST,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::LST,
+            'etype' => TType::STRUCT,
+            'elem' => array(
+              'type' => TType::STRUCT,
+              'class' => '\metastore\PrivilegeGrantInfo',
+              ),
+            ),
+          ),
+        3 => array(
+          'var' => 'rolePrivileges',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::LST,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::LST,
+            'etype' => TType::STRUCT,
+            'elem' => array(
+              'type' => TType::STRUCT,
+              'class' => '\metastore\PrivilegeGrantInfo',
+              ),
+            ),
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
+      if (isset($vals['userPrivileges'])) {
+        $this->userPrivileges = $vals['userPrivileges'];
+      }
+      if (isset($vals['groupPrivileges'])) {
+        $this->groupPrivileges = $vals['groupPrivileges'];
+      }
+      if (isset($vals['rolePrivileges'])) {
+        $this->rolePrivileges = $vals['rolePrivileges'];
       }
     }
   }
 
   public function getName() {
-    return 'GrantRevokePrivilegeResponse';
+    return 'PrincipalPrivilegeSet';
   }
 
   public function read($input)
@@ -1626,28 +1873,190 @@ class GrantRevokePrivilegeResponse {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->success);
+          if ($ftype == TType::MAP) {
+            $this->userPrivileges = array();
+            $_size21 = 0;
+            $_ktype22 = 0;
+            $_vtype23 = 0;
+            $xfer += $input->readMapBegin($_ktype22, $_vtype23, $_size21);
+            for ($_i25 = 0; $_i25 < $_size21; ++$_i25)
+            {
+              $key26 = '';
+              $val27 = array();
+              $xfer += $input->readString($key26);
+              $val27 = array();
+              $_size28 = 0;
+              $_etype31 = 0;
+              $xfer += $input->readListBegin($_etype31, $_size28);
+              for ($_i32 = 0; $_i32 < $_size28; ++$_i32)
+              {
+                $elem33 = null;
+                $elem33 = new \metastore\PrivilegeGrantInfo();
+                $xfer += $elem33->read($input);
+                $val27 []= $elem33;
+              }
+              $xfer += $input->readListEnd();
+              $this->userPrivileges[$key26] = $val27;
+            }
+            $xfer += $input->readMapEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
+        case 2:
+          if ($ftype == TType::MAP) {
+            $this->groupPrivileges = array();
+            $_size34 = 0;
+            $_ktype35 = 0;
+            $_vtype36 = 0;
+            $xfer += $input->readMapBegin($_ktype35, $_vtype36, $_size34);
+            for ($_i38 = 0; $_i38 < $_size34; ++$_i38)
+            {
+              $key39 = '';
+              $val40 = array();
+              $xfer += $input->readString($key39);
+              $val40 = array();
+              $_size41 = 0;
+              $_etype44 = 0;
+              $xfer += $input->readListBegin($_etype44, $_size41);
+              for ($_i45 = 0; $_i45 < $_size41; ++$_i45)
+              {
+                $elem46 = null;
+                $elem46 = new \metastore\PrivilegeGrantInfo();
+                $xfer += $elem46->read($input);
+                $val40 []= $elem46;
+              }
+              $xfer += $input->readListEnd();
+              $this->groupPrivileges[$key39] = $val40;
+            }
+            $xfer += $input->readMapEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::MAP) {
+            $this->rolePrivileges = array();
+            $_size47 = 0;
+            $_ktype48 = 0;
+            $_vtype49 = 0;
+            $xfer += $input->readMapBegin($_ktype48, $_vtype49, $_size47);
+            for ($_i51 = 0; $_i51 < $_size47; ++$_i51)
+            {
+              $key52 = '';
+              $val53 = array();
+              $xfer += $input->readString($key52);
+              $val53 = array();
+              $_size54 = 0;
+              $_etype57 = 0;
+              $xfer += $input->readListBegin($_etype57, $_size54);
+              for ($_i58 = 0; $_i58 < $_size54; ++$_i58)
+              {
+                $elem59 = null;
+                $elem59 = new \metastore\PrivilegeGrantInfo();
+                $xfer += $elem59->read($input);
+                $val53 []= $elem59;
+              }
+              $xfer += $input->readListEnd();
+              $this->rolePrivileges[$key52] = $val53;
+            }
+            $xfer += $input->readMapEnd();
+          } 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('GrantRevokePrivilegeResponse');
-    if ($this->success !== null) {
-      $xfer += $output->writeFieldBegin('success', TType::BOOL, 1);
-      $xfer += $output->writeBool($this->success);
+    $xfer += $output->writeStructBegin('PrincipalPrivilegeSet');
+    if ($this->userPrivileges !== null) {
+      if (!is_array($this->userPrivileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('userPrivileges', TType::MAP, 1);
+      {
+        $output->writeMapBegin(TType::STRING, TType::LST, count($this->userPrivileges));
+        {
+          foreach ($this->userPrivileges as $kiter60 => $viter61)
+          {
+            $xfer += $output->writeString($kiter60);
+            {
+              $output->writeListBegin(TType::STRUCT, count($viter61));
+              {
+                foreach ($viter61 as $iter62)
+                {
+                  $xfer += $iter62->write($output);
+                }
+              }
+              $output->writeListEnd();
+            }
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->groupPrivileges !== null) {
+      if (!is_array($this->groupPrivileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('groupPrivileges', TType::MAP, 2);
+      {
+        $output->writeMapBegin(TType::STRING, TType::LST, count($this->groupPrivileges));
+        {
+          foreach ($this->groupPrivileges as $kiter63 => $viter64)
+          {
+            $xfer += $output->writeString($kiter63);
+            {
+              $output->writeListBegin(TType::STRUCT, count($viter64));
+              {
+                foreach ($viter64 as $iter65)
+                {
+                  $xfer += $iter65->write($output);
+                }
+              }
+              $output->writeListEnd();
+            }
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->rolePrivileges !== null) {
+      if (!is_array($this->rolePrivileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('rolePrivileges', TType::MAP, 3);
+      {
+        $output->writeMapBegin(TType::STRING, TType::LST, count($this->rolePrivileges));
+        {
+          foreach ($this->rolePrivileges as $kiter66 => $viter67)
+          {
+            $xfer += $output->writeString($kiter66);
+            {
+              $output->writeListBegin(TType::STRUCT, count($viter67));
+              {
+                foreach ($viter67 as $iter68)
+                {
+                  $xfer += $iter68->write($output);
+                }
+              }
+              $output->writeListEnd();
+            }
+          }
+        }
+        $output->writeMapEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -1657,54 +2066,55 @@ class GrantRevokePrivilegeResponse {
 
 }
 
-class Role {
+class GrantRevokePrivilegeRequest {
   static $_TSPEC;
 
   /**
-   * @var string
+   * @var int
    */
-  public $roleName = null;
+  public $requestType = null;
   /**
-   * @var int
+   * @var \metastore\PrivilegeBag
    */
-  public $createTime = null;
+  public $privileges = null;
   /**
-   * @var string
+   * @var bool
    */
-  public $ownerName = null;
+  public $revokeGrantOption = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'roleName',
-          'type' => TType::STRING,
+          'var' => 'requestType',
+          'type' => TType::I32,
           ),
         2 => array(
-          'var' => 'createTime',
-          'type' => TType::I32,
+          'var' => 'privileges',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\PrivilegeBag',
           ),
         3 => array(
-          'var' => 'ownerName',
-          'type' => TType::STRING,
+          'var' => 'revokeGrantOption',
+          'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['roleName'])) {
-        $this->roleName = $vals['roleName'];
+      if (isset($vals['requestType'])) {
+        $this->requestType = $vals['requestType'];
       }
-      if (isset($vals['createTime'])) {
-        $this->createTime = $vals['createTime'];
+      if (isset($vals['privileges'])) {
+        $this->privileges = $vals['privileges'];
       }
-      if (isset($vals['ownerName'])) {
-        $this->ownerName = $vals['ownerName'];
+      if (isset($vals['revokeGrantOption'])) {
+        $this->revokeGrantOption = $vals['revokeGrantOption'];
       }
     }
   }
 
   public function getName() {
-    return 'Role';
+    return 'GrantRevokePrivilegeRequest';
   }
 
   public function read($input)
@@ -1723,22 +2133,23 @@ class Role {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->roleName);
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->requestType);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->createTime);
+          if ($ftype == TType::STRUCT) {
+            $this->privileges = new \metastore\PrivilegeBag();
+            $xfer += $this->privileges->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->ownerName);
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->revokeGrantOption);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1755,20 +2166,23 @@ class Role {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('Role');
-    if ($this->roleName !== null) {
-      $xfer += $output->writeFieldBegin('roleName', TType::STRING, 1);
-      $xfer += $output->writeString($this->roleName);
+    $xfer += $output->writeStructBegin('GrantRevokePrivilegeRequest');
+    if ($this->requestType !== null) {
+      $xfer += $output->writeFieldBegin('requestType', TType::I32, 1);
+      $xfer += $output->writeI32($this->requestType);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->createTime !== null) {
-      $xfer += $output->writeFieldBegin('createTime', TType::I32, 2);
-      $xfer += $output->writeI32($this->createTime);
+    if ($this->privileges !== null) {
+      if (!is_object($this->privileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 2);
+      $xfer += $this->privileges->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->ownerName !== null) {
-      $xfer += $output->writeFieldBegin('ownerName', TType::STRING, 3);
-      $xfer += $output->writeString($this->ownerName);
+    if ($this->revokeGrantOption !== null) {
+      $xfer += $output->writeFieldBegin('revokeGrantOption', TType::BOOL, 3);
+      $xfer += $output->writeBool($this->revokeGrantOption);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -1778,32 +2192,228 @@ class Role {
 
 }
 
-class RolePrincipalGrant {
+class GrantRevokePrivilegeResponse {
   static $_TSPEC;
 
   /**
-   * @var string
-   */
-  public $roleName = null;
-  /**
-   * @var string
-   */
-  public $principalName = null;
-  /**
-   * @var int
-   */
-  public $principalType = null;
-  /**
    * @var bool
    */
-  public $grantOption = null;
-  /**
-   * @var int
-   */
-  public $grantTime = null;
-  /**
-   * @var string
-   */
+  public $success = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'success',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GrantRevokePrivilegeResponse';
+  }
+
+  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::BOOL) {
+            $xfer += $input->readBool($this->success);
+          } 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('GrantRevokePrivilegeResponse');
+    if ($this->success !== null) {
+      $xfer += $output->writeFieldBegin('success', TType::BOOL, 1);
+      $xfer += $output->writeBool($this->success);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class Role {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $roleName = null;
+  /**
+   * @var int
+   */
+  public $createTime = null;
+  /**
+   * @var string
+   */
+  public $ownerName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'roleName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'createTime',
+          'type' => TType::I32,
+          ),
+        3 => array(
+          'var' => 'ownerName',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['roleName'])) {
+        $this->roleName = $vals['roleName'];
+      }
+      if (isset($vals['createTime'])) {
+        $this->createTime = $vals['createTime'];
+      }
+      if (isset($vals['ownerName'])) {
+        $this->ownerName = $vals['ownerName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'Role';
+  }
+
+  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->roleName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->createTime);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->ownerName);
+          } 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('Role');
+    if ($this->roleName !== null) {
+      $xfer += $output->writeFieldBegin('roleName', TType::STRING, 1);
+      $xfer += $output->writeString($this->roleName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->createTime !== null) {
+      $xfer += $output->writeFieldBegin('createTime', TType::I32, 2);
+      $xfer += $output->writeI32($this->createTime);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->ownerName !== null) {
+      $xfer += $output->writeFieldBegin('ownerName', TType::STRING, 3);
+      $xfer += $output->writeString($this->ownerName);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class RolePrincipalGrant {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $roleName = null;
+  /**
+   * @var string
+   */
+  public $principalName = null;
+  /**
+   * @var int
+   */
+  public $principalType = null;
+  /**
+   * @var bool
+   */
+  public $grantOption = null;
+  /**
+   * @var int
+   */
+  public $grantTime = null;
+  /**
+   * @var string
+   */
   public $grantorName = null;
   /**
    * @var int
@@ -5078,32 +5688,530 @@ class PartitionListComposingSpec {
           $xfer += $input->skip($ftype);
           break;
       }
-      $xfer += $input->readFieldEnd();
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('PartitionListComposingSpec');
+    if ($this->partitions !== null) {
+      if (!is_array($this->partitions)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('partitions', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->partitions));
+        {
+          foreach ($this->partitions as $iter229)
+          {
+            $xfer += $iter229->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class PartitionSpec {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tableName = null;
+  /**
+   * @var string
+   */
+  public $rootPath = null;
+  /**
+   * @var \metastore\PartitionSpecWithSharedSD
+   */
+  public $sharedSDPartitionSpec = null;
+  /**
+   * @var \metastore\PartitionListComposingSpec
+   */
+  public $partitionList = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tableName',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'rootPath',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'sharedSDPartitionSpec',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\PartitionSpecWithSharedSD',
+          ),
+        5 => array(
+          'var' => 'partitionList',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\PartitionListComposingSpec',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['tableName'])) {
+        $this->tableName = $vals['tableName'];
+      }
+      if (isset($vals['rootPath'])) {
+        $this->rootPath = $vals['rootPath'];
+      }
+      if (isset($vals['sharedSDPartitionSpec'])) {
+        $this->sharedSDPartitionSpec = $vals['sharedSDPartitionSpec'];
+      }
+      if (isset($vals['partitionList'])) {
+        $this->partitionList = $vals['partitionList'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'PartitionSpec';
+  }
+
+  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->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->rootPath);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRUCT) {
+            $this->sharedSDPartitionSpec = new \metastore\PartitionSpecWithSharedSD();
+            $xfer += $this->sharedSDPartitionSpec->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRUCT) {
+            $this->partitionList = new \metastore\PartitionListComposingSpec();
+            $xfer += $this->partitionList->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('PartitionSpec');
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableName !== null) {
+      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2);
+      $xfer += $output->writeString($this->tableName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->rootPath !== null) {
+      $xfer += $output->writeFieldBegin('rootPath', TType::STRING, 3);
+      $xfer += $output->writeString($this->rootPath);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->sharedSDPartitionSpec !== null) {
+      if (!is_object($this->sharedSDPartitionSpec)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('sharedSDPartitionSpec', TType::STRUCT, 4);
+      $xfer += $this->sharedSDPartitionSpec->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->partitionList !== null) {
+      if (!is_object($this->partitionList)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('partitionList', TType::STRUCT, 5);
+      $xfer += $this->partitionList->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class Index {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $indexName = null;
+  /**
+   * @var string
+   */
+  public $indexHandlerClass = null;
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $origTableName = null;
+  /**
+   * @var int
+   */
+  public $createTime = null;
+  /**
+   * @var int
+   */
+  public $lastAccessTime = null;
+  /**
+   * @var string
+   */
+  public $indexTableName = null;
+  /**
+   * @var \metastore\StorageDescriptor
+   */
+  public $sd = null;
+  /**
+   * @var array
+   */
+  public $parameters = null;
+  /**
+   * @var bool
+   */
+  public $deferredRebuild = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'indexName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'indexHandlerClass',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'origTableName',
+          'type' => TType::STRING,
+          ),
+        5 => array(
+          'var' => 'createTime',
+          'type' => TType::I32,
+          ),
+        6 => array(
+          'var' => 'lastAccessTime',
+          'type' => TType::I32,
+          ),
+        7 => array(
+          'var' => 'indexTableName',
+          'type' => TType::STRING,
+          ),
+        8 => array(
+          'var' => 'sd',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\StorageDescriptor',
+          ),
+        9 => array(
+          'var' => 'parameters',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::STRING,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        10 => array(
+          'var' => 'deferredRebuild',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['indexName'])) {
+        $this->indexName = $vals['indexName'];
+      }
+      if (isset($vals['indexHandlerClass'])) {
+        $this->indexHandlerClass = $vals['indexHandlerClass'];
+      }
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['origTableName'])) {
+        $this->origTableName = $vals['origTableName'];
+      }
+      if (isset($vals['createTime'])) {
+        $this->createTime = $vals['createTime'];
+      }
+      if (isset($vals['lastAccessTime'])) {
+        $this->lastAccessTime = $vals['lastAccessTime'];
+      }
+      if (isset($vals['indexTableName'])) {
+        $this->indexTableName = $vals['indexTableName'];
+      }
+      if (isset($vals['sd'])) {
+        $this->sd = $vals['sd'];
+      }
+      if (isset($vals['parameters'])) {
+        $this->parameters = $vals['parameters'];
+      }
+      if (isset($vals['deferredRebuild'])) {
+        $this->deferredRebuild = $vals['deferredRebuild'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'Index';
+  }
+
+  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->indexName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->indexHandlerClass);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case

<TRUNCATED>

[08/30] hive git commit: HIVE-13500 : Fix OOM with explan output being logged (Rajat Khandelwal, reviewed by Amareshwari)

Posted by jd...@apache.org.
HIVE-13500 : Fix OOM with explan output being logged (Rajat Khandelwal, reviewed by Amareshwari)


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

Branch: refs/heads/llap
Commit: 68dc9ddd3e304ffc523e92d51990b15a2884ba10
Parents: ff6ddfb
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Sun Apr 17 06:46:30 2016 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Sun Apr 17 06:46:30 2016 +0530

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/Driver.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/68dc9ddd/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 6062ac0..65744ac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -520,8 +520,7 @@ public class Driver implements CommandProcessor {
         }
       }
 
-      if (conf.getBoolVar(ConfVars.HIVE_LOG_EXPLAIN_OUTPUT) ||
-           conf.isWebUiQueryInfoCacheEnabled()) {
+      if (conf.getBoolVar(ConfVars.HIVE_LOG_EXPLAIN_OUTPUT)) {
         String explainOutput = getExplainOutput(sem, plan, tree);
         if (explainOutput != null) {
           if (conf.getBoolVar(ConfVars.HIVE_LOG_EXPLAIN_OUTPUT)) {


[30/30] hive git commit: Merge remote-tracking branch 'origin/master' into llap

Posted by jd...@apache.org.
Merge remote-tracking branch 'origin/master' into llap


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

Branch: refs/heads/llap
Commit: e69bd1ee53ba8f7aef8614d1cd3fb339e07db984
Parents: 5816ff3 9b5eb45
Author: Jason Dere <jd...@hortonworks.com>
Authored: Wed Apr 20 13:04:26 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Wed Apr 20 13:04:26 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |   85 +-
 .../test/resources/testconfiguration.properties |    3 +-
 .../apache/hive/jdbc/HiveQueryResultSet.java    |    9 +
 .../org/apache/hive/jdbc/HiveStatement.java     |   75 +-
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   45 +-
 metastore/if/hive_metastore.thrift              |   59 +
 .../upgrade/derby/034-HIVE-13076.derby.sql      |    3 +
 .../upgrade/derby/hive-schema-2.1.0.derby.sql   |    6 +
 .../derby/upgrade-2.0.0-to-2.1.0.derby.sql      |    1 +
 .../upgrade/mssql/019-HIVE-13076.mssql.sql      |   15 +
 .../upgrade/mssql/hive-schema-2.1.0.mssql.sql   |   18 +
 .../mssql/upgrade-2.0.0-to-2.1.0.mssql.sql      |    1 +
 .../upgrade/mysql/034-HIVE-13076.mysql.sql      |   17 +
 .../upgrade/mysql/hive-schema-2.1.0.mysql.sql   |   17 +-
 .../mysql/upgrade-2.0.0-to-2.1.0.mysql.sql      |    1 +
 .../upgrade/oracle/034-HIVE-13076.oracle.sql    |   15 +
 .../upgrade/oracle/hive-schema-2.1.0.oracle.sql |   19 +
 .../oracle/upgrade-2.0.0-to-2.1.0.oracle.sql    |    1 +
 .../postgres/033-HIVE-13076.postgres.sql        |   15 +
 .../postgres/hive-schema-2.1.0.postgres.sql     |   17 +
 .../upgrade-2.0.0-to-2.1.0.postgres.sql         |    1 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 4885 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  447 ++
 .../ThriftHiveMetastore_server.skeleton.cpp     |   15 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 6512 +++++++++-------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  404 +
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../metastore/api/AddPartitionsRequest.java     |   36 +-
 .../hive/metastore/api/AddPartitionsResult.java |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../metastore/api/DropPartitionsResult.java     |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../hive/metastore/api/ForeignKeysRequest.java  |  692 ++
 .../hive/metastore/api/ForeignKeysResponse.java |  443 ++
 .../hadoop/hive/metastore/api/Function.java     |   36 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../metastore/api/GetOpenTxnsInfoResponse.java  |   36 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |   32 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |   32 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |   32 +-
 .../metastore/api/PartitionsByExprResult.java   |   36 +-
 .../metastore/api/PartitionsStatsRequest.java   |   64 +-
 .../metastore/api/PartitionsStatsResult.java    |   76 +-
 .../hive/metastore/api/PrimaryKeysRequest.java  |  490 ++
 .../hive/metastore/api/PrimaryKeysResponse.java |  443 ++
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../hive/metastore/api/RequestPartsSpec.java    |   68 +-
 .../hive/metastore/api/SQLForeignKey.java       | 1715 +++++
 .../hive/metastore/api/SQLPrimaryKey.java       | 1103 +++
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/TableStatsRequest.java   |   32 +-
 .../hive/metastore/api/TableStatsResult.java    |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 7122 +++++++++++++-----
 .../gen-php/metastore/ThriftHiveMetastore.php   | 2260 ++++--
 .../src/gen/thrift/gen-php/metastore/Types.php  | 5564 ++++++++------
 .../hive_metastore/ThriftHiveMetastore-remote   |   21 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1890 +++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 1440 +++-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  152 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |  209 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   97 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   46 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   16 +
 .../hive/metastore/MetaStoreDirectSql.java      |  134 +
 .../hadoop/hive/metastore/ObjectStore.java      |  397 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   12 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   24 +
 .../hive/metastore/model/MConstraint.java       |  148 +
 metastore/src/model/package.jdo                 |   33 +
 .../DummyRawStoreControlledCommit.java          |   24 +
 .../DummyRawStoreForJdoConnection.java          |   24 +
 orc/src/java/org/apache/orc/DataReader.java     |    6 +-
 .../java/org/apache/orc/DataReaderFactory.java  |    9 +
 .../org/apache/orc/MetadataReaderFactory.java   |   12 +
 orc/src/java/org/apache/orc/OrcUtils.java       |   75 -
 orc/src/java/org/apache/orc/Reader.java         |    6 -
 orc/src/java/org/apache/orc/RecordReader.java   |    8 +-
 .../java/org/apache/orc/TypeDescription.java    |   62 +-
 .../org/apache/orc/impl/BitFieldReader.java     |    5 +-
 .../apache/orc/impl/DataReaderProperties.java   |   84 +
 .../orc/impl/DefaultMetadataReaderFactory.java  |   14 +
 .../java/org/apache/orc/impl/IntegerReader.java |   26 +-
 .../org/apache/orc/impl/MetadataReader.java     |    5 +-
 .../org/apache/orc/impl/MetadataReaderImpl.java |   17 +-
 .../orc/impl/MetadataReaderProperties.java      |   96 +
 .../apache/orc/impl/RunLengthByteReader.java    |   36 +-
 .../apache/orc/impl/RunLengthIntegerReader.java |   31 +-
 .../orc/impl/RunLengthIntegerReaderV2.java      |   33 +-
 .../java/org/apache/orc/impl/WriterImpl.java    |   47 +-
 .../orc/impl/TestDataReaderProperties.java      |   69 +
 .../orc/impl/TestMetadataReaderProperties.java  |   72 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   42 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |   13 +-
 .../ql/io/orc/DefaultDataReaderFactory.java     |   14 +
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   43 +-
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |    3 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   50 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |  203 +-
 .../hive/ql/io/orc/RecordReaderUtils.java       |   17 +-
 .../hadoop/hive/ql/io/orc/SchemaEvolution.java  |  234 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |  838 ++-
 .../ql/io/orc/VectorizedOrcInputFormat.java     |   32 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |    2 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    7 +-
 .../hadoop/hive/ql/session/SessionState.java    |    2 +-
 .../hive/ql/io/orc/TestRecordReaderImpl.java    |  145 +-
 .../hive/ql/io/orc/TestTypeDescription.java     |    4 +-
 .../hive/ql/io/orc/TestVectorOrcFile.java       | 1634 ++--
 .../hive/ql/io/orc/TestVectorizedORCReader.java |    7 +-
 .../queries/clientpositive/encryption_ctas.q    |   16 +
 .../encrypted/encryption_ctas.q.out             |   56 +
 .../gen-py/hive_service/ThriftHive-remote       |   21 +
 .../cli/operation/GetCatalogsOperation.java     |    5 +-
 .../cli/operation/GetColumnsOperation.java      |    5 +-
 .../cli/operation/GetFunctionsOperation.java    |    6 +-
 .../cli/operation/GetSchemasOperation.java      |    7 +-
 .../cli/operation/GetTableTypesOperation.java   |    7 +-
 .../cli/operation/GetTablesOperation.java       |   10 +-
 .../cli/operation/GetTypeInfoOperation.java     |    7 +-
 .../hive/service/cli/operation/Operation.java   |    8 +-
 .../service/cli/operation/SQLOperation.java     |    7 +-
 .../hive/ql/exec/vector/BytesColumnVector.java  |   11 -
 .../ql/exec/vector/TimestampColumnVector.java   |    2 +-
 .../hive/ql/exec/vector/UnionColumnVector.java  |    2 +
 .../ptest2/src/main/resources/source-prep.vm    |    2 -
 133 files changed, 30015 insertions(+), 12493 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e69bd1ee/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------


[28/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information - Adding metastore scripts (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information - Adding metastore scripts (Hari Subramaniyan, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/llap
Commit: 6c3d3b495bafcf78445d785184b7da025e348c6b
Parents: 55375ec
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Tue Apr 19 11:47:12 2016 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Tue Apr 19 11:47:12 2016 -0700

----------------------------------------------------------------------
 .../scripts/upgrade/derby/034-HIVE-13076.derby.sql |  3 +++
 .../scripts/upgrade/mssql/019-HIVE-13076.mssql.sql | 15 +++++++++++++++
 .../scripts/upgrade/mysql/034-HIVE-13076.mysql.sql | 17 +++++++++++++++++
 .../upgrade/oracle/034-HIVE-13076.oracle.sql       | 15 +++++++++++++++
 .../upgrade/postgres/033-HIVE-13076.postgres.sql   | 15 +++++++++++++++
 5 files changed, 65 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6c3d3b49/metastore/scripts/upgrade/derby/034-HIVE-13076.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/034-HIVE-13076.derby.sql b/metastore/scripts/upgrade/derby/034-HIVE-13076.derby.sql
new file mode 100644
index 0000000..b062c56
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/034-HIVE-13076.derby.sql
@@ -0,0 +1,3 @@
+CREATE TABLE "APP"."KEY_CONSTRAINTS" ("CHILD_CD_ID" BIGINT, "CHILD_TBL_ID" BIGINT, "PARENT_CD_ID" BIGINT NOT NULL, "PARENT_TBL_ID" BIGINT NOT NULL,  "POSITION" BIGINT NOT NULL, "CONSTRAINT_NAME" VARCHAR(400) NOT NULL, "CONSTRAINT_TYPE" SMALLINT NOT NULL, "UPDATE_RULE" SMALLINT, "DELETE_RULE" SMALLINT, "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL);
+ALTER TABLE "APP"."KEY_CONSTRAINTS" ADD CONSTRAINT "CONSTRAINTS_PK" PRIMARY KEY ("CONSTRAINT_NAME", "POSITION");
+CREATE INDEX "APP"."CONSTRAINTS_PARENT_TBL_ID_INDEX" ON "APP"."KEY_CONSTRAINTS"("PARENT_TBL_ID");

http://git-wip-us.apache.org/repos/asf/hive/blob/6c3d3b49/metastore/scripts/upgrade/mssql/019-HIVE-13076.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/019-HIVE-13076.mssql.sql b/metastore/scripts/upgrade/mssql/019-HIVE-13076.mssql.sql
new file mode 100644
index 0000000..00ddb73
--- /dev/null
+++ b/metastore/scripts/upgrade/mssql/019-HIVE-13076.mssql.sql
@@ -0,0 +1,15 @@
+CREATE TABLE KEY_CONSTRAINTS
+(
+  CHILD_CD_ID BIGINT,
+  CHILD_TBL_ID BIGINT,
+  PARENT_CD_ID BIGINT NOT NULL,
+  PARENT_TBL_ID BIGINT NOT NULL,
+  POSITION INT NOT NULL,
+  CONSTRAINT_NAME VARCHAR(400) NOT NULL,
+  CONSTRAINT_TYPE SMALLINT NOT NULL,
+  UPDATE_RULE SMALLINT,
+  DELETE_RULE SMALLINT,
+  ENABLE_VALIDATE_RELY SMALLINT NOT NULL
+) ;
+ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAINT_NAME, POSITION);
+CREATE INDEX CONSTRAINTS_PARENT_TBL_ID__INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/6c3d3b49/metastore/scripts/upgrade/mysql/034-HIVE-13076.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/034-HIVE-13076.mysql.sql b/metastore/scripts/upgrade/mysql/034-HIVE-13076.mysql.sql
new file mode 100644
index 0000000..c9a5e1d
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/034-HIVE-13076.mysql.sql
@@ -0,0 +1,17 @@
+CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
+(
+  `CHILD_CD_ID` BIGINT,
+  `CHILD_TBL_ID` BIGINT,
+  `PARENT_CD_ID` BIGINT NOT NULL,
+  `PARENT_TBL_ID` BIGINT NOT NULL,
+  `POSITION` BIGINT NOT NULL,
+  `CONSTRAINT_NAME` VARCHAR(400) NOT NULL,
+  `CONSTRAINT_TYPE` SMALLINT(6)  NOT NULL,
+  `UPDATE_RULE` SMALLINT(6),
+  `DELETE_RULE` SMALLINT(6),
+  `ENABLE_VALIDATE_RELY` SMALLINT(6) NOT NULL,
+  PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+CREATE INDEX `CONSTRAINTS_PARENT_TABLE_ID_INDEX` ON KEY_CONSTRAINTS (`PARENT_TBL_ID`) USING BTREE;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/6c3d3b49/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql b/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql
new file mode 100644
index 0000000..baf855c
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql
@@ -0,0 +1,15 @@
+CREATE TABLE IF NOT EXISTS  KEY_CONSTRAINTS
+(
+  CHILD_CD_ID NUMBER,
+  CHILD_TBL_ID NUMBER,
+  PARENT_CD_ID NUMBER NOT NULL,
+  PARENT_TBL_ID NUMBER NOT NULL,
+  POSITION NUMBER NOT NULL,
+  CONSTRAINT_NAME VARCHAR(400) NOT NULL,
+  CONSTRAINT_TYPE NUMBER NOT NULL,
+  UPDATE_RULE NUMBER,
+  DELETE_RULE NUMBER,
+  ENABLE_VALIDATE_RELY NUMBER NOT NULL
+) ;
+ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAINT_NAME, POSITION);
+CREATE INDEX CONSTRAINTS_PARENT_TBL_ID_INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/6c3d3b49/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql b/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql
new file mode 100644
index 0000000..ec1fb48
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql
@@ -0,0 +1,15 @@
+CREATE TABLE IF NOT EXISTS  "KEY_CONSTRAINTS"
+(
+  "CHILD_CD_ID" BIGINT,
+  "CHILD_TBL_ID" BIGINT,
+  "PARENT_CD_ID" BIGINT NOT NULL,
+  "PARENT_TBL_ID" BIGINT NOT NULL,
+  "POSITION" BIGINT NOT NULL,
+  "CONSTRAINT_NAME" VARCHAR(400) NOT NULL,
+  "CONSTRAINT_TYPE" SMALLINT NOT NULL,
+  "UPDATE_RULE" SMALLINT,
+  "DELETE_RULE" SMALLINT,
+  "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL,
+  PRIMARY KEY ("CONSTRAINT_NAME", "POSITION")
+) ;
+CREATE INDEX "CONSTRAINTS_PARENT_TBLID_INDEX" ON "KEY_CONSTRAINTS" USING BTREE ("PARENT_TBL_ID");


[09/30] hive git commit: HIVE-11427: Location of temporary table for CREATE TABLE SELECT broken by HIVE-7079. (Yongzhi Chen, reviewed by Sergio Pena)

Posted by jd...@apache.org.
HIVE-11427: Location of temporary table for CREATE TABLE SELECT broken by HIVE-7079. (Yongzhi Chen, reviewed by Sergio Pena)


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

Branch: refs/heads/llap
Commit: cc3544bd893ce758a23b8d52e9c44bb4f80f14ad
Parents: 68dc9dd
Author: Yongzhi Chen <yc...@apache.org>
Authored: Tue Apr 12 13:33:43 2016 -0400
Committer: Yongzhi Chen <yc...@apache.org>
Committed: Sun Apr 17 01:35:22 2016 -0400

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |  3 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  7 ++-
 .../queries/clientpositive/encryption_ctas.q    | 16 ++++++
 .../encrypted/encryption_ctas.q.out             | 56 ++++++++++++++++++++
 4 files changed, 80 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cc3544bd/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 889884c..e46e6ce 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -522,7 +522,8 @@ encrypted.query.files=encryption_join_unencrypted_tbl.q,\
   encryption_insert_values.q \
   encryption_drop_view.q \
   encryption_drop_partition.q \
-  encryption_with_trash.q
+  encryption_with_trash.q \
+  encryption_ctas.q
 
 beeline.positive.exclude=add_part_exist.q,\
   alter1.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/cc3544bd/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 329c617..96df189 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -2073,7 +2073,12 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
               Path location;
               try {
                 Warehouse wh = new Warehouse(conf);
-                location = wh.getDatabasePath(db.getDatabase(names[0]));
+                //Use destination table's db location.
+                String destTableDb = qb.getTableDesc() != null? qb.getTableDesc().getDatabaseName(): null;
+                if (destTableDb == null) {
+                  destTableDb = names[0];
+                }
+                location = wh.getDatabasePath(db.getDatabase(destTableDb));
               } catch (MetaException e) {
                 throw new SemanticException(e);
               }

http://git-wip-us.apache.org/repos/asf/hive/blob/cc3544bd/ql/src/test/queries/clientpositive/encryption_ctas.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/encryption_ctas.q b/ql/src/test/queries/clientpositive/encryption_ctas.q
new file mode 100644
index 0000000..93058b6
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/encryption_ctas.q
@@ -0,0 +1,16 @@
+DROP TABLE IF EXISTS testCT.encrypted_tablectas PURGE;
+DROP DATABASE IF EXISTS testCT;
+CREATE DATABASE testCT;
+dfs ${system:test.dfs.mkdir} ${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_tablectas;
+
+CRYPTO CREATE_KEY --keyName key_128 --bitLength 128;
+CRYPTO CREATE_ZONE --keyName key_128 --path ${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_tablectas;
+
+CREATE TABLE testCT.encrypted_tablectas LOCATION '${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_tablectas'
+AS SELECT * from src where key = 100 limit 1;
+
+select * from testCT.encrypted_tablectas;
+
+DROP TABLE testCT.encrypted_tablectas PURGE;
+CRYPTO DELETE_KEY --keyName key_128;
+DROP DATABASE testCT;

http://git-wip-us.apache.org/repos/asf/hive/blob/cc3544bd/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out
new file mode 100644
index 0000000..5b503ac
--- /dev/null
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out
@@ -0,0 +1,56 @@
+PREHOOK: query: DROP TABLE IF EXISTS testCT.encrypted_tablectas PURGE
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS testCT.encrypted_tablectas PURGE
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: DROP DATABASE IF EXISTS testCT
+PREHOOK: type: DROPDATABASE
+POSTHOOK: query: DROP DATABASE IF EXISTS testCT
+POSTHOOK: type: DROPDATABASE
+PREHOOK: query: CREATE DATABASE testCT
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:testCT
+POSTHOOK: query: CREATE DATABASE testCT
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:testCT
+Encryption key created: 'key_128'
+Encryption zone created: '/build/ql/test/data/warehouse/default/encrypted_tablectas' using key: 'key_128'
+#### A masked pattern was here ####
+AS SELECT * from src where key = 100 limit 1
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+PREHOOK: Output: database:testct
+PREHOOK: Output: testCT@encrypted_tablectas
+#### A masked pattern was here ####
+AS SELECT * from src where key = 100 limit 1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: Output: database:testct
+POSTHOOK: Output: testCT@encrypted_tablectas
+PREHOOK: query: select * from testCT.encrypted_tablectas
+PREHOOK: type: QUERY
+PREHOOK: Input: testct@encrypted_tablectas
+#### A masked pattern was here ####
+POSTHOOK: query: select * from testCT.encrypted_tablectas
+POSTHOOK: type: QUERY
+POSTHOOK: Input: testct@encrypted_tablectas
+#### A masked pattern was here ####
+100	val_100
+PREHOOK: query: DROP TABLE testCT.encrypted_tablectas PURGE
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: testct@encrypted_tablectas
+PREHOOK: Output: testct@encrypted_tablectas
+POSTHOOK: query: DROP TABLE testCT.encrypted_tablectas PURGE
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: testct@encrypted_tablectas
+POSTHOOK: Output: testct@encrypted_tablectas
+Encryption key deleted: 'key_128'
+PREHOOK: query: DROP DATABASE testCT
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:testct
+PREHOOK: Output: database:testct
+POSTHOOK: query: DROP DATABASE testCT
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:testct
+POSTHOOK: Output: database:testct


[21/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
new file mode 100644
index 0000000..a25234b
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
@@ -0,0 +1,443 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * 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.3)")
+public class PrimaryKeysResponse implements org.apache.thrift.TBase<PrimaryKeysResponse, PrimaryKeysResponse._Fields>, java.io.Serializable, Cloneable, Comparable<PrimaryKeysResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("PrimaryKeysResponse");
+
+  private static final org.apache.thrift.protocol.TField PRIMARY_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("primaryKeys", 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 PrimaryKeysResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new PrimaryKeysResponseTupleSchemeFactory());
+  }
+
+  private List<SQLPrimaryKey> primaryKeys; // 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 {
+    PRIMARY_KEYS((short)1, "primaryKeys");
+
+    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: // PRIMARY_KEYS
+          return PRIMARY_KEYS;
+        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.PRIMARY_KEYS, new org.apache.thrift.meta_data.FieldMetaData("primaryKeys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        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, SQLPrimaryKey.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PrimaryKeysResponse.class, metaDataMap);
+  }
+
+  public PrimaryKeysResponse() {
+  }
+
+  public PrimaryKeysResponse(
+    List<SQLPrimaryKey> primaryKeys)
+  {
+    this();
+    this.primaryKeys = primaryKeys;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public PrimaryKeysResponse(PrimaryKeysResponse other) {
+    if (other.isSetPrimaryKeys()) {
+      List<SQLPrimaryKey> __this__primaryKeys = new ArrayList<SQLPrimaryKey>(other.primaryKeys.size());
+      for (SQLPrimaryKey other_element : other.primaryKeys) {
+        __this__primaryKeys.add(new SQLPrimaryKey(other_element));
+      }
+      this.primaryKeys = __this__primaryKeys;
+    }
+  }
+
+  public PrimaryKeysResponse deepCopy() {
+    return new PrimaryKeysResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.primaryKeys = null;
+  }
+
+  public int getPrimaryKeysSize() {
+    return (this.primaryKeys == null) ? 0 : this.primaryKeys.size();
+  }
+
+  public java.util.Iterator<SQLPrimaryKey> getPrimaryKeysIterator() {
+    return (this.primaryKeys == null) ? null : this.primaryKeys.iterator();
+  }
+
+  public void addToPrimaryKeys(SQLPrimaryKey elem) {
+    if (this.primaryKeys == null) {
+      this.primaryKeys = new ArrayList<SQLPrimaryKey>();
+    }
+    this.primaryKeys.add(elem);
+  }
+
+  public List<SQLPrimaryKey> getPrimaryKeys() {
+    return this.primaryKeys;
+  }
+
+  public void setPrimaryKeys(List<SQLPrimaryKey> primaryKeys) {
+    this.primaryKeys = primaryKeys;
+  }
+
+  public void unsetPrimaryKeys() {
+    this.primaryKeys = null;
+  }
+
+  /** Returns true if field primaryKeys is set (has been assigned a value) and false otherwise */
+  public boolean isSetPrimaryKeys() {
+    return this.primaryKeys != null;
+  }
+
+  public void setPrimaryKeysIsSet(boolean value) {
+    if (!value) {
+      this.primaryKeys = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PRIMARY_KEYS:
+      if (value == null) {
+        unsetPrimaryKeys();
+      } else {
+        setPrimaryKeys((List<SQLPrimaryKey>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PRIMARY_KEYS:
+      return getPrimaryKeys();
+
+    }
+    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 PRIMARY_KEYS:
+      return isSetPrimaryKeys();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof PrimaryKeysResponse)
+      return this.equals((PrimaryKeysResponse)that);
+    return false;
+  }
+
+  public boolean equals(PrimaryKeysResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_primaryKeys = true && this.isSetPrimaryKeys();
+    boolean that_present_primaryKeys = true && that.isSetPrimaryKeys();
+    if (this_present_primaryKeys || that_present_primaryKeys) {
+      if (!(this_present_primaryKeys && that_present_primaryKeys))
+        return false;
+      if (!this.primaryKeys.equals(that.primaryKeys))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_primaryKeys = true && (isSetPrimaryKeys());
+    list.add(present_primaryKeys);
+    if (present_primaryKeys)
+      list.add(primaryKeys);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(PrimaryKeysResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetPrimaryKeys()).compareTo(other.isSetPrimaryKeys());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPrimaryKeys()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.primaryKeys, other.primaryKeys);
+      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("PrimaryKeysResponse(");
+    boolean first = true;
+
+    sb.append("primaryKeys:");
+    if (this.primaryKeys == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.primaryKeys);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetPrimaryKeys()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'primaryKeys' 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 PrimaryKeysResponseStandardSchemeFactory implements SchemeFactory {
+    public PrimaryKeysResponseStandardScheme getScheme() {
+      return new PrimaryKeysResponseStandardScheme();
+    }
+  }
+
+  private static class PrimaryKeysResponseStandardScheme extends StandardScheme<PrimaryKeysResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, PrimaryKeysResponse 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: // PRIMARY_KEYS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list322 = iprot.readListBegin();
+                struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list322.size);
+                SQLPrimaryKey _elem323;
+                for (int _i324 = 0; _i324 < _list322.size; ++_i324)
+                {
+                  _elem323 = new SQLPrimaryKey();
+                  _elem323.read(iprot);
+                  struct.primaryKeys.add(_elem323);
+                }
+                iprot.readListEnd();
+              }
+              struct.setPrimaryKeysIsSet(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, PrimaryKeysResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.primaryKeys != null) {
+        oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
+          for (SQLPrimaryKey _iter325 : struct.primaryKeys)
+          {
+            _iter325.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class PrimaryKeysResponseTupleSchemeFactory implements SchemeFactory {
+    public PrimaryKeysResponseTupleScheme getScheme() {
+      return new PrimaryKeysResponseTupleScheme();
+    }
+  }
+
+  private static class PrimaryKeysResponseTupleScheme extends TupleScheme<PrimaryKeysResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, PrimaryKeysResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.primaryKeys.size());
+        for (SQLPrimaryKey _iter326 : struct.primaryKeys)
+        {
+          _iter326.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, PrimaryKeysResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list327.size);
+        SQLPrimaryKey _elem328;
+        for (int _i329 = 0; _i329 < _list327.size; ++_i329)
+        {
+          _elem328 = new SQLPrimaryKey();
+          _elem328.read(iprot);
+          struct.primaryKeys.add(_elem328);
+        }
+      }
+      struct.setPrimaryKeysIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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
index ab151b1..0905181 100644
--- 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
@@ -547,13 +547,13 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
           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)
+                org.apache.thrift.protocol.TList _list576 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list576.size);
+                long _elem577;
+                for (int _i578 = 0; _i578 < _list576.size; ++_i578)
                 {
-                  _elem561 = iprot.readI64();
-                  struct.fileIds.add(_elem561);
+                  _elem577 = iprot.readI64();
+                  struct.fileIds.add(_elem577);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list563 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list563.size);
-                ByteBuffer _elem564;
-                for (int _i565 = 0; _i565 < _list563.size; ++_i565)
+                org.apache.thrift.protocol.TList _list579 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list579.size);
+                ByteBuffer _elem580;
+                for (int _i581 = 0; _i581 < _list579.size; ++_i581)
                 {
-                  _elem564 = iprot.readBinary();
-                  struct.metadata.add(_elem564);
+                  _elem580 = iprot.readBinary();
+                  struct.metadata.add(_elem580);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         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 _iter566 : struct.fileIds)
+          for (long _iter582 : struct.fileIds)
           {
-            oprot.writeI64(_iter566);
+            oprot.writeI64(_iter582);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter567 : struct.metadata)
+          for (ByteBuffer _iter583 : struct.metadata)
           {
-            oprot.writeBinary(_iter567);
+            oprot.writeBinary(_iter583);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter568 : struct.fileIds)
+        for (long _iter584 : struct.fileIds)
         {
-          oprot.writeI64(_iter568);
+          oprot.writeI64(_iter584);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter569 : struct.metadata)
+        for (ByteBuffer _iter585 : struct.metadata)
         {
-          oprot.writeBinary(_iter569);
+          oprot.writeBinary(_iter585);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
     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 _list570 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list570.size);
-        long _elem571;
-        for (int _i572 = 0; _i572 < _list570.size; ++_i572)
+        org.apache.thrift.protocol.TList _list586 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list586.size);
+        long _elem587;
+        for (int _i588 = 0; _i588 < _list586.size; ++_i588)
         {
-          _elem571 = iprot.readI64();
-          struct.fileIds.add(_elem571);
+          _elem587 = iprot.readI64();
+          struct.fileIds.add(_elem587);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list573 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list573.size);
-        ByteBuffer _elem574;
-        for (int _i575 = 0; _i575 < _list573.size; ++_i575)
+        org.apache.thrift.protocol.TList _list589 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list589.size);
+        ByteBuffer _elem590;
+        for (int _i591 = 0; _i591 < _list589.size; ++_i591)
         {
-          _elem574 = iprot.readBinary();
-          struct.metadata.add(_elem574);
+          _elem590 = iprot.readBinary();
+          struct.metadata.add(_elem590);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
index 7f8a044..1921bf5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
@@ -168,13 +168,13 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
           if (field.type == NAMES_FIELD_DESC.type) {
             List<String> names;
             {
-              org.apache.thrift.protocol.TList _list404 = iprot.readListBegin();
-              names = new ArrayList<String>(_list404.size);
-              String _elem405;
-              for (int _i406 = 0; _i406 < _list404.size; ++_i406)
+              org.apache.thrift.protocol.TList _list420 = iprot.readListBegin();
+              names = new ArrayList<String>(_list420.size);
+              String _elem421;
+              for (int _i422 = 0; _i422 < _list420.size; ++_i422)
               {
-                _elem405 = iprot.readString();
-                names.add(_elem405);
+                _elem421 = iprot.readString();
+                names.add(_elem421);
               }
               iprot.readListEnd();
             }
@@ -187,14 +187,14 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
           if (field.type == EXPRS_FIELD_DESC.type) {
             List<DropPartitionsExpr> exprs;
             {
-              org.apache.thrift.protocol.TList _list407 = iprot.readListBegin();
-              exprs = new ArrayList<DropPartitionsExpr>(_list407.size);
-              DropPartitionsExpr _elem408;
-              for (int _i409 = 0; _i409 < _list407.size; ++_i409)
+              org.apache.thrift.protocol.TList _list423 = iprot.readListBegin();
+              exprs = new ArrayList<DropPartitionsExpr>(_list423.size);
+              DropPartitionsExpr _elem424;
+              for (int _i425 = 0; _i425 < _list423.size; ++_i425)
               {
-                _elem408 = new DropPartitionsExpr();
-                _elem408.read(iprot);
-                exprs.add(_elem408);
+                _elem424 = new DropPartitionsExpr();
+                _elem424.read(iprot);
+                exprs.add(_elem424);
               }
               iprot.readListEnd();
             }
@@ -219,9 +219,9 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter410 : names)
+          for (String _iter426 : names)
           {
-            oprot.writeString(_iter410);
+            oprot.writeString(_iter426);
           }
           oprot.writeListEnd();
         }
@@ -230,9 +230,9 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter411 : exprs)
+          for (DropPartitionsExpr _iter427 : exprs)
           {
-            _iter411.write(oprot);
+            _iter427.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -250,13 +250,13 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         case NAMES:
           List<String> names;
           {
-            org.apache.thrift.protocol.TList _list412 = iprot.readListBegin();
-            names = new ArrayList<String>(_list412.size);
-            String _elem413;
-            for (int _i414 = 0; _i414 < _list412.size; ++_i414)
+            org.apache.thrift.protocol.TList _list428 = iprot.readListBegin();
+            names = new ArrayList<String>(_list428.size);
+            String _elem429;
+            for (int _i430 = 0; _i430 < _list428.size; ++_i430)
             {
-              _elem413 = iprot.readString();
-              names.add(_elem413);
+              _elem429 = iprot.readString();
+              names.add(_elem429);
             }
             iprot.readListEnd();
           }
@@ -264,14 +264,14 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         case EXPRS:
           List<DropPartitionsExpr> exprs;
           {
-            org.apache.thrift.protocol.TList _list415 = iprot.readListBegin();
-            exprs = new ArrayList<DropPartitionsExpr>(_list415.size);
-            DropPartitionsExpr _elem416;
-            for (int _i417 = 0; _i417 < _list415.size; ++_i417)
+            org.apache.thrift.protocol.TList _list431 = iprot.readListBegin();
+            exprs = new ArrayList<DropPartitionsExpr>(_list431.size);
+            DropPartitionsExpr _elem432;
+            for (int _i433 = 0; _i433 < _list431.size; ++_i433)
             {
-              _elem416 = new DropPartitionsExpr();
-              _elem416.read(iprot);
-              exprs.add(_elem416);
+              _elem432 = new DropPartitionsExpr();
+              _elem432.read(iprot);
+              exprs.add(_elem432);
             }
             iprot.readListEnd();
           }
@@ -291,9 +291,9 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter418 : names)
+          for (String _iter434 : names)
           {
-            oprot.writeString(_iter418);
+            oprot.writeString(_iter434);
           }
           oprot.writeListEnd();
         }
@@ -302,9 +302,9 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter419 : exprs)
+          for (DropPartitionsExpr _iter435 : exprs)
           {
-            _iter419.write(oprot);
+            _iter435.write(oprot);
           }
           oprot.writeListEnd();
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java
new file mode 100644
index 0000000..44c5d72
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java
@@ -0,0 +1,1715 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * 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.3)")
+public class SQLForeignKey implements org.apache.thrift.TBase<SQLForeignKey, SQLForeignKey._Fields>, java.io.Serializable, Cloneable, Comparable<SQLForeignKey> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLForeignKey");
+
+  private static final org.apache.thrift.protocol.TField PKTABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("pktable_db", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PKTABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("pktable_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField PKCOLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("pkcolumn_name", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField FKTABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("fktable_db", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField FKTABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("fktable_name", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField FKCOLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("fkcolumn_name", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField KEY_SEQ_FIELD_DESC = new org.apache.thrift.protocol.TField("key_seq", org.apache.thrift.protocol.TType.I32, (short)7);
+  private static final org.apache.thrift.protocol.TField UPDATE_RULE_FIELD_DESC = new org.apache.thrift.protocol.TField("update_rule", org.apache.thrift.protocol.TType.I32, (short)8);
+  private static final org.apache.thrift.protocol.TField DELETE_RULE_FIELD_DESC = new org.apache.thrift.protocol.TField("delete_rule", org.apache.thrift.protocol.TType.I32, (short)9);
+  private static final org.apache.thrift.protocol.TField FK_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("fk_name", org.apache.thrift.protocol.TType.STRING, (short)10);
+  private static final org.apache.thrift.protocol.TField PK_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("pk_name", org.apache.thrift.protocol.TType.STRING, (short)11);
+  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)12);
+  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)13);
+  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)14);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SQLForeignKeyStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SQLForeignKeyTupleSchemeFactory());
+  }
+
+  private String pktable_db; // required
+  private String pktable_name; // required
+  private String pkcolumn_name; // required
+  private String fktable_db; // required
+  private String fktable_name; // required
+  private String fkcolumn_name; // required
+  private int key_seq; // required
+  private int update_rule; // required
+  private int delete_rule; // required
+  private String fk_name; // required
+  private String pk_name; // required
+  private boolean enable_cstr; // required
+  private boolean validate_cstr; // required
+  private boolean rely_cstr; // 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 {
+    PKTABLE_DB((short)1, "pktable_db"),
+    PKTABLE_NAME((short)2, "pktable_name"),
+    PKCOLUMN_NAME((short)3, "pkcolumn_name"),
+    FKTABLE_DB((short)4, "fktable_db"),
+    FKTABLE_NAME((short)5, "fktable_name"),
+    FKCOLUMN_NAME((short)6, "fkcolumn_name"),
+    KEY_SEQ((short)7, "key_seq"),
+    UPDATE_RULE((short)8, "update_rule"),
+    DELETE_RULE((short)9, "delete_rule"),
+    FK_NAME((short)10, "fk_name"),
+    PK_NAME((short)11, "pk_name"),
+    ENABLE_CSTR((short)12, "enable_cstr"),
+    VALIDATE_CSTR((short)13, "validate_cstr"),
+    RELY_CSTR((short)14, "rely_cstr");
+
+    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: // PKTABLE_DB
+          return PKTABLE_DB;
+        case 2: // PKTABLE_NAME
+          return PKTABLE_NAME;
+        case 3: // PKCOLUMN_NAME
+          return PKCOLUMN_NAME;
+        case 4: // FKTABLE_DB
+          return FKTABLE_DB;
+        case 5: // FKTABLE_NAME
+          return FKTABLE_NAME;
+        case 6: // FKCOLUMN_NAME
+          return FKCOLUMN_NAME;
+        case 7: // KEY_SEQ
+          return KEY_SEQ;
+        case 8: // UPDATE_RULE
+          return UPDATE_RULE;
+        case 9: // DELETE_RULE
+          return DELETE_RULE;
+        case 10: // FK_NAME
+          return FK_NAME;
+        case 11: // PK_NAME
+          return PK_NAME;
+        case 12: // ENABLE_CSTR
+          return ENABLE_CSTR;
+        case 13: // VALIDATE_CSTR
+          return VALIDATE_CSTR;
+        case 14: // RELY_CSTR
+          return RELY_CSTR;
+        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 __KEY_SEQ_ISSET_ID = 0;
+  private static final int __UPDATE_RULE_ISSET_ID = 1;
+  private static final int __DELETE_RULE_ISSET_ID = 2;
+  private static final int __ENABLE_CSTR_ISSET_ID = 3;
+  private static final int __VALIDATE_CSTR_ISSET_ID = 4;
+  private static final int __RELY_CSTR_ISSET_ID = 5;
+  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.PKTABLE_DB, new org.apache.thrift.meta_data.FieldMetaData("pktable_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PKTABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("pktable_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PKCOLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("pkcolumn_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FKTABLE_DB, new org.apache.thrift.meta_data.FieldMetaData("fktable_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FKTABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("fktable_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FKCOLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("fkcolumn_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.KEY_SEQ, new org.apache.thrift.meta_data.FieldMetaData("key_seq", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.UPDATE_RULE, new org.apache.thrift.meta_data.FieldMetaData("update_rule", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.DELETE_RULE, new org.apache.thrift.meta_data.FieldMetaData("delete_rule", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.FK_NAME, new org.apache.thrift.meta_data.FieldMetaData("fk_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PK_NAME, new org.apache.thrift.meta_data.FieldMetaData("pk_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ENABLE_CSTR, new org.apache.thrift.meta_data.FieldMetaData("enable_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.VALIDATE_CSTR, new org.apache.thrift.meta_data.FieldMetaData("validate_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.RELY_CSTR, new org.apache.thrift.meta_data.FieldMetaData("rely_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SQLForeignKey.class, metaDataMap);
+  }
+
+  public SQLForeignKey() {
+  }
+
+  public SQLForeignKey(
+    String pktable_db,
+    String pktable_name,
+    String pkcolumn_name,
+    String fktable_db,
+    String fktable_name,
+    String fkcolumn_name,
+    int key_seq,
+    int update_rule,
+    int delete_rule,
+    String fk_name,
+    String pk_name,
+    boolean enable_cstr,
+    boolean validate_cstr,
+    boolean rely_cstr)
+  {
+    this();
+    this.pktable_db = pktable_db;
+    this.pktable_name = pktable_name;
+    this.pkcolumn_name = pkcolumn_name;
+    this.fktable_db = fktable_db;
+    this.fktable_name = fktable_name;
+    this.fkcolumn_name = fkcolumn_name;
+    this.key_seq = key_seq;
+    setKey_seqIsSet(true);
+    this.update_rule = update_rule;
+    setUpdate_ruleIsSet(true);
+    this.delete_rule = delete_rule;
+    setDelete_ruleIsSet(true);
+    this.fk_name = fk_name;
+    this.pk_name = pk_name;
+    this.enable_cstr = enable_cstr;
+    setEnable_cstrIsSet(true);
+    this.validate_cstr = validate_cstr;
+    setValidate_cstrIsSet(true);
+    this.rely_cstr = rely_cstr;
+    setRely_cstrIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SQLForeignKey(SQLForeignKey other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetPktable_db()) {
+      this.pktable_db = other.pktable_db;
+    }
+    if (other.isSetPktable_name()) {
+      this.pktable_name = other.pktable_name;
+    }
+    if (other.isSetPkcolumn_name()) {
+      this.pkcolumn_name = other.pkcolumn_name;
+    }
+    if (other.isSetFktable_db()) {
+      this.fktable_db = other.fktable_db;
+    }
+    if (other.isSetFktable_name()) {
+      this.fktable_name = other.fktable_name;
+    }
+    if (other.isSetFkcolumn_name()) {
+      this.fkcolumn_name = other.fkcolumn_name;
+    }
+    this.key_seq = other.key_seq;
+    this.update_rule = other.update_rule;
+    this.delete_rule = other.delete_rule;
+    if (other.isSetFk_name()) {
+      this.fk_name = other.fk_name;
+    }
+    if (other.isSetPk_name()) {
+      this.pk_name = other.pk_name;
+    }
+    this.enable_cstr = other.enable_cstr;
+    this.validate_cstr = other.validate_cstr;
+    this.rely_cstr = other.rely_cstr;
+  }
+
+  public SQLForeignKey deepCopy() {
+    return new SQLForeignKey(this);
+  }
+
+  @Override
+  public void clear() {
+    this.pktable_db = null;
+    this.pktable_name = null;
+    this.pkcolumn_name = null;
+    this.fktable_db = null;
+    this.fktable_name = null;
+    this.fkcolumn_name = null;
+    setKey_seqIsSet(false);
+    this.key_seq = 0;
+    setUpdate_ruleIsSet(false);
+    this.update_rule = 0;
+    setDelete_ruleIsSet(false);
+    this.delete_rule = 0;
+    this.fk_name = null;
+    this.pk_name = null;
+    setEnable_cstrIsSet(false);
+    this.enable_cstr = false;
+    setValidate_cstrIsSet(false);
+    this.validate_cstr = false;
+    setRely_cstrIsSet(false);
+    this.rely_cstr = false;
+  }
+
+  public String getPktable_db() {
+    return this.pktable_db;
+  }
+
+  public void setPktable_db(String pktable_db) {
+    this.pktable_db = pktable_db;
+  }
+
+  public void unsetPktable_db() {
+    this.pktable_db = null;
+  }
+
+  /** Returns true if field pktable_db is set (has been assigned a value) and false otherwise */
+  public boolean isSetPktable_db() {
+    return this.pktable_db != null;
+  }
+
+  public void setPktable_dbIsSet(boolean value) {
+    if (!value) {
+      this.pktable_db = null;
+    }
+  }
+
+  public String getPktable_name() {
+    return this.pktable_name;
+  }
+
+  public void setPktable_name(String pktable_name) {
+    this.pktable_name = pktable_name;
+  }
+
+  public void unsetPktable_name() {
+    this.pktable_name = null;
+  }
+
+  /** Returns true if field pktable_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetPktable_name() {
+    return this.pktable_name != null;
+  }
+
+  public void setPktable_nameIsSet(boolean value) {
+    if (!value) {
+      this.pktable_name = null;
+    }
+  }
+
+  public String getPkcolumn_name() {
+    return this.pkcolumn_name;
+  }
+
+  public void setPkcolumn_name(String pkcolumn_name) {
+    this.pkcolumn_name = pkcolumn_name;
+  }
+
+  public void unsetPkcolumn_name() {
+    this.pkcolumn_name = null;
+  }
+
+  /** Returns true if field pkcolumn_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetPkcolumn_name() {
+    return this.pkcolumn_name != null;
+  }
+
+  public void setPkcolumn_nameIsSet(boolean value) {
+    if (!value) {
+      this.pkcolumn_name = null;
+    }
+  }
+
+  public String getFktable_db() {
+    return this.fktable_db;
+  }
+
+  public void setFktable_db(String fktable_db) {
+    this.fktable_db = fktable_db;
+  }
+
+  public void unsetFktable_db() {
+    this.fktable_db = null;
+  }
+
+  /** Returns true if field fktable_db is set (has been assigned a value) and false otherwise */
+  public boolean isSetFktable_db() {
+    return this.fktable_db != null;
+  }
+
+  public void setFktable_dbIsSet(boolean value) {
+    if (!value) {
+      this.fktable_db = null;
+    }
+  }
+
+  public String getFktable_name() {
+    return this.fktable_name;
+  }
+
+  public void setFktable_name(String fktable_name) {
+    this.fktable_name = fktable_name;
+  }
+
+  public void unsetFktable_name() {
+    this.fktable_name = null;
+  }
+
+  /** Returns true if field fktable_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetFktable_name() {
+    return this.fktable_name != null;
+  }
+
+  public void setFktable_nameIsSet(boolean value) {
+    if (!value) {
+      this.fktable_name = null;
+    }
+  }
+
+  public String getFkcolumn_name() {
+    return this.fkcolumn_name;
+  }
+
+  public void setFkcolumn_name(String fkcolumn_name) {
+    this.fkcolumn_name = fkcolumn_name;
+  }
+
+  public void unsetFkcolumn_name() {
+    this.fkcolumn_name = null;
+  }
+
+  /** Returns true if field fkcolumn_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetFkcolumn_name() {
+    return this.fkcolumn_name != null;
+  }
+
+  public void setFkcolumn_nameIsSet(boolean value) {
+    if (!value) {
+      this.fkcolumn_name = null;
+    }
+  }
+
+  public int getKey_seq() {
+    return this.key_seq;
+  }
+
+  public void setKey_seq(int key_seq) {
+    this.key_seq = key_seq;
+    setKey_seqIsSet(true);
+  }
+
+  public void unsetKey_seq() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __KEY_SEQ_ISSET_ID);
+  }
+
+  /** Returns true if field key_seq is set (has been assigned a value) and false otherwise */
+  public boolean isSetKey_seq() {
+    return EncodingUtils.testBit(__isset_bitfield, __KEY_SEQ_ISSET_ID);
+  }
+
+  public void setKey_seqIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __KEY_SEQ_ISSET_ID, value);
+  }
+
+  public int getUpdate_rule() {
+    return this.update_rule;
+  }
+
+  public void setUpdate_rule(int update_rule) {
+    this.update_rule = update_rule;
+    setUpdate_ruleIsSet(true);
+  }
+
+  public void unsetUpdate_rule() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPDATE_RULE_ISSET_ID);
+  }
+
+  /** Returns true if field update_rule is set (has been assigned a value) and false otherwise */
+  public boolean isSetUpdate_rule() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPDATE_RULE_ISSET_ID);
+  }
+
+  public void setUpdate_ruleIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPDATE_RULE_ISSET_ID, value);
+  }
+
+  public int getDelete_rule() {
+    return this.delete_rule;
+  }
+
+  public void setDelete_rule(int delete_rule) {
+    this.delete_rule = delete_rule;
+    setDelete_ruleIsSet(true);
+  }
+
+  public void unsetDelete_rule() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DELETE_RULE_ISSET_ID);
+  }
+
+  /** Returns true if field delete_rule is set (has been assigned a value) and false otherwise */
+  public boolean isSetDelete_rule() {
+    return EncodingUtils.testBit(__isset_bitfield, __DELETE_RULE_ISSET_ID);
+  }
+
+  public void setDelete_ruleIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DELETE_RULE_ISSET_ID, value);
+  }
+
+  public String getFk_name() {
+    return this.fk_name;
+  }
+
+  public void setFk_name(String fk_name) {
+    this.fk_name = fk_name;
+  }
+
+  public void unsetFk_name() {
+    this.fk_name = null;
+  }
+
+  /** Returns true if field fk_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetFk_name() {
+    return this.fk_name != null;
+  }
+
+  public void setFk_nameIsSet(boolean value) {
+    if (!value) {
+      this.fk_name = null;
+    }
+  }
+
+  public String getPk_name() {
+    return this.pk_name;
+  }
+
+  public void setPk_name(String pk_name) {
+    this.pk_name = pk_name;
+  }
+
+  public void unsetPk_name() {
+    this.pk_name = null;
+  }
+
+  /** Returns true if field pk_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetPk_name() {
+    return this.pk_name != null;
+  }
+
+  public void setPk_nameIsSet(boolean value) {
+    if (!value) {
+      this.pk_name = null;
+    }
+  }
+
+  public boolean isEnable_cstr() {
+    return this.enable_cstr;
+  }
+
+  public void setEnable_cstr(boolean enable_cstr) {
+    this.enable_cstr = enable_cstr;
+    setEnable_cstrIsSet(true);
+  }
+
+  public void unsetEnable_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field enable_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetEnable_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID);
+  }
+
+  public void setEnable_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID, value);
+  }
+
+  public boolean isValidate_cstr() {
+    return this.validate_cstr;
+  }
+
+  public void setValidate_cstr(boolean validate_cstr) {
+    this.validate_cstr = validate_cstr;
+    setValidate_cstrIsSet(true);
+  }
+
+  public void unsetValidate_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field validate_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetValidate_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID);
+  }
+
+  public void setValidate_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID, value);
+  }
+
+  public boolean isRely_cstr() {
+    return this.rely_cstr;
+  }
+
+  public void setRely_cstr(boolean rely_cstr) {
+    this.rely_cstr = rely_cstr;
+    setRely_cstrIsSet(true);
+  }
+
+  public void unsetRely_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RELY_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field rely_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetRely_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __RELY_CSTR_ISSET_ID);
+  }
+
+  public void setRely_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RELY_CSTR_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PKTABLE_DB:
+      if (value == null) {
+        unsetPktable_db();
+      } else {
+        setPktable_db((String)value);
+      }
+      break;
+
+    case PKTABLE_NAME:
+      if (value == null) {
+        unsetPktable_name();
+      } else {
+        setPktable_name((String)value);
+      }
+      break;
+
+    case PKCOLUMN_NAME:
+      if (value == null) {
+        unsetPkcolumn_name();
+      } else {
+        setPkcolumn_name((String)value);
+      }
+      break;
+
+    case FKTABLE_DB:
+      if (value == null) {
+        unsetFktable_db();
+      } else {
+        setFktable_db((String)value);
+      }
+      break;
+
+    case FKTABLE_NAME:
+      if (value == null) {
+        unsetFktable_name();
+      } else {
+        setFktable_name((String)value);
+      }
+      break;
+
+    case FKCOLUMN_NAME:
+      if (value == null) {
+        unsetFkcolumn_name();
+      } else {
+        setFkcolumn_name((String)value);
+      }
+      break;
+
+    case KEY_SEQ:
+      if (value == null) {
+        unsetKey_seq();
+      } else {
+        setKey_seq((Integer)value);
+      }
+      break;
+
+    case UPDATE_RULE:
+      if (value == null) {
+        unsetUpdate_rule();
+      } else {
+        setUpdate_rule((Integer)value);
+      }
+      break;
+
+    case DELETE_RULE:
+      if (value == null) {
+        unsetDelete_rule();
+      } else {
+        setDelete_rule((Integer)value);
+      }
+      break;
+
+    case FK_NAME:
+      if (value == null) {
+        unsetFk_name();
+      } else {
+        setFk_name((String)value);
+      }
+      break;
+
+    case PK_NAME:
+      if (value == null) {
+        unsetPk_name();
+      } else {
+        setPk_name((String)value);
+      }
+      break;
+
+    case ENABLE_CSTR:
+      if (value == null) {
+        unsetEnable_cstr();
+      } else {
+        setEnable_cstr((Boolean)value);
+      }
+      break;
+
+    case VALIDATE_CSTR:
+      if (value == null) {
+        unsetValidate_cstr();
+      } else {
+        setValidate_cstr((Boolean)value);
+      }
+      break;
+
+    case RELY_CSTR:
+      if (value == null) {
+        unsetRely_cstr();
+      } else {
+        setRely_cstr((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PKTABLE_DB:
+      return getPktable_db();
+
+    case PKTABLE_NAME:
+      return getPktable_name();
+
+    case PKCOLUMN_NAME:
+      return getPkcolumn_name();
+
+    case FKTABLE_DB:
+      return getFktable_db();
+
+    case FKTABLE_NAME:
+      return getFktable_name();
+
+    case FKCOLUMN_NAME:
+      return getFkcolumn_name();
+
+    case KEY_SEQ:
+      return getKey_seq();
+
+    case UPDATE_RULE:
+      return getUpdate_rule();
+
+    case DELETE_RULE:
+      return getDelete_rule();
+
+    case FK_NAME:
+      return getFk_name();
+
+    case PK_NAME:
+      return getPk_name();
+
+    case ENABLE_CSTR:
+      return isEnable_cstr();
+
+    case VALIDATE_CSTR:
+      return isValidate_cstr();
+
+    case RELY_CSTR:
+      return isRely_cstr();
+
+    }
+    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 PKTABLE_DB:
+      return isSetPktable_db();
+    case PKTABLE_NAME:
+      return isSetPktable_name();
+    case PKCOLUMN_NAME:
+      return isSetPkcolumn_name();
+    case FKTABLE_DB:
+      return isSetFktable_db();
+    case FKTABLE_NAME:
+      return isSetFktable_name();
+    case FKCOLUMN_NAME:
+      return isSetFkcolumn_name();
+    case KEY_SEQ:
+      return isSetKey_seq();
+    case UPDATE_RULE:
+      return isSetUpdate_rule();
+    case DELETE_RULE:
+      return isSetDelete_rule();
+    case FK_NAME:
+      return isSetFk_name();
+    case PK_NAME:
+      return isSetPk_name();
+    case ENABLE_CSTR:
+      return isSetEnable_cstr();
+    case VALIDATE_CSTR:
+      return isSetValidate_cstr();
+    case RELY_CSTR:
+      return isSetRely_cstr();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SQLForeignKey)
+      return this.equals((SQLForeignKey)that);
+    return false;
+  }
+
+  public boolean equals(SQLForeignKey that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_pktable_db = true && this.isSetPktable_db();
+    boolean that_present_pktable_db = true && that.isSetPktable_db();
+    if (this_present_pktable_db || that_present_pktable_db) {
+      if (!(this_present_pktable_db && that_present_pktable_db))
+        return false;
+      if (!this.pktable_db.equals(that.pktable_db))
+        return false;
+    }
+
+    boolean this_present_pktable_name = true && this.isSetPktable_name();
+    boolean that_present_pktable_name = true && that.isSetPktable_name();
+    if (this_present_pktable_name || that_present_pktable_name) {
+      if (!(this_present_pktable_name && that_present_pktable_name))
+        return false;
+      if (!this.pktable_name.equals(that.pktable_name))
+        return false;
+    }
+
+    boolean this_present_pkcolumn_name = true && this.isSetPkcolumn_name();
+    boolean that_present_pkcolumn_name = true && that.isSetPkcolumn_name();
+    if (this_present_pkcolumn_name || that_present_pkcolumn_name) {
+      if (!(this_present_pkcolumn_name && that_present_pkcolumn_name))
+        return false;
+      if (!this.pkcolumn_name.equals(that.pkcolumn_name))
+        return false;
+    }
+
+    boolean this_present_fktable_db = true && this.isSetFktable_db();
+    boolean that_present_fktable_db = true && that.isSetFktable_db();
+    if (this_present_fktable_db || that_present_fktable_db) {
+      if (!(this_present_fktable_db && that_present_fktable_db))
+        return false;
+      if (!this.fktable_db.equals(that.fktable_db))
+        return false;
+    }
+
+    boolean this_present_fktable_name = true && this.isSetFktable_name();
+    boolean that_present_fktable_name = true && that.isSetFktable_name();
+    if (this_present_fktable_name || that_present_fktable_name) {
+      if (!(this_present_fktable_name && that_present_fktable_name))
+        return false;
+      if (!this.fktable_name.equals(that.fktable_name))
+        return false;
+    }
+
+    boolean this_present_fkcolumn_name = true && this.isSetFkcolumn_name();
+    boolean that_present_fkcolumn_name = true && that.isSetFkcolumn_name();
+    if (this_present_fkcolumn_name || that_present_fkcolumn_name) {
+      if (!(this_present_fkcolumn_name && that_present_fkcolumn_name))
+        return false;
+      if (!this.fkcolumn_name.equals(that.fkcolumn_name))
+        return false;
+    }
+
+    boolean this_present_key_seq = true;
+    boolean that_present_key_seq = true;
+    if (this_present_key_seq || that_present_key_seq) {
+      if (!(this_present_key_seq && that_present_key_seq))
+        return false;
+      if (this.key_seq != that.key_seq)
+        return false;
+    }
+
+    boolean this_present_update_rule = true;
+    boolean that_present_update_rule = true;
+    if (this_present_update_rule || that_present_update_rule) {
+      if (!(this_present_update_rule && that_present_update_rule))
+        return false;
+      if (this.update_rule != that.update_rule)
+        return false;
+    }
+
+    boolean this_present_delete_rule = true;
+    boolean that_present_delete_rule = true;
+    if (this_present_delete_rule || that_present_delete_rule) {
+      if (!(this_present_delete_rule && that_present_delete_rule))
+        return false;
+      if (this.delete_rule != that.delete_rule)
+        return false;
+    }
+
+    boolean this_present_fk_name = true && this.isSetFk_name();
+    boolean that_present_fk_name = true && that.isSetFk_name();
+    if (this_present_fk_name || that_present_fk_name) {
+      if (!(this_present_fk_name && that_present_fk_name))
+        return false;
+      if (!this.fk_name.equals(that.fk_name))
+        return false;
+    }
+
+    boolean this_present_pk_name = true && this.isSetPk_name();
+    boolean that_present_pk_name = true && that.isSetPk_name();
+    if (this_present_pk_name || that_present_pk_name) {
+      if (!(this_present_pk_name && that_present_pk_name))
+        return false;
+      if (!this.pk_name.equals(that.pk_name))
+        return false;
+    }
+
+    boolean this_present_enable_cstr = true;
+    boolean that_present_enable_cstr = true;
+    if (this_present_enable_cstr || that_present_enable_cstr) {
+      if (!(this_present_enable_cstr && that_present_enable_cstr))
+        return false;
+      if (this.enable_cstr != that.enable_cstr)
+        return false;
+    }
+
+    boolean this_present_validate_cstr = true;
+    boolean that_present_validate_cstr = true;
+    if (this_present_validate_cstr || that_present_validate_cstr) {
+      if (!(this_present_validate_cstr && that_present_validate_cstr))
+        return false;
+      if (this.validate_cstr != that.validate_cstr)
+        return false;
+    }
+
+    boolean this_present_rely_cstr = true;
+    boolean that_present_rely_cstr = true;
+    if (this_present_rely_cstr || that_present_rely_cstr) {
+      if (!(this_present_rely_cstr && that_present_rely_cstr))
+        return false;
+      if (this.rely_cstr != that.rely_cstr)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_pktable_db = true && (isSetPktable_db());
+    list.add(present_pktable_db);
+    if (present_pktable_db)
+      list.add(pktable_db);
+
+    boolean present_pktable_name = true && (isSetPktable_name());
+    list.add(present_pktable_name);
+    if (present_pktable_name)
+      list.add(pktable_name);
+
+    boolean present_pkcolumn_name = true && (isSetPkcolumn_name());
+    list.add(present_pkcolumn_name);
+    if (present_pkcolumn_name)
+      list.add(pkcolumn_name);
+
+    boolean present_fktable_db = true && (isSetFktable_db());
+    list.add(present_fktable_db);
+    if (present_fktable_db)
+      list.add(fktable_db);
+
+    boolean present_fktable_name = true && (isSetFktable_name());
+    list.add(present_fktable_name);
+    if (present_fktable_name)
+      list.add(fktable_name);
+
+    boolean present_fkcolumn_name = true && (isSetFkcolumn_name());
+    list.add(present_fkcolumn_name);
+    if (present_fkcolumn_name)
+      list.add(fkcolumn_name);
+
+    boolean present_key_seq = true;
+    list.add(present_key_seq);
+    if (present_key_seq)
+      list.add(key_seq);
+
+    boolean present_update_rule = true;
+    list.add(present_update_rule);
+    if (present_update_rule)
+      list.add(update_rule);
+
+    boolean present_delete_rule = true;
+    list.add(present_delete_rule);
+    if (present_delete_rule)
+      list.add(delete_rule);
+
+    boolean present_fk_name = true && (isSetFk_name());
+    list.add(present_fk_name);
+    if (present_fk_name)
+      list.add(fk_name);
+
+    boolean present_pk_name = true && (isSetPk_name());
+    list.add(present_pk_name);
+    if (present_pk_name)
+      list.add(pk_name);
+
+    boolean present_enable_cstr = true;
+    list.add(present_enable_cstr);
+    if (present_enable_cstr)
+      list.add(enable_cstr);
+
+    boolean present_validate_cstr = true;
+    list.add(present_validate_cstr);
+    if (present_validate_cstr)
+      list.add(validate_cstr);
+
+    boolean present_rely_cstr = true;
+    list.add(present_rely_cstr);
+    if (present_rely_cstr)
+      list.add(rely_cstr);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SQLForeignKey other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetPktable_db()).compareTo(other.isSetPktable_db());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPktable_db()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pktable_db, other.pktable_db);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPktable_name()).compareTo(other.isSetPktable_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPktable_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pktable_name, other.pktable_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPkcolumn_name()).compareTo(other.isSetPkcolumn_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPkcolumn_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pkcolumn_name, other.pkcolumn_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFktable_db()).compareTo(other.isSetFktable_db());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFktable_db()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fktable_db, other.fktable_db);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFktable_name()).compareTo(other.isSetFktable_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFktable_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fktable_name, other.fktable_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFkcolumn_name()).compareTo(other.isSetFkcolumn_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFkcolumn_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fkcolumn_name, other.fkcolumn_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetKey_seq()).compareTo(other.isSetKey_seq());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetKey_seq()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.key_seq, other.key_seq);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetUpdate_rule()).compareTo(other.isSetUpdate_rule());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUpdate_rule()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.update_rule, other.update_rule);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDelete_rule()).compareTo(other.isSetDelete_rule());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDelete_rule()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delete_rule, other.delete_rule);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFk_name()).compareTo(other.isSetFk_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFk_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fk_name, other.fk_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPk_name()).compareTo(other.isSetPk_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPk_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pk_name, other.pk_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetEnable_cstr()).compareTo(other.isSetEnable_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEnable_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enable_cstr, other.enable_cstr);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetValidate_cstr()).compareTo(other.isSetValidate_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetValidate_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.validate_cstr, other.validate_cstr);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetRely_cstr()).compareTo(other.isSetRely_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetRely_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rely_cstr, other.rely_cstr);
+      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("SQLForeignKey(");
+    boolean first = true;
+
+    sb.append("pktable_db:");
+    if (this.pktable_db == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pktable_db);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("pktable_name:");
+    if (this.pktable_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pktable_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("pkcolumn_name:");
+    if (this.pkcolumn_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pkcolumn_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("fktable_db:");
+    if (this.fktable_db == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fktable_db);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("fktable_name:");
+    if (this.fktable_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fktable_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("fkcolumn_name:");
+    if (this.fkcolumn_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fkcolumn_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("key_seq:");
+    sb.append(this.key_seq);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("update_rule:");
+    sb.append(this.update_rule);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("delete_rule:");
+    sb.append(this.delete_rule);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("fk_name:");
+    if (this.fk_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fk_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("pk_name:");
+    if (this.pk_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pk_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("enable_cstr:");
+    sb.append(this.enable_cstr);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("validate_cstr:");
+    sb.append(this.validate_cstr);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("rely_cstr:");
+    sb.append(this.rely_cstr);
+    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 {
+      // 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 SQLForeignKeyStandardSchemeFactory implements SchemeFactory {
+    public SQLForeignKeyStandardScheme getScheme() {
+      return new SQLForeignKeyStandardScheme();
+    }
+  }
+
+  private static class SQLForeignKeyStandardScheme extends StandardScheme<SQLForeignKey> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SQLForeignKey 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: // PKTABLE_DB
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.pktable_db = iprot.readString();
+              struct.setPktable_dbIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PKTABLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.pktable_name = iprot.readString();
+              struct.setPktable_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // PKCOLUMN_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.pkcolumn_name = iprot.readString();
+              struct.setPkcolumn_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // FKTABLE_DB
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.fktable_db = iprot.readString();
+              struct.setFktable_dbIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // FKTABLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.fktable_name = iprot.readString();
+              struct.setFktable_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // FKCOLUMN_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.fkcolumn_name = iprot.readString();
+              struct.setFkcolumn_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // KEY_SEQ
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.key_seq = iprot.readI32();
+              struct.setKey_seqIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // UPDATE_RULE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.update_rule = iprot.readI32();
+              struct.setUpdate_ruleIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // DELETE_RULE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.delete_rule = iprot.readI32();
+              struct.setDelete_ruleIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 10: // FK_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.fk_name = iprot.readString();
+              struct.setFk_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 11: // PK_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.pk_name = iprot.readString();
+              struct.setPk_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 12: // ENABLE_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.enable_cstr = iprot.readBool();
+              struct.setEnable_cstrIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 13: // VALIDATE_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.validate_cstr = iprot.readBool();
+              struct.setValidate_cstrIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 14: // RELY_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.rely_cstr = iprot.readBool();
+              struct.setRely_cstrIsSet(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, SQLForeignKey struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.pktable_db != null) {
+        oprot.writeFieldBegin(PKTABLE_DB_FIELD_DESC);
+        oprot.writeString(struct.pktable_db);
+        oprot.writeFieldEnd();
+      }
+      if (struct.pktable_name != null) {
+        oprot.writeFieldBegin(PKTABLE_NAME_FIELD_DESC);
+        oprot.writeString(struct.pktable_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.pkcolumn_name != null) {
+        oprot.writeFieldBegin(PKCOLUMN_NAME_FIELD_DESC);
+        oprot.writeString(struct.pkcolumn_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.fktable_db != null) {
+        oprot.writeFieldBegin(FKTABLE_DB_FIELD_DESC);
+        oprot.writeString(struct.fktable_db);
+        oprot.writeFieldEnd();
+      }
+      if (struct.fktable_name != null) {
+        oprot.writeFieldBegin(FKTABLE_NAME_FIELD_DESC);
+        oprot.writeString(struct.fktable_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.fkcolumn_name != null) {
+        oprot.writeFieldBegin(FKCOLUMN_NAME_FIELD_DESC);
+        oprot.writeString(struct.fkcolumn_name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(KEY_SEQ_FIELD_DESC);
+      oprot.writeI32(struct.key_seq);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(UPDATE_RULE_FIELD_DESC);
+      oprot.writeI32(struct.update_rule);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(DELETE_RULE_FIELD_DESC);
+      oprot.writeI32(struct.delete_rule);
+      oprot.writeFieldEnd();
+      if (struct.fk_name != null) {
+        oprot.writeFieldBegin(FK_NAME_FIELD_DESC);
+        oprot.writeString(struct.fk_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.pk_name != null) {
+        oprot.writeFieldBegin(PK_NAME_FIELD_DESC);
+        oprot.writeString(struct.pk_name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(ENABLE_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.enable_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(VALIDATE_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.validate_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(RELY_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.rely_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SQLForeignKeyTupleSchemeFactory implements SchemeFactory {
+    public SQLForeignKeyTupleScheme getScheme() {
+      return new SQLForeignKeyTupleScheme();
+    }
+  }
+
+  private static class SQLForeignKeyTupleScheme extends TupleScheme<SQLForeignKey> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SQLForeignKey struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetPktable_db()) {
+        optionals.set(0);
+      }
+      if (struct.isSetPktable_name()) {
+        optionals.set(1);
+      }
+      if (struct.isSetPkcolumn_name()) {
+        optionals.set(2);
+      }
+      if (struct.isSetFktable_db()) {
+        optionals.set(3);
+      }
+      if (struct.isSetFktable_name()) {
+        optionals.set(4);
+      }
+      if (struct.isSetFkcolumn_name()) {
+        optionals.set(5);
+      }
+      if (struct.isSetKey_seq()) {
+        optionals.set(6);
+      }
+      if (struct.isSetUpdate_rule()) {
+        optionals.set(7);
+      }
+      if (struct.isSetDelete_rule()) {
+        optionals.set(8);
+      }
+      if (struct.isSetFk_name()) {
+        optionals.set(9);
+      }
+      if (struct.isSetPk_name()) {
+        optionals.set(10);
+      }
+      if (struct.isSetEnable_cstr()) {
+        optionals.set(11);
+      }
+      if (struct.isSetValidate_cstr()) {
+        optionals.set(12);
+      }
+      if (struct.isSetRely_cstr()) {
+        optionals.set(13);
+      }
+      oprot.writeBitSet(optionals, 14);
+      if (struct.isSetPktable_db()) {
+        oprot.writeString(struct.pktable_db);
+      }
+      if (struct.isSetPktable_name()) {
+        oprot.writeString(struct.pktable_name);
+      }
+      if (struct.isSetPkcolumn_name()) {
+        oprot.writeString(struct.pkcolumn_name);
+      }
+      if (struct.isSetFktable_db()) {
+        oprot.writeString(struct.fktable_db);
+      }
+      if (struct.isSetFktable_name()) {
+        oprot.writeString(struct.fktable_name);
+      }
+      if (struct.isSetFkcolumn_name()) {
+        oprot.writeString(struct.fkcolumn_name);
+      }
+      if (struct.isSetKey_seq()) {
+        oprot.writeI32(struct.key_seq);
+      }
+      if (struct.isSetUpdate_rule()) {
+        oprot.writeI32(struct.update_rule);
+      }
+      if (struct.isSetDelete_rule()) {
+        oprot.writeI32(struct.delete_rule);
+      }
+      if (struct.isSetFk_name()) {
+        oprot.writeString(struct.fk_name);
+      }
+      if (struct.isSetPk_name()) {
+        oprot.writeString(struct.pk_name);
+      }
+      if (struct.isSetEnable_cstr()) {
+        oprot.writeBool(struct.enable_cstr);
+      }
+      if (struct.isSetValidate_cstr()) {
+        oprot.writeBool(struct.validate_cstr);
+      }
+      if (struct.isSetRely_cstr()) {
+        oprot.writeBool(struct.rely_cstr);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SQLForeignKey struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(14);
+      if (incoming.get(0)) {
+        struct.pktable_db = iprot.readString();
+        struct.setPktable_dbIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.pktable_name = iprot.readString();
+        struct.setPktable_nameIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.pkcolumn_name = iprot.readString();
+        struct.setPkcolumn_nameIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.fktable_db = iprot.readString();
+        struct.setFktable_dbIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.fktable_name = iprot.readString();
+        struct.setFktable_nameIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.fkcolumn_name = iprot.readString();
+        struct.setFkcolumn_nameIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.key_seq = iprot.readI32();
+        struct.setKey_seqIsSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.update_rule = iprot.readI32();
+        struct.setUpdate_ruleIsSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.delete_rule = iprot.readI32();
+        struct.setDelete_ruleIsSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.fk_name = iprot.readString();
+        struct.setFk_nameIsSet(true);
+      }
+      if (incoming.get(10)) {
+        struct.pk_name = iprot.readString();
+        struct.setPk_nameIsSet(true);
+      }
+      if (incoming.get(11)) {
+        struct.enable_cstr = iprot.readBool();
+        struct.setEnable_cstrIsSet(true);
+      }
+      if (incoming.get(12)) {
+        struct.validate_cstr = iprot.readBool();
+        struct.setValidate_cstrIsSet(true);
+      }
+      if (incoming.get(13)) {
+        struct.rely_cstr = iprot.readBool();
+        struct.setRely_cstrIsSet(true);
+      }
+    }
+  }
+
+}
+


[29/30] hive git commit: Revert "HIVE-13507. Improved logging for ptest. (Siddharth Seth, reviewed by Thejas M Nair)"

Posted by jd...@apache.org.
Revert "HIVE-13507. Improved logging for ptest. (Siddharth Seth, reviewed by Thejas M Nair)"

This patch is causing Hive ptest to stay in a loop creating nodes due to an error exception.

This reverts commit fcc2e79511bb1a0db70f4a17ede6ade5e73b1f42.


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

Branch: refs/heads/llap
Commit: 9b5eb45b83d29adf0eb83f65b8821384d9254985
Parents: 6c3d3b4
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed Apr 20 13:36:31 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed Apr 20 13:36:31 2016 -0500

----------------------------------------------------------------------
 .../hive/ptest/execution/ExecutionPhase.java    |  2 -
 .../hive/ptest/execution/HostExecutor.java      | 48 ++------------------
 .../hive/ptest/execution/LocalCommand.java      | 18 +-------
 .../apache/hive/ptest/execution/PrepPhase.java  |  1 -
 .../apache/hive/ptest/execution/conf/Host.java  |  3 --
 5 files changed, 7 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9b5eb45b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
index 6063afc..3026ea0 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
@@ -86,8 +86,6 @@ public class ExecutionPhase extends Phase {
         isolatedWorkQueue.add(batch);
       }
     }
-    logger.info("ParallelWorkQueueSize={}, IsolatedWorkQueueSize={}", parallelWorkQueue.size(),
-        isolatedWorkQueue.size());
     try {
       int expectedNumHosts = hostExecutors.size();
       initalizeHosts();

http://git-wip-us.apache.org/repos/asf/hive/blob/9b5eb45b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
index 735b261..b05d2c2 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
@@ -29,7 +29,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.base.Stopwatch;
 import org.apache.hive.ptest.execution.conf.Host;
 import org.apache.hive.ptest.execution.conf.TestBatch;
 import org.apache.hive.ptest.execution.ssh.RSyncCommand;
@@ -66,8 +65,6 @@ class HostExecutor {
   private final File mFailedTestLogDir;
   private final long mNumPollSeconds;
   private volatile boolean mShutdown;
-  private int numParallelBatchesProcessed = 0;
-  private int numIsolatedBatchesProcessed = 0;
   
   HostExecutor(Host host, String privateKey, ListeningExecutorService executor,
       SSHCommandExecutor sshCommandExecutor,
@@ -103,18 +100,7 @@ class HostExecutor {
     return mExecutor.submit(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        Stopwatch stopwatch = Stopwatch.createStarted();
-        mLogger.info("Starting SubmitTests on host {}", getHost());
-        try {
-          executeTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults);
-        } finally {
-          stopwatch.stop();
-          mLogger.info("Finishing submitTests on host: {}. ElapsedTime(seconds)={}," +
-              " NumParallelBatchesProcessed={}, NumIsolatedBatchesProcessed={}",
-              new Object[]{getHost().toString(),
-                  stopwatch.elapsed(TimeUnit.SECONDS), numParallelBatchesProcessed,
-                  numIsolatedBatchesProcessed});
-        }
+        executeTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults);
         return null;
       }
 
@@ -157,7 +143,6 @@ class HostExecutor {
         @Override
         public Void call() throws Exception {
           TestBatch batch = null;
-          Stopwatch sw = Stopwatch.createUnstarted();
           try {
             do {
               batch = parallelWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
@@ -166,16 +151,8 @@ class HostExecutor {
                 return null;
               }
               if(batch != null) {
-                numParallelBatchesProcessed++;
-                sw.reset().start();
-                try {
-                  if (!executeTestBatch(drone, batch, failedTestResults)) {
-                    failedTestResults.add(batch);
-                  }
-                } finally {
-                  sw.stop();
-                  mLogger.info("Finished processing parallel batch [{}] on host {}. ElapsedTime(seconds)={}",
-                      new Object[]{batch.getName(), getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
+                if(!executeTestBatch(drone, batch, failedTestResults)) {
+                  failedTestResults.add(batch);
                 }
               }
             } while(!mShutdown && !parallelWorkQueue.isEmpty());
@@ -199,22 +176,12 @@ class HostExecutor {
     mLogger.info("Starting isolated execution on " + mHost.getName());
     for(Drone drone : ImmutableList.copyOf(mDrones)) {
       TestBatch batch = null;
-      Stopwatch sw = Stopwatch.createUnstarted();
       try {
         do {
-
           batch = isolatedWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
           if(batch != null) {
-            numIsolatedBatchesProcessed++;
-            sw.reset().start();
-            try {
-              if (!executeTestBatch(drone, batch, failedTestResults)) {
-                failedTestResults.add(batch);
-              }
-            } finally {
-              sw.stop();
-              mLogger.info("Finished processing isolated batch [{}] on host {}. ElapsedTime(seconds)={}",
-                  new Object[]{batch.getName(), getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
+            if(!executeTestBatch(drone, batch, failedTestResults)) {
+              failedTestResults.add(batch);
             }
           }
         } while(!mShutdown && !isolatedWorkQueue.isEmpty());
@@ -248,15 +215,10 @@ class HostExecutor {
     Templates.writeTemplateResult("batch-exec.vm", script, templateVariables);
     copyToDroneFromLocal(drone, script.getAbsolutePath(), "$localDir/$instanceName/scratch/" + scriptName);
     script.delete();
-    Stopwatch sw = Stopwatch.createStarted();
     mLogger.info(drone + " executing " + batch + " with " + command);
     RemoteCommandResult sshResult = new SSHCommand(mSSHCommandExecutor, drone.getPrivateKey(), drone.getUser(),
         drone.getHost(), drone.getInstance(), command, true).
         call();
-    sw.stop();
-    mLogger.info("Completed executing tests for batch [{}] on host {}. ElapsedTime(seconds)={}",
-        new Object[]{batch.getName(),
-            getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
     File batchLogDir = null;
     if(sshResult.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
       throw new AbortDroneException("Drone " + drone.toString() + " exited with " +

http://git-wip-us.apache.org/repos/asf/hive/blob/9b5eb45b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
index 8e2c5c7..ec99656 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
@@ -22,28 +22,17 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.base.Stopwatch;
 import org.slf4j.Logger;
 
 public class LocalCommand {
 
-  private static final AtomicInteger localCommandCounter = new AtomicInteger(0);
-
-  private final Logger logger;
   private final Process process;
   private final StreamReader streamReader;
   private Integer exitCode;
-  private final int commandId;
-  private final Stopwatch stopwatch = Stopwatch.createUnstarted();
 
   public LocalCommand(Logger logger, OutputPolicy outputPolicy, String command) throws IOException {
-    this.commandId = localCommandCounter.incrementAndGet();
-    this.logger = logger;
-    logger.info("Starting LocalCommandId={}: {}" + commandId, command);
-    stopwatch.start();
+    logger.info("Starting " + command);
     process = new ProcessBuilder().command(new String[] {"bash", "-c", command}).redirectErrorStream(true).start();
     streamReader = new StreamReader(outputPolicy, process.getInputStream());
     streamReader.setName("StreamReader-[" + command + "]");
@@ -56,13 +45,10 @@ public class LocalCommand {
       if(exitCode == null) {
         exitCode = process.waitFor();
       }
-      stopwatch.stop();
-      logger.info("Finished LocalCommandId={}. ElapsedTime(seconds)={}", commandId, stopwatch.elapsed(
-          TimeUnit.SECONDS));
       return exitCode;
     }
   }
-
+  
   public void kill() {
     synchronized (process) {
       process.destroy();

http://git-wip-us.apache.org/repos/asf/hive/blob/9b5eb45b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
index 8fef413..825f0c0 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
@@ -62,7 +62,6 @@ public class PrepPhase extends Phase {
     // source prep
     start = System.currentTimeMillis();
     File sourcePrepScript = new File(mScratchDir, "source-prep.sh");
-    logger.info("Writing {} from template", sourcePrepScript);
     Templates.writeTemplateResult("source-prep.vm", sourcePrepScript, getTemplateDefaults());
     execLocally("bash " + sourcePrepScript.getPath());
     logger.debug("Deleting " + sourcePrepScript + ": " + sourcePrepScript.delete());

http://git-wip-us.apache.org/repos/asf/hive/blob/9b5eb45b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
index a56824c..c1216c1 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
@@ -47,9 +47,6 @@ public class Host {
   public String[] getLocalDirectories() {
     return localDirectories;
   }
-  public String toShortString() {
-    return name;
-  }
   @Override
   public String toString() {
     return "Host [name=" + name + ", user=" + user + ", threads=" + threads


[02/30] hive git commit: Revert "HIVE-12159: Create vectorized readers for the complex types (Owen O'Malley, reviewed by Matt McCline)"

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java
index a52b3ef..adb52f0 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java
@@ -151,11 +151,12 @@ public class TestVectorizedORCReader {
         OrcFile.readerOptions(conf));
     RecordReaderImpl vrr = (RecordReaderImpl) vreader.rows();
     RecordReaderImpl rr = (RecordReaderImpl) reader.rows();
-    VectorizedRowBatch batch = reader.getSchema().createRowBatch();
+    VectorizedRowBatch batch = null;
     OrcStruct row = null;
 
     // Check Vectorized ORC reader against ORC row reader
-    while (vrr.nextBatch(batch)) {
+    while (vrr.hasNext()) {
+      batch = vrr.nextBatch(batch);
       for (int i = 0; i < batch.size; i++) {
         row = (OrcStruct) rr.next(row);
         for (int j = 0; j < batch.cols.length; j++) {
@@ -238,6 +239,6 @@ public class TestVectorizedORCReader {
       Assert.assertEquals(false, batch.cols[8].noNulls);
       Assert.assertEquals(false, batch.cols[9].noNulls);
     }
-    Assert.assertEquals(false, rr.nextBatch(batch));
+    Assert.assertEquals(false, rr.hasNext());
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
index f915a7e..99744cd 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
@@ -338,17 +338,6 @@ public class BytesColumnVector extends ColumnVector {
     initBuffer(0);
   }
 
-  public String toString(int row) {
-    if (isRepeating) {
-      row = 0;
-    }
-    if (noNulls || !isNull[row]) {
-      return new String(vector[row], start[row], length[row]);
-    } else {
-      return null;
-    }
-  }
-
   @Override
   public void stringifyValue(StringBuilder buffer, int row) {
     if (isRepeating) {

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
index d971339..c0dd5ed 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
@@ -392,4 +392,4 @@ public class TimestampColumnVector extends ColumnVector {
       buffer.append("null");
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java
index 0c61243..298d588 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
+import java.util.Arrays;
+
 /**
  * The representation of a vectorized column of struct objects.
  *


[23/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 97c07a5..d392f67 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -149,6 +149,10 @@ class Version;
 
 class FieldSchema;
 
+class SQLPrimaryKey;
+
+class SQLForeignKey;
+
 class Type;
 
 class HiveObjectRef;
@@ -239,6 +243,14 @@ class Schema;
 
 class EnvironmentContext;
 
+class PrimaryKeysRequest;
+
+class PrimaryKeysResponse;
+
+class ForeignKeysRequest;
+
+class ForeignKeysResponse;
+
 class PartitionsByExprResult;
 
 class PartitionsByExprRequest;
@@ -501,6 +513,218 @@ inline std::ostream& operator<<(std::ostream& out, const FieldSchema& obj)
   return out;
 }
 
+typedef struct _SQLPrimaryKey__isset {
+  _SQLPrimaryKey__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool table_db :1;
+  bool table_name :1;
+  bool column_name :1;
+  bool key_seq :1;
+  bool pk_name :1;
+  bool enable_cstr :1;
+  bool validate_cstr :1;
+  bool rely_cstr :1;
+} _SQLPrimaryKey__isset;
+
+class SQLPrimaryKey {
+ public:
+
+  SQLPrimaryKey(const SQLPrimaryKey&);
+  SQLPrimaryKey& operator=(const SQLPrimaryKey&);
+  SQLPrimaryKey() : table_db(), table_name(), column_name(), key_seq(0), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  }
+
+  virtual ~SQLPrimaryKey() throw();
+  std::string table_db;
+  std::string table_name;
+  std::string column_name;
+  int32_t key_seq;
+  std::string pk_name;
+  bool enable_cstr;
+  bool validate_cstr;
+  bool rely_cstr;
+
+  _SQLPrimaryKey__isset __isset;
+
+  void __set_table_db(const std::string& val);
+
+  void __set_table_name(const std::string& val);
+
+  void __set_column_name(const std::string& val);
+
+  void __set_key_seq(const int32_t val);
+
+  void __set_pk_name(const std::string& val);
+
+  void __set_enable_cstr(const bool val);
+
+  void __set_validate_cstr(const bool val);
+
+  void __set_rely_cstr(const bool val);
+
+  bool operator == (const SQLPrimaryKey & rhs) const
+  {
+    if (!(table_db == rhs.table_db))
+      return false;
+    if (!(table_name == rhs.table_name))
+      return false;
+    if (!(column_name == rhs.column_name))
+      return false;
+    if (!(key_seq == rhs.key_seq))
+      return false;
+    if (!(pk_name == rhs.pk_name))
+      return false;
+    if (!(enable_cstr == rhs.enable_cstr))
+      return false;
+    if (!(validate_cstr == rhs.validate_cstr))
+      return false;
+    if (!(rely_cstr == rhs.rely_cstr))
+      return false;
+    return true;
+  }
+  bool operator != (const SQLPrimaryKey &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const SQLPrimaryKey & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(SQLPrimaryKey &a, SQLPrimaryKey &b);
+
+inline std::ostream& operator<<(std::ostream& out, const SQLPrimaryKey& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _SQLForeignKey__isset {
+  _SQLForeignKey__isset() : pktable_db(false), pktable_name(false), pkcolumn_name(false), fktable_db(false), fktable_name(false), fkcolumn_name(false), key_seq(false), update_rule(false), delete_rule(false), fk_name(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool pktable_db :1;
+  bool pktable_name :1;
+  bool pkcolumn_name :1;
+  bool fktable_db :1;
+  bool fktable_name :1;
+  bool fkcolumn_name :1;
+  bool key_seq :1;
+  bool update_rule :1;
+  bool delete_rule :1;
+  bool fk_name :1;
+  bool pk_name :1;
+  bool enable_cstr :1;
+  bool validate_cstr :1;
+  bool rely_cstr :1;
+} _SQLForeignKey__isset;
+
+class SQLForeignKey {
+ public:
+
+  SQLForeignKey(const SQLForeignKey&);
+  SQLForeignKey& operator=(const SQLForeignKey&);
+  SQLForeignKey() : pktable_db(), pktable_name(), pkcolumn_name(), fktable_db(), fktable_name(), fkcolumn_name(), key_seq(0), update_rule(0), delete_rule(0), fk_name(), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  }
+
+  virtual ~SQLForeignKey() throw();
+  std::string pktable_db;
+  std::string pktable_name;
+  std::string pkcolumn_name;
+  std::string fktable_db;
+  std::string fktable_name;
+  std::string fkcolumn_name;
+  int32_t key_seq;
+  int32_t update_rule;
+  int32_t delete_rule;
+  std::string fk_name;
+  std::string pk_name;
+  bool enable_cstr;
+  bool validate_cstr;
+  bool rely_cstr;
+
+  _SQLForeignKey__isset __isset;
+
+  void __set_pktable_db(const std::string& val);
+
+  void __set_pktable_name(const std::string& val);
+
+  void __set_pkcolumn_name(const std::string& val);
+
+  void __set_fktable_db(const std::string& val);
+
+  void __set_fktable_name(const std::string& val);
+
+  void __set_fkcolumn_name(const std::string& val);
+
+  void __set_key_seq(const int32_t val);
+
+  void __set_update_rule(const int32_t val);
+
+  void __set_delete_rule(const int32_t val);
+
+  void __set_fk_name(const std::string& val);
+
+  void __set_pk_name(const std::string& val);
+
+  void __set_enable_cstr(const bool val);
+
+  void __set_validate_cstr(const bool val);
+
+  void __set_rely_cstr(const bool val);
+
+  bool operator == (const SQLForeignKey & rhs) const
+  {
+    if (!(pktable_db == rhs.pktable_db))
+      return false;
+    if (!(pktable_name == rhs.pktable_name))
+      return false;
+    if (!(pkcolumn_name == rhs.pkcolumn_name))
+      return false;
+    if (!(fktable_db == rhs.fktable_db))
+      return false;
+    if (!(fktable_name == rhs.fktable_name))
+      return false;
+    if (!(fkcolumn_name == rhs.fkcolumn_name))
+      return false;
+    if (!(key_seq == rhs.key_seq))
+      return false;
+    if (!(update_rule == rhs.update_rule))
+      return false;
+    if (!(delete_rule == rhs.delete_rule))
+      return false;
+    if (!(fk_name == rhs.fk_name))
+      return false;
+    if (!(pk_name == rhs.pk_name))
+      return false;
+    if (!(enable_cstr == rhs.enable_cstr))
+      return false;
+    if (!(validate_cstr == rhs.validate_cstr))
+      return false;
+    if (!(rely_cstr == rhs.rely_cstr))
+      return false;
+    return true;
+  }
+  bool operator != (const SQLForeignKey &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const SQLForeignKey & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(SQLForeignKey &a, SQLForeignKey &b);
+
+inline std::ostream& operator<<(std::ostream& out, const SQLForeignKey& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _Type__isset {
   _Type__isset() : name(false), type1(false), type2(false), fields(false) {}
   bool name :1;
@@ -3375,6 +3599,186 @@ inline std::ostream& operator<<(std::ostream& out, const EnvironmentContext& obj
 }
 
 
+class PrimaryKeysRequest {
+ public:
+
+  PrimaryKeysRequest(const PrimaryKeysRequest&);
+  PrimaryKeysRequest& operator=(const PrimaryKeysRequest&);
+  PrimaryKeysRequest() : db_name(), tbl_name() {
+  }
+
+  virtual ~PrimaryKeysRequest() throw();
+  std::string db_name;
+  std::string tbl_name;
+
+  void __set_db_name(const std::string& val);
+
+  void __set_tbl_name(const std::string& val);
+
+  bool operator == (const PrimaryKeysRequest & rhs) const
+  {
+    if (!(db_name == rhs.db_name))
+      return false;
+    if (!(tbl_name == rhs.tbl_name))
+      return false;
+    return true;
+  }
+  bool operator != (const PrimaryKeysRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const PrimaryKeysRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(PrimaryKeysRequest &a, PrimaryKeysRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const PrimaryKeysRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class PrimaryKeysResponse {
+ public:
+
+  PrimaryKeysResponse(const PrimaryKeysResponse&);
+  PrimaryKeysResponse& operator=(const PrimaryKeysResponse&);
+  PrimaryKeysResponse() {
+  }
+
+  virtual ~PrimaryKeysResponse() throw();
+  std::vector<SQLPrimaryKey>  primaryKeys;
+
+  void __set_primaryKeys(const std::vector<SQLPrimaryKey> & val);
+
+  bool operator == (const PrimaryKeysResponse & rhs) const
+  {
+    if (!(primaryKeys == rhs.primaryKeys))
+      return false;
+    return true;
+  }
+  bool operator != (const PrimaryKeysResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const PrimaryKeysResponse & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(PrimaryKeysResponse &a, PrimaryKeysResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const PrimaryKeysResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class ForeignKeysRequest {
+ public:
+
+  ForeignKeysRequest(const ForeignKeysRequest&);
+  ForeignKeysRequest& operator=(const ForeignKeysRequest&);
+  ForeignKeysRequest() : parent_db_name(), parent_tbl_name(), foreign_db_name(), foreign_tbl_name() {
+  }
+
+  virtual ~ForeignKeysRequest() throw();
+  std::string parent_db_name;
+  std::string parent_tbl_name;
+  std::string foreign_db_name;
+  std::string foreign_tbl_name;
+
+  void __set_parent_db_name(const std::string& val);
+
+  void __set_parent_tbl_name(const std::string& val);
+
+  void __set_foreign_db_name(const std::string& val);
+
+  void __set_foreign_tbl_name(const std::string& val);
+
+  bool operator == (const ForeignKeysRequest & rhs) const
+  {
+    if (!(parent_db_name == rhs.parent_db_name))
+      return false;
+    if (!(parent_tbl_name == rhs.parent_tbl_name))
+      return false;
+    if (!(foreign_db_name == rhs.foreign_db_name))
+      return false;
+    if (!(foreign_tbl_name == rhs.foreign_tbl_name))
+      return false;
+    return true;
+  }
+  bool operator != (const ForeignKeysRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ForeignKeysRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(ForeignKeysRequest &a, ForeignKeysRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const ForeignKeysRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class ForeignKeysResponse {
+ public:
+
+  ForeignKeysResponse(const ForeignKeysResponse&);
+  ForeignKeysResponse& operator=(const ForeignKeysResponse&);
+  ForeignKeysResponse() {
+  }
+
+  virtual ~ForeignKeysResponse() throw();
+  std::vector<SQLForeignKey>  foreignKeys;
+
+  void __set_foreignKeys(const std::vector<SQLForeignKey> & val);
+
+  bool operator == (const ForeignKeysResponse & rhs) const
+  {
+    if (!(foreignKeys == rhs.foreignKeys))
+      return false;
+    return true;
+  }
+  bool operator != (const ForeignKeysResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ForeignKeysResponse & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(ForeignKeysResponse &a, ForeignKeysResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const ForeignKeysResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class PartitionsByExprResult {
  public:
 

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 bb6e584..19bdf10 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
@@ -630,13 +630,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
           case 4: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list492 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list492.size);
-                String _elem493;
-                for (int _i494 = 0; _i494 < _list492.size; ++_i494)
+                org.apache.thrift.protocol.TList _list508 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list508.size);
+                String _elem509;
+                for (int _i510 = 0; _i510 < _list508.size; ++_i510)
                 {
-                  _elem493 = iprot.readString();
-                  struct.partitionnames.add(_elem493);
+                  _elem509 = iprot.readString();
+                  struct.partitionnames.add(_elem509);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter495 : struct.partitionnames)
+          for (String _iter511 : struct.partitionnames)
           {
-            oprot.writeString(_iter495);
+            oprot.writeString(_iter511);
           }
           oprot.writeListEnd();
         }
@@ -705,9 +705,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter496 : struct.partitionnames)
+        for (String _iter512 : struct.partitionnames)
         {
-          oprot.writeString(_iter496);
+          oprot.writeString(_iter512);
         }
       }
     }
@@ -722,13 +722,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list497 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list497.size);
-        String _elem498;
-        for (int _i499 = 0; _i499 < _list497.size; ++_i499)
+        org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list513.size);
+        String _elem514;
+        for (int _i515 = 0; _i515 < _list513.size; ++_i515)
         {
-          _elem498 = iprot.readString();
-          struct.partitionnames.add(_elem498);
+          _elem514 = iprot.readString();
+          struct.partitionnames.add(_elem514);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 083d340..6df6fa5 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
@@ -704,14 +704,14 @@ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitio
           case 3: // PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list388 = iprot.readListBegin();
-                struct.parts = new ArrayList<Partition>(_list388.size);
-                Partition _elem389;
-                for (int _i390 = 0; _i390 < _list388.size; ++_i390)
+                org.apache.thrift.protocol.TList _list404 = iprot.readListBegin();
+                struct.parts = new ArrayList<Partition>(_list404.size);
+                Partition _elem405;
+                for (int _i406 = 0; _i406 < _list404.size; ++_i406)
                 {
-                  _elem389 = new Partition();
-                  _elem389.read(iprot);
-                  struct.parts.add(_elem389);
+                  _elem405 = new Partition();
+                  _elem405.read(iprot);
+                  struct.parts.add(_elem405);
                 }
                 iprot.readListEnd();
               }
@@ -763,9 +763,9 @@ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitio
         oprot.writeFieldBegin(PARTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.parts.size()));
-          for (Partition _iter391 : struct.parts)
+          for (Partition _iter407 : struct.parts)
           {
-            _iter391.write(oprot);
+            _iter407.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -800,9 +800,9 @@ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitio
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.parts.size());
-        for (Partition _iter392 : struct.parts)
+        for (Partition _iter408 : struct.parts)
         {
-          _iter392.write(oprot);
+          _iter408.write(oprot);
         }
       }
       oprot.writeBool(struct.ifNotExists);
@@ -824,14 +824,14 @@ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitio
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list393 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.parts = new ArrayList<Partition>(_list393.size);
-        Partition _elem394;
-        for (int _i395 = 0; _i395 < _list393.size; ++_i395)
+        org.apache.thrift.protocol.TList _list409 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.parts = new ArrayList<Partition>(_list409.size);
+        Partition _elem410;
+        for (int _i411 = 0; _i411 < _list409.size; ++_i411)
         {
-          _elem394 = new Partition();
-          _elem394.read(iprot);
-          struct.parts.add(_elem394);
+          _elem410 = new Partition();
+          _elem410.read(iprot);
+          struct.parts.add(_elem410);
         }
       }
       struct.setPartsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 9004457..521ed38 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
@@ -346,14 +346,14 @@ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartition
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list380 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list380.size);
-                Partition _elem381;
-                for (int _i382 = 0; _i382 < _list380.size; ++_i382)
+                org.apache.thrift.protocol.TList _list396 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list396.size);
+                Partition _elem397;
+                for (int _i398 = 0; _i398 < _list396.size; ++_i398)
                 {
-                  _elem381 = new Partition();
-                  _elem381.read(iprot);
-                  struct.partitions.add(_elem381);
+                  _elem397 = new Partition();
+                  _elem397.read(iprot);
+                  struct.partitions.add(_elem397);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartition
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter383 : struct.partitions)
+            for (Partition _iter399 : struct.partitions)
             {
-              _iter383.write(oprot);
+              _iter399.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartition
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter384 : struct.partitions)
+          for (Partition _iter400 : struct.partitions)
           {
-            _iter384.write(oprot);
+            _iter400.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartition
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list385 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list385.size);
-          Partition _elem386;
-          for (int _i387 = 0; _i387 < _list385.size; ++_i387)
+          org.apache.thrift.protocol.TList _list401 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list401.size);
+          Partition _elem402;
+          for (int _i403 = 0; _i403 < _list401.size; ++_i403)
           {
-            _elem386 = new Partition();
-            _elem386.read(iprot);
-            struct.partitions.add(_elem386);
+            _elem402 = new Partition();
+            _elem402.read(iprot);
+            struct.partitions.add(_elem402);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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
index 657bb7b..cfec32e 100644
--- 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
@@ -351,13 +351,13 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list576 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list576.size);
-                long _elem577;
-                for (int _i578 = 0; _i578 < _list576.size; ++_i578)
+                org.apache.thrift.protocol.TList _list592 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list592.size);
+                long _elem593;
+                for (int _i594 = 0; _i594 < _list592.size; ++_i594)
                 {
-                  _elem577 = iprot.readI64();
-                  struct.fileIds.add(_elem577);
+                  _elem593 = iprot.readI64();
+                  struct.fileIds.add(_elem593);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
         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 _iter579 : struct.fileIds)
+          for (long _iter595 : struct.fileIds)
           {
-            oprot.writeI64(_iter579);
+            oprot.writeI64(_iter595);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter580 : struct.fileIds)
+        for (long _iter596 : struct.fileIds)
         {
-          oprot.writeI64(_iter580);
+          oprot.writeI64(_iter596);
         }
       }
     }
@@ -421,13 +421,13 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
     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 _list581 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list581.size);
-        long _elem582;
-        for (int _i583 = 0; _i583 < _list581.size; ++_i583)
+        org.apache.thrift.protocol.TList _list597 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list597.size);
+        long _elem598;
+        for (int _i599 = 0; _i599 < _list597.size; ++_i599)
         {
-          _elem582 = iprot.readI64();
-          struct.fileIds.add(_elem582);
+          _elem598 = iprot.readI64();
+          struct.fileIds.add(_elem598);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 adef415..a70730a 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
@@ -346,14 +346,14 @@ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartiti
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list396 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list396.size);
-                Partition _elem397;
-                for (int _i398 = 0; _i398 < _list396.size; ++_i398)
+                org.apache.thrift.protocol.TList _list412 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list412.size);
+                Partition _elem413;
+                for (int _i414 = 0; _i414 < _list412.size; ++_i414)
                 {
-                  _elem397 = new Partition();
-                  _elem397.read(iprot);
-                  struct.partitions.add(_elem397);
+                  _elem413 = new Partition();
+                  _elem413.read(iprot);
+                  struct.partitions.add(_elem413);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartiti
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter399 : struct.partitions)
+            for (Partition _iter415 : struct.partitions)
             {
-              _iter399.write(oprot);
+              _iter415.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartiti
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter400 : struct.partitions)
+          for (Partition _iter416 : struct.partitions)
           {
-            _iter400.write(oprot);
+            _iter416.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartiti
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list401 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list401.size);
-          Partition _elem402;
-          for (int _i403 = 0; _i403 < _list401.size; ++_i403)
+          org.apache.thrift.protocol.TList _list417 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list417.size);
+          Partition _elem418;
+          for (int _i419 = 0; _i419 < _list417.size; ++_i419)
           {
-            _elem402 = new Partition();
-            _elem402.read(iprot);
-            struct.partitions.add(_elem402);
+            _elem418 = new Partition();
+            _elem418.read(iprot);
+            struct.partitions.add(_elem418);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 6b08234..44308cc 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
@@ -713,13 +713,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list516 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list516.size);
-                String _elem517;
-                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
+                org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list532.size);
+                String _elem533;
+                for (int _i534 = 0; _i534 < _list532.size; ++_i534)
                 {
-                  _elem517 = iprot.readString();
-                  struct.partitionVals.add(_elem517);
+                  _elem533 = iprot.readString();
+                  struct.partitionVals.add(_elem533);
                 }
                 iprot.readListEnd();
               }
@@ -768,9 +768,9 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter519 : struct.partitionVals)
+            for (String _iter535 : struct.partitionVals)
             {
-              oprot.writeString(_iter519);
+              oprot.writeString(_iter535);
             }
             oprot.writeListEnd();
           }
@@ -816,9 +816,9 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter520 : struct.partitionVals)
+          for (String _iter536 : struct.partitionVals)
           {
-            oprot.writeString(_iter520);
+            oprot.writeString(_iter536);
           }
         }
       }
@@ -843,13 +843,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list521.size);
-          String _elem522;
-          for (int _i523 = 0; _i523 < _list521.size; ++_i523)
+          org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list537.size);
+          String _elem538;
+          for (int _i539 = 0; _i539 < _list537.size; ++_i539)
           {
-            _elem522 = iprot.readString();
-            struct.partitionVals.add(_elem522);
+            _elem538 = iprot.readString();
+            struct.partitionVals.add(_elem538);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java
new file mode 100644
index 0000000..7788780
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java
@@ -0,0 +1,692 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * 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.3)")
+public class ForeignKeysRequest implements org.apache.thrift.TBase<ForeignKeysRequest, ForeignKeysRequest._Fields>, java.io.Serializable, Cloneable, Comparable<ForeignKeysRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ForeignKeysRequest");
+
+  private static final org.apache.thrift.protocol.TField PARENT_DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("parent_db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PARENT_TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("parent_tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField FOREIGN_DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("foreign_db_name", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField FOREIGN_TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("foreign_tbl_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ForeignKeysRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ForeignKeysRequestTupleSchemeFactory());
+  }
+
+  private String parent_db_name; // required
+  private String parent_tbl_name; // required
+  private String foreign_db_name; // required
+  private String foreign_tbl_name; // 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 {
+    PARENT_DB_NAME((short)1, "parent_db_name"),
+    PARENT_TBL_NAME((short)2, "parent_tbl_name"),
+    FOREIGN_DB_NAME((short)3, "foreign_db_name"),
+    FOREIGN_TBL_NAME((short)4, "foreign_tbl_name");
+
+    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: // PARENT_DB_NAME
+          return PARENT_DB_NAME;
+        case 2: // PARENT_TBL_NAME
+          return PARENT_TBL_NAME;
+        case 3: // FOREIGN_DB_NAME
+          return FOREIGN_DB_NAME;
+        case 4: // FOREIGN_TBL_NAME
+          return FOREIGN_TBL_NAME;
+        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.PARENT_DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("parent_db_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PARENT_TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("parent_tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FOREIGN_DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("foreign_db_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FOREIGN_TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("foreign_tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ForeignKeysRequest.class, metaDataMap);
+  }
+
+  public ForeignKeysRequest() {
+  }
+
+  public ForeignKeysRequest(
+    String parent_db_name,
+    String parent_tbl_name,
+    String foreign_db_name,
+    String foreign_tbl_name)
+  {
+    this();
+    this.parent_db_name = parent_db_name;
+    this.parent_tbl_name = parent_tbl_name;
+    this.foreign_db_name = foreign_db_name;
+    this.foreign_tbl_name = foreign_tbl_name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ForeignKeysRequest(ForeignKeysRequest other) {
+    if (other.isSetParent_db_name()) {
+      this.parent_db_name = other.parent_db_name;
+    }
+    if (other.isSetParent_tbl_name()) {
+      this.parent_tbl_name = other.parent_tbl_name;
+    }
+    if (other.isSetForeign_db_name()) {
+      this.foreign_db_name = other.foreign_db_name;
+    }
+    if (other.isSetForeign_tbl_name()) {
+      this.foreign_tbl_name = other.foreign_tbl_name;
+    }
+  }
+
+  public ForeignKeysRequest deepCopy() {
+    return new ForeignKeysRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.parent_db_name = null;
+    this.parent_tbl_name = null;
+    this.foreign_db_name = null;
+    this.foreign_tbl_name = null;
+  }
+
+  public String getParent_db_name() {
+    return this.parent_db_name;
+  }
+
+  public void setParent_db_name(String parent_db_name) {
+    this.parent_db_name = parent_db_name;
+  }
+
+  public void unsetParent_db_name() {
+    this.parent_db_name = null;
+  }
+
+  /** Returns true if field parent_db_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetParent_db_name() {
+    return this.parent_db_name != null;
+  }
+
+  public void setParent_db_nameIsSet(boolean value) {
+    if (!value) {
+      this.parent_db_name = null;
+    }
+  }
+
+  public String getParent_tbl_name() {
+    return this.parent_tbl_name;
+  }
+
+  public void setParent_tbl_name(String parent_tbl_name) {
+    this.parent_tbl_name = parent_tbl_name;
+  }
+
+  public void unsetParent_tbl_name() {
+    this.parent_tbl_name = null;
+  }
+
+  /** Returns true if field parent_tbl_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetParent_tbl_name() {
+    return this.parent_tbl_name != null;
+  }
+
+  public void setParent_tbl_nameIsSet(boolean value) {
+    if (!value) {
+      this.parent_tbl_name = null;
+    }
+  }
+
+  public String getForeign_db_name() {
+    return this.foreign_db_name;
+  }
+
+  public void setForeign_db_name(String foreign_db_name) {
+    this.foreign_db_name = foreign_db_name;
+  }
+
+  public void unsetForeign_db_name() {
+    this.foreign_db_name = null;
+  }
+
+  /** Returns true if field foreign_db_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetForeign_db_name() {
+    return this.foreign_db_name != null;
+  }
+
+  public void setForeign_db_nameIsSet(boolean value) {
+    if (!value) {
+      this.foreign_db_name = null;
+    }
+  }
+
+  public String getForeign_tbl_name() {
+    return this.foreign_tbl_name;
+  }
+
+  public void setForeign_tbl_name(String foreign_tbl_name) {
+    this.foreign_tbl_name = foreign_tbl_name;
+  }
+
+  public void unsetForeign_tbl_name() {
+    this.foreign_tbl_name = null;
+  }
+
+  /** Returns true if field foreign_tbl_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetForeign_tbl_name() {
+    return this.foreign_tbl_name != null;
+  }
+
+  public void setForeign_tbl_nameIsSet(boolean value) {
+    if (!value) {
+      this.foreign_tbl_name = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PARENT_DB_NAME:
+      if (value == null) {
+        unsetParent_db_name();
+      } else {
+        setParent_db_name((String)value);
+      }
+      break;
+
+    case PARENT_TBL_NAME:
+      if (value == null) {
+        unsetParent_tbl_name();
+      } else {
+        setParent_tbl_name((String)value);
+      }
+      break;
+
+    case FOREIGN_DB_NAME:
+      if (value == null) {
+        unsetForeign_db_name();
+      } else {
+        setForeign_db_name((String)value);
+      }
+      break;
+
+    case FOREIGN_TBL_NAME:
+      if (value == null) {
+        unsetForeign_tbl_name();
+      } else {
+        setForeign_tbl_name((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PARENT_DB_NAME:
+      return getParent_db_name();
+
+    case PARENT_TBL_NAME:
+      return getParent_tbl_name();
+
+    case FOREIGN_DB_NAME:
+      return getForeign_db_name();
+
+    case FOREIGN_TBL_NAME:
+      return getForeign_tbl_name();
+
+    }
+    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 PARENT_DB_NAME:
+      return isSetParent_db_name();
+    case PARENT_TBL_NAME:
+      return isSetParent_tbl_name();
+    case FOREIGN_DB_NAME:
+      return isSetForeign_db_name();
+    case FOREIGN_TBL_NAME:
+      return isSetForeign_tbl_name();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ForeignKeysRequest)
+      return this.equals((ForeignKeysRequest)that);
+    return false;
+  }
+
+  public boolean equals(ForeignKeysRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_parent_db_name = true && this.isSetParent_db_name();
+    boolean that_present_parent_db_name = true && that.isSetParent_db_name();
+    if (this_present_parent_db_name || that_present_parent_db_name) {
+      if (!(this_present_parent_db_name && that_present_parent_db_name))
+        return false;
+      if (!this.parent_db_name.equals(that.parent_db_name))
+        return false;
+    }
+
+    boolean this_present_parent_tbl_name = true && this.isSetParent_tbl_name();
+    boolean that_present_parent_tbl_name = true && that.isSetParent_tbl_name();
+    if (this_present_parent_tbl_name || that_present_parent_tbl_name) {
+      if (!(this_present_parent_tbl_name && that_present_parent_tbl_name))
+        return false;
+      if (!this.parent_tbl_name.equals(that.parent_tbl_name))
+        return false;
+    }
+
+    boolean this_present_foreign_db_name = true && this.isSetForeign_db_name();
+    boolean that_present_foreign_db_name = true && that.isSetForeign_db_name();
+    if (this_present_foreign_db_name || that_present_foreign_db_name) {
+      if (!(this_present_foreign_db_name && that_present_foreign_db_name))
+        return false;
+      if (!this.foreign_db_name.equals(that.foreign_db_name))
+        return false;
+    }
+
+    boolean this_present_foreign_tbl_name = true && this.isSetForeign_tbl_name();
+    boolean that_present_foreign_tbl_name = true && that.isSetForeign_tbl_name();
+    if (this_present_foreign_tbl_name || that_present_foreign_tbl_name) {
+      if (!(this_present_foreign_tbl_name && that_present_foreign_tbl_name))
+        return false;
+      if (!this.foreign_tbl_name.equals(that.foreign_tbl_name))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_parent_db_name = true && (isSetParent_db_name());
+    list.add(present_parent_db_name);
+    if (present_parent_db_name)
+      list.add(parent_db_name);
+
+    boolean present_parent_tbl_name = true && (isSetParent_tbl_name());
+    list.add(present_parent_tbl_name);
+    if (present_parent_tbl_name)
+      list.add(parent_tbl_name);
+
+    boolean present_foreign_db_name = true && (isSetForeign_db_name());
+    list.add(present_foreign_db_name);
+    if (present_foreign_db_name)
+      list.add(foreign_db_name);
+
+    boolean present_foreign_tbl_name = true && (isSetForeign_tbl_name());
+    list.add(present_foreign_tbl_name);
+    if (present_foreign_tbl_name)
+      list.add(foreign_tbl_name);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ForeignKeysRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetParent_db_name()).compareTo(other.isSetParent_db_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetParent_db_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parent_db_name, other.parent_db_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetParent_tbl_name()).compareTo(other.isSetParent_tbl_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetParent_tbl_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parent_tbl_name, other.parent_tbl_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetForeign_db_name()).compareTo(other.isSetForeign_db_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetForeign_db_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.foreign_db_name, other.foreign_db_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetForeign_tbl_name()).compareTo(other.isSetForeign_tbl_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetForeign_tbl_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.foreign_tbl_name, other.foreign_tbl_name);
+      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("ForeignKeysRequest(");
+    boolean first = true;
+
+    sb.append("parent_db_name:");
+    if (this.parent_db_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.parent_db_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("parent_tbl_name:");
+    if (this.parent_tbl_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.parent_tbl_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("foreign_db_name:");
+    if (this.foreign_db_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.foreign_db_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("foreign_tbl_name:");
+    if (this.foreign_tbl_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.foreign_tbl_name);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetParent_db_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'parent_db_name' is unset! Struct:" + toString());
+    }
+
+    if (!isSetParent_tbl_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'parent_tbl_name' is unset! Struct:" + toString());
+    }
+
+    if (!isSetForeign_db_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'foreign_db_name' is unset! Struct:" + toString());
+    }
+
+    if (!isSetForeign_tbl_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'foreign_tbl_name' 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 ForeignKeysRequestStandardSchemeFactory implements SchemeFactory {
+    public ForeignKeysRequestStandardScheme getScheme() {
+      return new ForeignKeysRequestStandardScheme();
+    }
+  }
+
+  private static class ForeignKeysRequestStandardScheme extends StandardScheme<ForeignKeysRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ForeignKeysRequest 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: // PARENT_DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.parent_db_name = iprot.readString();
+              struct.setParent_db_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PARENT_TBL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.parent_tbl_name = iprot.readString();
+              struct.setParent_tbl_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // FOREIGN_DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.foreign_db_name = iprot.readString();
+              struct.setForeign_db_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // FOREIGN_TBL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.foreign_tbl_name = iprot.readString();
+              struct.setForeign_tbl_nameIsSet(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, ForeignKeysRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.parent_db_name != null) {
+        oprot.writeFieldBegin(PARENT_DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.parent_db_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.parent_tbl_name != null) {
+        oprot.writeFieldBegin(PARENT_TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.parent_tbl_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.foreign_db_name != null) {
+        oprot.writeFieldBegin(FOREIGN_DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.foreign_db_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.foreign_tbl_name != null) {
+        oprot.writeFieldBegin(FOREIGN_TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.foreign_tbl_name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ForeignKeysRequestTupleSchemeFactory implements SchemeFactory {
+    public ForeignKeysRequestTupleScheme getScheme() {
+      return new ForeignKeysRequestTupleScheme();
+    }
+  }
+
+  private static class ForeignKeysRequestTupleScheme extends TupleScheme<ForeignKeysRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ForeignKeysRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.parent_db_name);
+      oprot.writeString(struct.parent_tbl_name);
+      oprot.writeString(struct.foreign_db_name);
+      oprot.writeString(struct.foreign_tbl_name);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ForeignKeysRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.parent_db_name = iprot.readString();
+      struct.setParent_db_nameIsSet(true);
+      struct.parent_tbl_name = iprot.readString();
+      struct.setParent_tbl_nameIsSet(true);
+      struct.foreign_db_name = iprot.readString();
+      struct.setForeign_db_nameIsSet(true);
+      struct.foreign_tbl_name = iprot.readString();
+      struct.setForeign_tbl_nameIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
new file mode 100644
index 0000000..7135a65
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
@@ -0,0 +1,443 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * 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.3)")
+public class ForeignKeysResponse implements org.apache.thrift.TBase<ForeignKeysResponse, ForeignKeysResponse._Fields>, java.io.Serializable, Cloneable, Comparable<ForeignKeysResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ForeignKeysResponse");
+
+  private static final org.apache.thrift.protocol.TField FOREIGN_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("foreignKeys", 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 ForeignKeysResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ForeignKeysResponseTupleSchemeFactory());
+  }
+
+  private List<SQLForeignKey> foreignKeys; // 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 {
+    FOREIGN_KEYS((short)1, "foreignKeys");
+
+    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: // FOREIGN_KEYS
+          return FOREIGN_KEYS;
+        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.FOREIGN_KEYS, new org.apache.thrift.meta_data.FieldMetaData("foreignKeys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        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, SQLForeignKey.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ForeignKeysResponse.class, metaDataMap);
+  }
+
+  public ForeignKeysResponse() {
+  }
+
+  public ForeignKeysResponse(
+    List<SQLForeignKey> foreignKeys)
+  {
+    this();
+    this.foreignKeys = foreignKeys;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ForeignKeysResponse(ForeignKeysResponse other) {
+    if (other.isSetForeignKeys()) {
+      List<SQLForeignKey> __this__foreignKeys = new ArrayList<SQLForeignKey>(other.foreignKeys.size());
+      for (SQLForeignKey other_element : other.foreignKeys) {
+        __this__foreignKeys.add(new SQLForeignKey(other_element));
+      }
+      this.foreignKeys = __this__foreignKeys;
+    }
+  }
+
+  public ForeignKeysResponse deepCopy() {
+    return new ForeignKeysResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.foreignKeys = null;
+  }
+
+  public int getForeignKeysSize() {
+    return (this.foreignKeys == null) ? 0 : this.foreignKeys.size();
+  }
+
+  public java.util.Iterator<SQLForeignKey> getForeignKeysIterator() {
+    return (this.foreignKeys == null) ? null : this.foreignKeys.iterator();
+  }
+
+  public void addToForeignKeys(SQLForeignKey elem) {
+    if (this.foreignKeys == null) {
+      this.foreignKeys = new ArrayList<SQLForeignKey>();
+    }
+    this.foreignKeys.add(elem);
+  }
+
+  public List<SQLForeignKey> getForeignKeys() {
+    return this.foreignKeys;
+  }
+
+  public void setForeignKeys(List<SQLForeignKey> foreignKeys) {
+    this.foreignKeys = foreignKeys;
+  }
+
+  public void unsetForeignKeys() {
+    this.foreignKeys = null;
+  }
+
+  /** Returns true if field foreignKeys is set (has been assigned a value) and false otherwise */
+  public boolean isSetForeignKeys() {
+    return this.foreignKeys != null;
+  }
+
+  public void setForeignKeysIsSet(boolean value) {
+    if (!value) {
+      this.foreignKeys = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FOREIGN_KEYS:
+      if (value == null) {
+        unsetForeignKeys();
+      } else {
+        setForeignKeys((List<SQLForeignKey>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FOREIGN_KEYS:
+      return getForeignKeys();
+
+    }
+    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 FOREIGN_KEYS:
+      return isSetForeignKeys();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ForeignKeysResponse)
+      return this.equals((ForeignKeysResponse)that);
+    return false;
+  }
+
+  public boolean equals(ForeignKeysResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_foreignKeys = true && this.isSetForeignKeys();
+    boolean that_present_foreignKeys = true && that.isSetForeignKeys();
+    if (this_present_foreignKeys || that_present_foreignKeys) {
+      if (!(this_present_foreignKeys && that_present_foreignKeys))
+        return false;
+      if (!this.foreignKeys.equals(that.foreignKeys))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_foreignKeys = true && (isSetForeignKeys());
+    list.add(present_foreignKeys);
+    if (present_foreignKeys)
+      list.add(foreignKeys);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ForeignKeysResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetForeignKeys()).compareTo(other.isSetForeignKeys());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetForeignKeys()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.foreignKeys, other.foreignKeys);
+      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("ForeignKeysResponse(");
+    boolean first = true;
+
+    sb.append("foreignKeys:");
+    if (this.foreignKeys == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.foreignKeys);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetForeignKeys()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'foreignKeys' 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 ForeignKeysResponseStandardSchemeFactory implements SchemeFactory {
+    public ForeignKeysResponseStandardScheme getScheme() {
+      return new ForeignKeysResponseStandardScheme();
+    }
+  }
+
+  private static class ForeignKeysResponseStandardScheme extends StandardScheme<ForeignKeysResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ForeignKeysResponse 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: // FOREIGN_KEYS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list330 = iprot.readListBegin();
+                struct.foreignKeys = new ArrayList<SQLForeignKey>(_list330.size);
+                SQLForeignKey _elem331;
+                for (int _i332 = 0; _i332 < _list330.size; ++_i332)
+                {
+                  _elem331 = new SQLForeignKey();
+                  _elem331.read(iprot);
+                  struct.foreignKeys.add(_elem331);
+                }
+                iprot.readListEnd();
+              }
+              struct.setForeignKeysIsSet(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, ForeignKeysResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.foreignKeys != null) {
+        oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
+          for (SQLForeignKey _iter333 : struct.foreignKeys)
+          {
+            _iter333.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ForeignKeysResponseTupleSchemeFactory implements SchemeFactory {
+    public ForeignKeysResponseTupleScheme getScheme() {
+      return new ForeignKeysResponseTupleScheme();
+    }
+  }
+
+  private static class ForeignKeysResponseTupleScheme extends TupleScheme<ForeignKeysResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ForeignKeysResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.foreignKeys.size());
+        for (SQLForeignKey _iter334 : struct.foreignKeys)
+        {
+          _iter334.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ForeignKeysResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list335 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.foreignKeys = new ArrayList<SQLForeignKey>(_list335.size);
+        SQLForeignKey _elem336;
+        for (int _i337 = 0; _i337 < _list335.size; ++_i337)
+        {
+          _elem336 = new SQLForeignKey();
+          _elem336.read(iprot);
+          struct.foreignKeys.add(_elem336);
+        }
+      }
+      struct.setForeignKeysIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 5f8ce0d..5cf2f59 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
@@ -997,14 +997,14 @@ public class Function implements org.apache.thrift.TBase<Function, Function._Fie
           case 8: // RESOURCE_URIS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list420 = iprot.readListBegin();
-                struct.resourceUris = new ArrayList<ResourceUri>(_list420.size);
-                ResourceUri _elem421;
-                for (int _i422 = 0; _i422 < _list420.size; ++_i422)
+                org.apache.thrift.protocol.TList _list436 = iprot.readListBegin();
+                struct.resourceUris = new ArrayList<ResourceUri>(_list436.size);
+                ResourceUri _elem437;
+                for (int _i438 = 0; _i438 < _list436.size; ++_i438)
                 {
-                  _elem421 = new ResourceUri();
-                  _elem421.read(iprot);
-                  struct.resourceUris.add(_elem421);
+                  _elem437 = new ResourceUri();
+                  _elem437.read(iprot);
+                  struct.resourceUris.add(_elem437);
                 }
                 iprot.readListEnd();
               }
@@ -1063,9 +1063,9 @@ public class Function implements org.apache.thrift.TBase<Function, Function._Fie
         oprot.writeFieldBegin(RESOURCE_URIS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourceUris.size()));
-          for (ResourceUri _iter423 : struct.resourceUris)
+          for (ResourceUri _iter439 : struct.resourceUris)
           {
-            _iter423.write(oprot);
+            _iter439.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1138,9 +1138,9 @@ public class Function implements org.apache.thrift.TBase<Function, Function._Fie
       if (struct.isSetResourceUris()) {
         {
           oprot.writeI32(struct.resourceUris.size());
-          for (ResourceUri _iter424 : struct.resourceUris)
+          for (ResourceUri _iter440 : struct.resourceUris)
           {
-            _iter424.write(oprot);
+            _iter440.write(oprot);
           }
         }
       }
@@ -1180,14 +1180,14 @@ public class Function implements org.apache.thrift.TBase<Function, Function._Fie
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list425 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourceUris = new ArrayList<ResourceUri>(_list425.size);
-          ResourceUri _elem426;
-          for (int _i427 = 0; _i427 < _list425.size; ++_i427)
+          org.apache.thrift.protocol.TList _list441 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourceUris = new ArrayList<ResourceUri>(_list441.size);
+          ResourceUri _elem442;
+          for (int _i443 = 0; _i443 < _list441.size; ++_i443)
           {
-            _elem426 = new ResourceUri();
-            _elem426.read(iprot);
-            struct.resourceUris.add(_elem426);
+            _elem442 = new ResourceUri();
+            _elem442.read(iprot);
+            struct.resourceUris.add(_elem442);
           }
         }
         struct.setResourceUrisIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 f88e279..2c297ef 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
@@ -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 _list584 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list584.size);
-                Function _elem585;
-                for (int _i586 = 0; _i586 < _list584.size; ++_i586)
+                org.apache.thrift.protocol.TList _list600 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list600.size);
+                Function _elem601;
+                for (int _i602 = 0; _i602 < _list600.size; ++_i602)
                 {
-                  _elem585 = new Function();
-                  _elem585.read(iprot);
-                  struct.functions.add(_elem585);
+                  _elem601 = new Function();
+                  _elem601.read(iprot);
+                  struct.functions.add(_elem601);
                 }
                 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 _iter587 : struct.functions)
+            for (Function _iter603 : struct.functions)
             {
-              _iter587.write(oprot);
+              _iter603.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 _iter588 : struct.functions)
+          for (Function _iter604 : struct.functions)
           {
-            _iter588.write(oprot);
+            _iter604.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 _list589 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list589.size);
-          Function _elem590;
-          for (int _i591 = 0; _i591 < _list589.size; ++_i591)
+          org.apache.thrift.protocol.TList _list605 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list605.size);
+          Function _elem606;
+          for (int _i607 = 0; _i607 < _list605.size; ++_i607)
           {
-            _elem590 = new Function();
-            _elem590.read(iprot);
-            struct.functions.add(_elem590);
+            _elem606 = new Function();
+            _elem606.read(iprot);
+            struct.functions.add(_elem606);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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
index 0236b4a..af3e4e3 100644
--- 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
@@ -619,13 +619,13 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list534 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list534.size);
-                long _elem535;
-                for (int _i536 = 0; _i536 < _list534.size; ++_i536)
+                org.apache.thrift.protocol.TList _list550 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list550.size);
+                long _elem551;
+                for (int _i552 = 0; _i552 < _list550.size; ++_i552)
                 {
-                  _elem535 = iprot.readI64();
-                  struct.fileIds.add(_elem535);
+                  _elem551 = iprot.readI64();
+                  struct.fileIds.add(_elem551);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
         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 _iter537 : struct.fileIds)
+          for (long _iter553 : struct.fileIds)
           {
-            oprot.writeI64(_iter537);
+            oprot.writeI64(_iter553);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter538 : struct.fileIds)
+        for (long _iter554 : struct.fileIds)
         {
-          oprot.writeI64(_iter538);
+          oprot.writeI64(_iter554);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
     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 _list539 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list539.size);
-        long _elem540;
-        for (int _i541 = 0; _i541 < _list539.size; ++_i541)
+        org.apache.thrift.protocol.TList _list555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list555.size);
+        long _elem556;
+        for (int _i557 = 0; _i557 < _list555.size; ++_i557)
         {
-          _elem540 = iprot.readI64();
-          struct.fileIds.add(_elem540);
+          _elem556 = iprot.readI64();
+          struct.fileIds.add(_elem556);
         }
       }
       struct.setFileIdsIsSet(true);


[25/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 8a8f8b1..11d3322 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -40,6 +40,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_schema_with_environment_context(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) = 0;
   virtual void create_table(const Table& tbl) = 0;
   virtual void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) = 0;
+  virtual void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys) = 0;
   virtual void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) = 0;
   virtual void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) = 0;
   virtual void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern) = 0;
@@ -99,6 +100,8 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) = 0;
   virtual void get_indexes(std::vector<Index> & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) = 0;
   virtual void get_index_names(std::vector<std::string> & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) = 0;
+  virtual void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) = 0;
+  virtual void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) = 0;
   virtual bool update_table_column_statistics(const ColumnStatistics& stats_obj) = 0;
   virtual bool update_partition_column_statistics(const ColumnStatistics& stats_obj) = 0;
   virtual void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) = 0;
@@ -250,6 +253,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void create_table_with_environment_context(const Table& /* tbl */, const EnvironmentContext& /* environment_context */) {
     return;
   }
+  void create_table_with_constraints(const Table& /* tbl */, const std::vector<SQLPrimaryKey> & /* primaryKeys */, const std::vector<SQLForeignKey> & /* foreignKeys */) {
+    return;
+  }
   void drop_table(const std::string& /* dbname */, const std::string& /* name */, const bool /* deleteData */) {
     return;
   }
@@ -437,6 +443,12 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_index_names(std::vector<std::string> & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_indexes */) {
     return;
   }
+  void get_primary_keys(PrimaryKeysResponse& /* _return */, const PrimaryKeysRequest& /* request */) {
+    return;
+  }
+  void get_foreign_keys(ForeignKeysResponse& /* _return */, const ForeignKeysRequest& /* request */) {
+    return;
+  }
   bool update_table_column_statistics(const ColumnStatistics& /* stats_obj */) {
     bool _return = false;
     return _return;
@@ -2878,6 +2890,148 @@ class ThriftHiveMetastore_create_table_with_environment_context_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_create_table_with_constraints_args__isset {
+  _ThriftHiveMetastore_create_table_with_constraints_args__isset() : tbl(false), primaryKeys(false), foreignKeys(false) {}
+  bool tbl :1;
+  bool primaryKeys :1;
+  bool foreignKeys :1;
+} _ThriftHiveMetastore_create_table_with_constraints_args__isset;
+
+class ThriftHiveMetastore_create_table_with_constraints_args {
+ public:
+
+  ThriftHiveMetastore_create_table_with_constraints_args(const ThriftHiveMetastore_create_table_with_constraints_args&);
+  ThriftHiveMetastore_create_table_with_constraints_args& operator=(const ThriftHiveMetastore_create_table_with_constraints_args&);
+  ThriftHiveMetastore_create_table_with_constraints_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_create_table_with_constraints_args() throw();
+  Table tbl;
+  std::vector<SQLPrimaryKey>  primaryKeys;
+  std::vector<SQLForeignKey>  foreignKeys;
+
+  _ThriftHiveMetastore_create_table_with_constraints_args__isset __isset;
+
+  void __set_tbl(const Table& val);
+
+  void __set_primaryKeys(const std::vector<SQLPrimaryKey> & val);
+
+  void __set_foreignKeys(const std::vector<SQLForeignKey> & val);
+
+  bool operator == (const ThriftHiveMetastore_create_table_with_constraints_args & rhs) const
+  {
+    if (!(tbl == rhs.tbl))
+      return false;
+    if (!(primaryKeys == rhs.primaryKeys))
+      return false;
+    if (!(foreignKeys == rhs.foreignKeys))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_create_table_with_constraints_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_create_table_with_constraints_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_create_table_with_constraints_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_create_table_with_constraints_pargs() throw();
+  const Table* tbl;
+  const std::vector<SQLPrimaryKey> * primaryKeys;
+  const std::vector<SQLForeignKey> * foreignKeys;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_create_table_with_constraints_result__isset {
+  _ThriftHiveMetastore_create_table_with_constraints_result__isset() : o1(false), o2(false), o3(false), o4(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+  bool o4 :1;
+} _ThriftHiveMetastore_create_table_with_constraints_result__isset;
+
+class ThriftHiveMetastore_create_table_with_constraints_result {
+ public:
+
+  ThriftHiveMetastore_create_table_with_constraints_result(const ThriftHiveMetastore_create_table_with_constraints_result&);
+  ThriftHiveMetastore_create_table_with_constraints_result& operator=(const ThriftHiveMetastore_create_table_with_constraints_result&);
+  ThriftHiveMetastore_create_table_with_constraints_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_create_table_with_constraints_result() throw();
+  AlreadyExistsException o1;
+  InvalidObjectException o2;
+  MetaException o3;
+  NoSuchObjectException o4;
+
+  _ThriftHiveMetastore_create_table_with_constraints_result__isset __isset;
+
+  void __set_o1(const AlreadyExistsException& val);
+
+  void __set_o2(const InvalidObjectException& val);
+
+  void __set_o3(const MetaException& val);
+
+  void __set_o4(const NoSuchObjectException& val);
+
+  bool operator == (const ThriftHiveMetastore_create_table_with_constraints_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    if (!(o3 == rhs.o3))
+      return false;
+    if (!(o4 == rhs.o4))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_create_table_with_constraints_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_create_table_with_constraints_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_create_table_with_constraints_presult__isset {
+  _ThriftHiveMetastore_create_table_with_constraints_presult__isset() : o1(false), o2(false), o3(false), o4(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+  bool o4 :1;
+} _ThriftHiveMetastore_create_table_with_constraints_presult__isset;
+
+class ThriftHiveMetastore_create_table_with_constraints_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_create_table_with_constraints_presult() throw();
+  AlreadyExistsException o1;
+  InvalidObjectException o2;
+  MetaException o3;
+  NoSuchObjectException o4;
+
+  _ThriftHiveMetastore_create_table_with_constraints_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_drop_table_args__isset {
   _ThriftHiveMetastore_drop_table_args__isset() : dbname(false), name(false), deleteData(false) {}
   bool dbname :1;
@@ -10864,6 +11018,246 @@ class ThriftHiveMetastore_get_index_names_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_get_primary_keys_args__isset {
+  _ThriftHiveMetastore_get_primary_keys_args__isset() : request(false) {}
+  bool request :1;
+} _ThriftHiveMetastore_get_primary_keys_args__isset;
+
+class ThriftHiveMetastore_get_primary_keys_args {
+ public:
+
+  ThriftHiveMetastore_get_primary_keys_args(const ThriftHiveMetastore_get_primary_keys_args&);
+  ThriftHiveMetastore_get_primary_keys_args& operator=(const ThriftHiveMetastore_get_primary_keys_args&);
+  ThriftHiveMetastore_get_primary_keys_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_primary_keys_args() throw();
+  PrimaryKeysRequest request;
+
+  _ThriftHiveMetastore_get_primary_keys_args__isset __isset;
+
+  void __set_request(const PrimaryKeysRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_primary_keys_args & rhs) const
+  {
+    if (!(request == rhs.request))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_primary_keys_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_primary_keys_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_primary_keys_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_primary_keys_pargs() throw();
+  const PrimaryKeysRequest* request;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_primary_keys_result__isset {
+  _ThriftHiveMetastore_get_primary_keys_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_primary_keys_result__isset;
+
+class ThriftHiveMetastore_get_primary_keys_result {
+ public:
+
+  ThriftHiveMetastore_get_primary_keys_result(const ThriftHiveMetastore_get_primary_keys_result&);
+  ThriftHiveMetastore_get_primary_keys_result& operator=(const ThriftHiveMetastore_get_primary_keys_result&);
+  ThriftHiveMetastore_get_primary_keys_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_primary_keys_result() throw();
+  PrimaryKeysResponse success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_primary_keys_result__isset __isset;
+
+  void __set_success(const PrimaryKeysResponse& val);
+
+  void __set_o1(const MetaException& val);
+
+  void __set_o2(const NoSuchObjectException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_primary_keys_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_primary_keys_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_primary_keys_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_primary_keys_presult__isset {
+  _ThriftHiveMetastore_get_primary_keys_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_primary_keys_presult__isset;
+
+class ThriftHiveMetastore_get_primary_keys_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_primary_keys_presult() throw();
+  PrimaryKeysResponse* success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_primary_keys_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_foreign_keys_args__isset {
+  _ThriftHiveMetastore_get_foreign_keys_args__isset() : request(false) {}
+  bool request :1;
+} _ThriftHiveMetastore_get_foreign_keys_args__isset;
+
+class ThriftHiveMetastore_get_foreign_keys_args {
+ public:
+
+  ThriftHiveMetastore_get_foreign_keys_args(const ThriftHiveMetastore_get_foreign_keys_args&);
+  ThriftHiveMetastore_get_foreign_keys_args& operator=(const ThriftHiveMetastore_get_foreign_keys_args&);
+  ThriftHiveMetastore_get_foreign_keys_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_foreign_keys_args() throw();
+  ForeignKeysRequest request;
+
+  _ThriftHiveMetastore_get_foreign_keys_args__isset __isset;
+
+  void __set_request(const ForeignKeysRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_foreign_keys_args & rhs) const
+  {
+    if (!(request == rhs.request))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_foreign_keys_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_foreign_keys_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_foreign_keys_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_foreign_keys_pargs() throw();
+  const ForeignKeysRequest* request;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_foreign_keys_result__isset {
+  _ThriftHiveMetastore_get_foreign_keys_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_foreign_keys_result__isset;
+
+class ThriftHiveMetastore_get_foreign_keys_result {
+ public:
+
+  ThriftHiveMetastore_get_foreign_keys_result(const ThriftHiveMetastore_get_foreign_keys_result&);
+  ThriftHiveMetastore_get_foreign_keys_result& operator=(const ThriftHiveMetastore_get_foreign_keys_result&);
+  ThriftHiveMetastore_get_foreign_keys_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_foreign_keys_result() throw();
+  ForeignKeysResponse success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_foreign_keys_result__isset __isset;
+
+  void __set_success(const ForeignKeysResponse& val);
+
+  void __set_o1(const MetaException& val);
+
+  void __set_o2(const NoSuchObjectException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_foreign_keys_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_foreign_keys_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_foreign_keys_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_foreign_keys_presult__isset {
+  _ThriftHiveMetastore_get_foreign_keys_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_foreign_keys_presult__isset;
+
+class ThriftHiveMetastore_get_foreign_keys_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_foreign_keys_presult() throw();
+  ForeignKeysResponse* success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_foreign_keys_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_update_table_column_statistics_args__isset {
   _ThriftHiveMetastore_update_table_column_statistics_args__isset() : stats_obj(false) {}
   bool stats_obj :1;
@@ -18454,6 +18848,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
   void send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
   void recv_create_table_with_environment_context();
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
+  void send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
+  void recv_create_table_with_constraints();
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void recv_drop_table();
@@ -18631,6 +19028,12 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_index_names(std::vector<std::string> & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes);
   void send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes);
   void recv_get_index_names(std::vector<std::string> & _return);
+  void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request);
+  void send_get_primary_keys(const PrimaryKeysRequest& request);
+  void recv_get_primary_keys(PrimaryKeysResponse& _return);
+  void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request);
+  void send_get_foreign_keys(const ForeignKeysRequest& request);
+  void recv_get_foreign_keys(ForeignKeysResponse& _return);
   bool update_table_column_statistics(const ColumnStatistics& stats_obj);
   void send_update_table_column_statistics(const ColumnStatistics& stats_obj);
   bool recv_update_table_column_statistics();
@@ -18857,6 +19260,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_schema_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_create_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_create_table_with_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -18916,6 +19320,8 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_indexes(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_primary_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_foreign_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -19004,6 +19410,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_schema_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_get_schema_with_environment_context;
     processMap_["create_table"] = &ThriftHiveMetastoreProcessor::process_create_table;
     processMap_["create_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_create_table_with_environment_context;
+    processMap_["create_table_with_constraints"] = &ThriftHiveMetastoreProcessor::process_create_table_with_constraints;
     processMap_["drop_table"] = &ThriftHiveMetastoreProcessor::process_drop_table;
     processMap_["drop_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context;
     processMap_["get_tables"] = &ThriftHiveMetastoreProcessor::process_get_tables;
@@ -19063,6 +19470,8 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_index_by_name"] = &ThriftHiveMetastoreProcessor::process_get_index_by_name;
     processMap_["get_indexes"] = &ThriftHiveMetastoreProcessor::process_get_indexes;
     processMap_["get_index_names"] = &ThriftHiveMetastoreProcessor::process_get_index_names;
+    processMap_["get_primary_keys"] = &ThriftHiveMetastoreProcessor::process_get_primary_keys;
+    processMap_["get_foreign_keys"] = &ThriftHiveMetastoreProcessor::process_get_foreign_keys;
     processMap_["update_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_update_table_column_statistics;
     processMap_["update_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_update_partition_column_statistics;
     processMap_["get_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_get_table_column_statistics;
@@ -19334,6 +19743,15 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->create_table_with_environment_context(tbl, environment_context);
   }
 
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys);
+    }
+    ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys);
+  }
+
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -19902,6 +20320,26 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_primary_keys(_return, request);
+    }
+    ifaces_[i]->get_primary_keys(_return, request);
+    return;
+  }
+
+  void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_foreign_keys(_return, request);
+    }
+    ifaces_[i]->get_foreign_keys(_return, request);
+    return;
+  }
+
   bool update_table_column_statistics(const ColumnStatistics& stats_obj) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -20604,6 +21042,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
   int32_t send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
   void recv_create_table_with_environment_context(const int32_t seqid);
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
+  int32_t send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
+  void recv_create_table_with_constraints(const int32_t seqid);
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   int32_t send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void recv_drop_table(const int32_t seqid);
@@ -20781,6 +21222,12 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_index_names(std::vector<std::string> & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes);
   int32_t send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes);
   void recv_get_index_names(std::vector<std::string> & _return, const int32_t seqid);
+  void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request);
+  int32_t send_get_primary_keys(const PrimaryKeysRequest& request);
+  void recv_get_primary_keys(PrimaryKeysResponse& _return, const int32_t seqid);
+  void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request);
+  int32_t send_get_foreign_keys(const ForeignKeysRequest& request);
+  void recv_get_foreign_keys(ForeignKeysResponse& _return, const int32_t seqid);
   bool update_table_column_statistics(const ColumnStatistics& stats_obj);
   int32_t send_update_table_column_statistics(const ColumnStatistics& stats_obj);
   bool recv_update_table_column_statistics(const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 3e7c6e7..fa87e34 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -112,6 +112,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("create_table_with_environment_context\n");
   }
 
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys) {
+    // Your implementation goes here
+    printf("create_table_with_constraints\n");
+  }
+
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) {
     // Your implementation goes here
     printf("drop_table\n");
@@ -407,6 +412,16 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_index_names\n");
   }
 
+  void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) {
+    // Your implementation goes here
+    printf("get_primary_keys\n");
+  }
+
+  void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) {
+    // Your implementation goes here
+    printf("get_foreign_keys\n");
+  }
+
   bool update_table_column_statistics(const ColumnStatistics& stats_obj) {
     // Your implementation goes here
     printf("update_table_column_statistics\n");


[24/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 2695ffa..8da883d 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -392,30 +392,43 @@ void FieldSchema::printTo(std::ostream& out) const {
 }
 
 
-Type::~Type() throw() {
+SQLPrimaryKey::~SQLPrimaryKey() throw() {
 }
 
 
-void Type::__set_name(const std::string& val) {
-  this->name = val;
+void SQLPrimaryKey::__set_table_db(const std::string& val) {
+  this->table_db = val;
 }
 
-void Type::__set_type1(const std::string& val) {
-  this->type1 = val;
-__isset.type1 = true;
+void SQLPrimaryKey::__set_table_name(const std::string& val) {
+  this->table_name = val;
 }
 
-void Type::__set_type2(const std::string& val) {
-  this->type2 = val;
-__isset.type2 = true;
+void SQLPrimaryKey::__set_column_name(const std::string& val) {
+  this->column_name = val;
 }
 
-void Type::__set_fields(const std::vector<FieldSchema> & val) {
-  this->fields = val;
-__isset.fields = true;
+void SQLPrimaryKey::__set_key_seq(const int32_t val) {
+  this->key_seq = val;
 }
 
-uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
+void SQLPrimaryKey::__set_pk_name(const std::string& val) {
+  this->pk_name = val;
+}
+
+void SQLPrimaryKey::__set_enable_cstr(const bool val) {
+  this->enable_cstr = val;
+}
+
+void SQLPrimaryKey::__set_validate_cstr(const bool val) {
+  this->validate_cstr = val;
+}
+
+void SQLPrimaryKey::__set_rely_cstr(const bool val) {
+  this->rely_cstr = val;
+}
+
+uint32_t SQLPrimaryKey::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -438,44 +451,64 @@ uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->name);
-          this->__isset.name = true;
+          xfer += iprot->readString(this->table_db);
+          this->__isset.table_db = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->type1);
-          this->__isset.type1 = true;
+          xfer += iprot->readString(this->table_name);
+          this->__isset.table_name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->type2);
-          this->__isset.type2 = true;
+          xfer += iprot->readString(this->column_name);
+          this->__isset.column_name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 4:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->fields.clear();
-            uint32_t _size4;
-            ::apache::thrift::protocol::TType _etype7;
-            xfer += iprot->readListBegin(_etype7, _size4);
-            this->fields.resize(_size4);
-            uint32_t _i8;
-            for (_i8 = 0; _i8 < _size4; ++_i8)
-            {
-              xfer += this->fields[_i8].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.fields = true;
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->key_seq);
+          this->__isset.key_seq = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->pk_name);
+          this->__isset.pk_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->enable_cstr);
+          this->__isset.enable_cstr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->validate_cstr);
+          this->__isset.validate_cstr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 8:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->rely_cstr);
+          this->__isset.rely_cstr = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -492,103 +525,160 @@ uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
   return xfer;
 }
 
-uint32_t Type::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t SQLPrimaryKey::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("Type");
+  xfer += oprot->writeStructBegin("SQLPrimaryKey");
 
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->table_db);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->table_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->column_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 4);
+  xfer += oprot->writeI32(this->key_seq);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("pk_name", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeString(this->pk_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6);
+  xfer += oprot->writeBool(this->enable_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7);
+  xfer += oprot->writeBool(this->validate_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8);
+  xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
-  if (this->__isset.type1) {
-    xfer += oprot->writeFieldBegin("type1", ::apache::thrift::protocol::T_STRING, 2);
-    xfer += oprot->writeString(this->type1);
-    xfer += oprot->writeFieldEnd();
-  }
-  if (this->__isset.type2) {
-    xfer += oprot->writeFieldBegin("type2", ::apache::thrift::protocol::T_STRING, 3);
-    xfer += oprot->writeString(this->type2);
-    xfer += oprot->writeFieldEnd();
-  }
-  if (this->__isset.fields) {
-    xfer += oprot->writeFieldBegin("fields", ::apache::thrift::protocol::T_LIST, 4);
-    {
-      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->fields.size()));
-      std::vector<FieldSchema> ::const_iterator _iter9;
-      for (_iter9 = this->fields.begin(); _iter9 != this->fields.end(); ++_iter9)
-      {
-        xfer += (*_iter9).write(oprot);
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(Type &a, Type &b) {
+void swap(SQLPrimaryKey &a, SQLPrimaryKey &b) {
   using ::std::swap;
-  swap(a.name, b.name);
-  swap(a.type1, b.type1);
-  swap(a.type2, b.type2);
-  swap(a.fields, b.fields);
+  swap(a.table_db, b.table_db);
+  swap(a.table_name, b.table_name);
+  swap(a.column_name, b.column_name);
+  swap(a.key_seq, b.key_seq);
+  swap(a.pk_name, b.pk_name);
+  swap(a.enable_cstr, b.enable_cstr);
+  swap(a.validate_cstr, b.validate_cstr);
+  swap(a.rely_cstr, b.rely_cstr);
   swap(a.__isset, b.__isset);
 }
 
-Type::Type(const Type& other10) {
-  name = other10.name;
-  type1 = other10.type1;
-  type2 = other10.type2;
-  fields = other10.fields;
-  __isset = other10.__isset;
-}
-Type& Type::operator=(const Type& other11) {
-  name = other11.name;
-  type1 = other11.type1;
-  type2 = other11.type2;
-  fields = other11.fields;
-  __isset = other11.__isset;
+SQLPrimaryKey::SQLPrimaryKey(const SQLPrimaryKey& other4) {
+  table_db = other4.table_db;
+  table_name = other4.table_name;
+  column_name = other4.column_name;
+  key_seq = other4.key_seq;
+  pk_name = other4.pk_name;
+  enable_cstr = other4.enable_cstr;
+  validate_cstr = other4.validate_cstr;
+  rely_cstr = other4.rely_cstr;
+  __isset = other4.__isset;
+}
+SQLPrimaryKey& SQLPrimaryKey::operator=(const SQLPrimaryKey& other5) {
+  table_db = other5.table_db;
+  table_name = other5.table_name;
+  column_name = other5.column_name;
+  key_seq = other5.key_seq;
+  pk_name = other5.pk_name;
+  enable_cstr = other5.enable_cstr;
+  validate_cstr = other5.validate_cstr;
+  rely_cstr = other5.rely_cstr;
+  __isset = other5.__isset;
   return *this;
 }
-void Type::printTo(std::ostream& out) const {
+void SQLPrimaryKey::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "Type(";
-  out << "name=" << to_string(name);
-  out << ", " << "type1="; (__isset.type1 ? (out << to_string(type1)) : (out << "<null>"));
-  out << ", " << "type2="; (__isset.type2 ? (out << to_string(type2)) : (out << "<null>"));
-  out << ", " << "fields="; (__isset.fields ? (out << to_string(fields)) : (out << "<null>"));
+  out << "SQLPrimaryKey(";
+  out << "table_db=" << to_string(table_db);
+  out << ", " << "table_name=" << to_string(table_name);
+  out << ", " << "column_name=" << to_string(column_name);
+  out << ", " << "key_seq=" << to_string(key_seq);
+  out << ", " << "pk_name=" << to_string(pk_name);
+  out << ", " << "enable_cstr=" << to_string(enable_cstr);
+  out << ", " << "validate_cstr=" << to_string(validate_cstr);
+  out << ", " << "rely_cstr=" << to_string(rely_cstr);
   out << ")";
 }
 
 
-HiveObjectRef::~HiveObjectRef() throw() {
+SQLForeignKey::~SQLForeignKey() throw() {
 }
 
 
-void HiveObjectRef::__set_objectType(const HiveObjectType::type val) {
-  this->objectType = val;
+void SQLForeignKey::__set_pktable_db(const std::string& val) {
+  this->pktable_db = val;
 }
 
-void HiveObjectRef::__set_dbName(const std::string& val) {
-  this->dbName = val;
+void SQLForeignKey::__set_pktable_name(const std::string& val) {
+  this->pktable_name = val;
 }
 
-void HiveObjectRef::__set_objectName(const std::string& val) {
-  this->objectName = val;
+void SQLForeignKey::__set_pkcolumn_name(const std::string& val) {
+  this->pkcolumn_name = val;
 }
 
-void HiveObjectRef::__set_partValues(const std::vector<std::string> & val) {
-  this->partValues = val;
+void SQLForeignKey::__set_fktable_db(const std::string& val) {
+  this->fktable_db = val;
 }
 
-void HiveObjectRef::__set_columnName(const std::string& val) {
-  this->columnName = val;
+void SQLForeignKey::__set_fktable_name(const std::string& val) {
+  this->fktable_name = val;
 }
 
-uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
+void SQLForeignKey::__set_fkcolumn_name(const std::string& val) {
+  this->fkcolumn_name = val;
+}
+
+void SQLForeignKey::__set_key_seq(const int32_t val) {
+  this->key_seq = val;
+}
+
+void SQLForeignKey::__set_update_rule(const int32_t val) {
+  this->update_rule = val;
+}
+
+void SQLForeignKey::__set_delete_rule(const int32_t val) {
+  this->delete_rule = val;
+}
+
+void SQLForeignKey::__set_fk_name(const std::string& val) {
+  this->fk_name = val;
+}
+
+void SQLForeignKey::__set_pk_name(const std::string& val) {
+  this->pk_name = val;
+}
+
+void SQLForeignKey::__set_enable_cstr(const bool val) {
+  this->enable_cstr = val;
+}
+
+void SQLForeignKey::__set_validate_cstr(const bool val) {
+  this->validate_cstr = val;
+}
+
+void SQLForeignKey::__set_rely_cstr(const bool val) {
+  this->rely_cstr = val;
+}
+
+uint32_t SQLForeignKey::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -610,55 +700,113 @@ uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast12;
-          xfer += iprot->readI32(ecast12);
-          this->objectType = (HiveObjectType::type)ecast12;
-          this->__isset.objectType = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->pktable_db);
+          this->__isset.pktable_db = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->dbName);
-          this->__isset.dbName = true;
+          xfer += iprot->readString(this->pktable_name);
+          this->__isset.pktable_name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->objectName);
-          this->__isset.objectName = true;
+          xfer += iprot->readString(this->pkcolumn_name);
+          this->__isset.pkcolumn_name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 4:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->partValues.clear();
-            uint32_t _size13;
-            ::apache::thrift::protocol::TType _etype16;
-            xfer += iprot->readListBegin(_etype16, _size13);
-            this->partValues.resize(_size13);
-            uint32_t _i17;
-            for (_i17 = 0; _i17 < _size13; ++_i17)
-            {
-              xfer += iprot->readString(this->partValues[_i17]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.partValues = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->fktable_db);
+          this->__isset.fktable_db = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->columnName);
-          this->__isset.columnName = true;
+          xfer += iprot->readString(this->fktable_name);
+          this->__isset.fktable_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->fkcolumn_name);
+          this->__isset.fkcolumn_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->key_seq);
+          this->__isset.key_seq = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 8:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->update_rule);
+          this->__isset.update_rule = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 9:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->delete_rule);
+          this->__isset.delete_rule = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 10:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->fk_name);
+          this->__isset.fk_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 11:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->pk_name);
+          this->__isset.pk_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 12:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->enable_cstr);
+          this->__isset.enable_cstr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 13:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->validate_cstr);
+          this->__isset.validate_cstr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 14:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->rely_cstr);
+          this->__isset.rely_cstr = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -675,37 +823,65 @@ uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
   return xfer;
 }
 
-uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t SQLForeignKey::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("HiveObjectRef");
+  xfer += oprot->writeStructBegin("SQLForeignKey");
 
-  xfer += oprot->writeFieldBegin("objectType", ::apache::thrift::protocol::T_I32, 1);
-  xfer += oprot->writeI32((int32_t)this->objectType);
+  xfer += oprot->writeFieldBegin("pktable_db", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->pktable_db);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->dbName);
+  xfer += oprot->writeFieldBegin("pktable_name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->pktable_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("objectName", ::apache::thrift::protocol::T_STRING, 3);
-  xfer += oprot->writeString(this->objectName);
+  xfer += oprot->writeFieldBegin("pkcolumn_name", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->pkcolumn_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("partValues", ::apache::thrift::protocol::T_LIST, 4);
-  {
-    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partValues.size()));
-    std::vector<std::string> ::const_iterator _iter18;
-    for (_iter18 = this->partValues.begin(); _iter18 != this->partValues.end(); ++_iter18)
-    {
-      xfer += oprot->writeString((*_iter18));
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("fktable_db", ::apache::thrift::protocol::T_STRING, 4);
+  xfer += oprot->writeString(this->fktable_db);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("columnName", ::apache::thrift::protocol::T_STRING, 5);
-  xfer += oprot->writeString(this->columnName);
+  xfer += oprot->writeFieldBegin("fktable_name", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeString(this->fktable_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("fkcolumn_name", ::apache::thrift::protocol::T_STRING, 6);
+  xfer += oprot->writeString(this->fkcolumn_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 7);
+  xfer += oprot->writeI32(this->key_seq);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("update_rule", ::apache::thrift::protocol::T_I32, 8);
+  xfer += oprot->writeI32(this->update_rule);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("delete_rule", ::apache::thrift::protocol::T_I32, 9);
+  xfer += oprot->writeI32(this->delete_rule);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("fk_name", ::apache::thrift::protocol::T_STRING, 10);
+  xfer += oprot->writeString(this->fk_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("pk_name", ::apache::thrift::protocol::T_STRING, 11);
+  xfer += oprot->writeString(this->pk_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 12);
+  xfer += oprot->writeBool(this->enable_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 13);
+  xfer += oprot->writeBool(this->validate_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 14);
+  xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -713,70 +889,105 @@ uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) cons
   return xfer;
 }
 
-void swap(HiveObjectRef &a, HiveObjectRef &b) {
+void swap(SQLForeignKey &a, SQLForeignKey &b) {
   using ::std::swap;
-  swap(a.objectType, b.objectType);
-  swap(a.dbName, b.dbName);
-  swap(a.objectName, b.objectName);
-  swap(a.partValues, b.partValues);
-  swap(a.columnName, b.columnName);
+  swap(a.pktable_db, b.pktable_db);
+  swap(a.pktable_name, b.pktable_name);
+  swap(a.pkcolumn_name, b.pkcolumn_name);
+  swap(a.fktable_db, b.fktable_db);
+  swap(a.fktable_name, b.fktable_name);
+  swap(a.fkcolumn_name, b.fkcolumn_name);
+  swap(a.key_seq, b.key_seq);
+  swap(a.update_rule, b.update_rule);
+  swap(a.delete_rule, b.delete_rule);
+  swap(a.fk_name, b.fk_name);
+  swap(a.pk_name, b.pk_name);
+  swap(a.enable_cstr, b.enable_cstr);
+  swap(a.validate_cstr, b.validate_cstr);
+  swap(a.rely_cstr, b.rely_cstr);
   swap(a.__isset, b.__isset);
 }
 
-HiveObjectRef::HiveObjectRef(const HiveObjectRef& other19) {
-  objectType = other19.objectType;
-  dbName = other19.dbName;
-  objectName = other19.objectName;
-  partValues = other19.partValues;
-  columnName = other19.columnName;
-  __isset = other19.__isset;
-}
-HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other20) {
-  objectType = other20.objectType;
-  dbName = other20.dbName;
-  objectName = other20.objectName;
-  partValues = other20.partValues;
-  columnName = other20.columnName;
-  __isset = other20.__isset;
+SQLForeignKey::SQLForeignKey(const SQLForeignKey& other6) {
+  pktable_db = other6.pktable_db;
+  pktable_name = other6.pktable_name;
+  pkcolumn_name = other6.pkcolumn_name;
+  fktable_db = other6.fktable_db;
+  fktable_name = other6.fktable_name;
+  fkcolumn_name = other6.fkcolumn_name;
+  key_seq = other6.key_seq;
+  update_rule = other6.update_rule;
+  delete_rule = other6.delete_rule;
+  fk_name = other6.fk_name;
+  pk_name = other6.pk_name;
+  enable_cstr = other6.enable_cstr;
+  validate_cstr = other6.validate_cstr;
+  rely_cstr = other6.rely_cstr;
+  __isset = other6.__isset;
+}
+SQLForeignKey& SQLForeignKey::operator=(const SQLForeignKey& other7) {
+  pktable_db = other7.pktable_db;
+  pktable_name = other7.pktable_name;
+  pkcolumn_name = other7.pkcolumn_name;
+  fktable_db = other7.fktable_db;
+  fktable_name = other7.fktable_name;
+  fkcolumn_name = other7.fkcolumn_name;
+  key_seq = other7.key_seq;
+  update_rule = other7.update_rule;
+  delete_rule = other7.delete_rule;
+  fk_name = other7.fk_name;
+  pk_name = other7.pk_name;
+  enable_cstr = other7.enable_cstr;
+  validate_cstr = other7.validate_cstr;
+  rely_cstr = other7.rely_cstr;
+  __isset = other7.__isset;
   return *this;
 }
-void HiveObjectRef::printTo(std::ostream& out) const {
+void SQLForeignKey::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "HiveObjectRef(";
-  out << "objectType=" << to_string(objectType);
-  out << ", " << "dbName=" << to_string(dbName);
-  out << ", " << "objectName=" << to_string(objectName);
-  out << ", " << "partValues=" << to_string(partValues);
-  out << ", " << "columnName=" << to_string(columnName);
+  out << "SQLForeignKey(";
+  out << "pktable_db=" << to_string(pktable_db);
+  out << ", " << "pktable_name=" << to_string(pktable_name);
+  out << ", " << "pkcolumn_name=" << to_string(pkcolumn_name);
+  out << ", " << "fktable_db=" << to_string(fktable_db);
+  out << ", " << "fktable_name=" << to_string(fktable_name);
+  out << ", " << "fkcolumn_name=" << to_string(fkcolumn_name);
+  out << ", " << "key_seq=" << to_string(key_seq);
+  out << ", " << "update_rule=" << to_string(update_rule);
+  out << ", " << "delete_rule=" << to_string(delete_rule);
+  out << ", " << "fk_name=" << to_string(fk_name);
+  out << ", " << "pk_name=" << to_string(pk_name);
+  out << ", " << "enable_cstr=" << to_string(enable_cstr);
+  out << ", " << "validate_cstr=" << to_string(validate_cstr);
+  out << ", " << "rely_cstr=" << to_string(rely_cstr);
   out << ")";
 }
 
 
-PrivilegeGrantInfo::~PrivilegeGrantInfo() throw() {
+Type::~Type() throw() {
 }
 
 
-void PrivilegeGrantInfo::__set_privilege(const std::string& val) {
-  this->privilege = val;
-}
-
-void PrivilegeGrantInfo::__set_createTime(const int32_t val) {
-  this->createTime = val;
+void Type::__set_name(const std::string& val) {
+  this->name = val;
 }
 
-void PrivilegeGrantInfo::__set_grantor(const std::string& val) {
-  this->grantor = val;
+void Type::__set_type1(const std::string& val) {
+  this->type1 = val;
+__isset.type1 = true;
 }
 
-void PrivilegeGrantInfo::__set_grantorType(const PrincipalType::type val) {
-  this->grantorType = val;
+void Type::__set_type2(const std::string& val) {
+  this->type2 = val;
+__isset.type2 = true;
 }
 
-void PrivilegeGrantInfo::__set_grantOption(const bool val) {
-  this->grantOption = val;
+void Type::__set_fields(const std::vector<FieldSchema> & val) {
+  this->fields = val;
+__isset.fields = true;
 }
 
-uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -799,42 +1010,44 @@ uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot)
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->privilege);
-          this->__isset.privilege = true;
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->createTime);
-          this->__isset.createTime = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->type1);
+          this->__isset.type1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->grantor);
-          this->__isset.grantor = true;
+          xfer += iprot->readString(this->type2);
+          this->__isset.type2 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 4:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast21;
-          xfer += iprot->readI32(ecast21);
-          this->grantorType = (PrincipalType::type)ecast21;
-          this->__isset.grantorType = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 5:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->grantOption);
-          this->__isset.grantOption = true;
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->fields.clear();
+            uint32_t _size8;
+            ::apache::thrift::protocol::TType _etype11;
+            xfer += iprot->readListBegin(_etype11, _size8);
+            this->fields.resize(_size8);
+            uint32_t _i12;
+            for (_i12 = 0; _i12 < _size8; ++_i12)
+            {
+              xfer += this->fields[_i12].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.fields = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -851,96 +1064,103 @@ uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot)
   return xfer;
 }
 
-uint32_t PrivilegeGrantInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t Type::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("PrivilegeGrantInfo");
-
-  xfer += oprot->writeFieldBegin("privilege", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->privilege);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2);
-  xfer += oprot->writeI32(this->createTime);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 3);
-  xfer += oprot->writeString(this->grantor);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 4);
-  xfer += oprot->writeI32((int32_t)this->grantorType);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("Type");
 
-  xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 5);
-  xfer += oprot->writeBool(this->grantOption);
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->name);
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.type1) {
+    xfer += oprot->writeFieldBegin("type1", ::apache::thrift::protocol::T_STRING, 2);
+    xfer += oprot->writeString(this->type1);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.type2) {
+    xfer += oprot->writeFieldBegin("type2", ::apache::thrift::protocol::T_STRING, 3);
+    xfer += oprot->writeString(this->type2);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.fields) {
+    xfer += oprot->writeFieldBegin("fields", ::apache::thrift::protocol::T_LIST, 4);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->fields.size()));
+      std::vector<FieldSchema> ::const_iterator _iter13;
+      for (_iter13 = this->fields.begin(); _iter13 != this->fields.end(); ++_iter13)
+      {
+        xfer += (*_iter13).write(oprot);
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b) {
+void swap(Type &a, Type &b) {
   using ::std::swap;
-  swap(a.privilege, b.privilege);
-  swap(a.createTime, b.createTime);
-  swap(a.grantor, b.grantor);
-  swap(a.grantorType, b.grantorType);
-  swap(a.grantOption, b.grantOption);
+  swap(a.name, b.name);
+  swap(a.type1, b.type1);
+  swap(a.type2, b.type2);
+  swap(a.fields, b.fields);
   swap(a.__isset, b.__isset);
 }
 
-PrivilegeGrantInfo::PrivilegeGrantInfo(const PrivilegeGrantInfo& other22) {
-  privilege = other22.privilege;
-  createTime = other22.createTime;
-  grantor = other22.grantor;
-  grantorType = other22.grantorType;
-  grantOption = other22.grantOption;
-  __isset = other22.__isset;
-}
-PrivilegeGrantInfo& PrivilegeGrantInfo::operator=(const PrivilegeGrantInfo& other23) {
-  privilege = other23.privilege;
-  createTime = other23.createTime;
-  grantor = other23.grantor;
-  grantorType = other23.grantorType;
-  grantOption = other23.grantOption;
-  __isset = other23.__isset;
+Type::Type(const Type& other14) {
+  name = other14.name;
+  type1 = other14.type1;
+  type2 = other14.type2;
+  fields = other14.fields;
+  __isset = other14.__isset;
+}
+Type& Type::operator=(const Type& other15) {
+  name = other15.name;
+  type1 = other15.type1;
+  type2 = other15.type2;
+  fields = other15.fields;
+  __isset = other15.__isset;
   return *this;
 }
-void PrivilegeGrantInfo::printTo(std::ostream& out) const {
+void Type::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "PrivilegeGrantInfo(";
-  out << "privilege=" << to_string(privilege);
-  out << ", " << "createTime=" << to_string(createTime);
-  out << ", " << "grantor=" << to_string(grantor);
-  out << ", " << "grantorType=" << to_string(grantorType);
-  out << ", " << "grantOption=" << to_string(grantOption);
+  out << "Type(";
+  out << "name=" << to_string(name);
+  out << ", " << "type1="; (__isset.type1 ? (out << to_string(type1)) : (out << "<null>"));
+  out << ", " << "type2="; (__isset.type2 ? (out << to_string(type2)) : (out << "<null>"));
+  out << ", " << "fields="; (__isset.fields ? (out << to_string(fields)) : (out << "<null>"));
   out << ")";
 }
 
 
-HiveObjectPrivilege::~HiveObjectPrivilege() throw() {
+HiveObjectRef::~HiveObjectRef() throw() {
 }
 
 
-void HiveObjectPrivilege::__set_hiveObject(const HiveObjectRef& val) {
-  this->hiveObject = val;
+void HiveObjectRef::__set_objectType(const HiveObjectType::type val) {
+  this->objectType = val;
 }
 
-void HiveObjectPrivilege::__set_principalName(const std::string& val) {
-  this->principalName = val;
+void HiveObjectRef::__set_dbName(const std::string& val) {
+  this->dbName = val;
 }
 
-void HiveObjectPrivilege::__set_principalType(const PrincipalType::type val) {
-  this->principalType = val;
+void HiveObjectRef::__set_objectName(const std::string& val) {
+  this->objectName = val;
 }
 
-void HiveObjectPrivilege::__set_grantInfo(const PrivilegeGrantInfo& val) {
-  this->grantInfo = val;
+void HiveObjectRef::__set_partValues(const std::vector<std::string> & val) {
+  this->partValues = val;
 }
 
-uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot) {
+void HiveObjectRef::__set_columnName(const std::string& val) {
+  this->columnName = val;
+}
+
+uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -962,35 +1182,55 @@ uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot)
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->hiveObject.read(iprot);
-          this->__isset.hiveObject = true;
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast16;
+          xfer += iprot->readI32(ecast16);
+          this->objectType = (HiveObjectType::type)ecast16;
+          this->__isset.objectType = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->principalName);
-          this->__isset.principalName = true;
+          xfer += iprot->readString(this->dbName);
+          this->__isset.dbName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast24;
-          xfer += iprot->readI32(ecast24);
-          this->principalType = (PrincipalType::type)ecast24;
-          this->__isset.principalType = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->objectName);
+          this->__isset.objectName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 4:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->grantInfo.read(iprot);
-          this->__isset.grantInfo = true;
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->partValues.clear();
+            uint32_t _size17;
+            ::apache::thrift::protocol::TType _etype20;
+            xfer += iprot->readListBegin(_etype20, _size17);
+            this->partValues.resize(_size17);
+            uint32_t _i21;
+            for (_i21 = 0; _i21 < _size17; ++_i21)
+            {
+              xfer += iprot->readString(this->partValues[_i21]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.partValues = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->columnName);
+          this->__isset.columnName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1007,25 +1247,37 @@ uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot)
   return xfer;
 }
 
-uint32_t HiveObjectPrivilege::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("HiveObjectPrivilege");
+  xfer += oprot->writeStructBegin("HiveObjectRef");
 
-  xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->hiveObject.write(oprot);
+  xfer += oprot->writeFieldBegin("objectType", ::apache::thrift::protocol::T_I32, 1);
+  xfer += oprot->writeI32((int32_t)this->objectType);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->principalName);
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->dbName);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 3);
-  xfer += oprot->writeI32((int32_t)this->principalType);
+  xfer += oprot->writeFieldBegin("objectName", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->objectName);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("grantInfo", ::apache::thrift::protocol::T_STRUCT, 4);
-  xfer += this->grantInfo.write(oprot);
+  xfer += oprot->writeFieldBegin("partValues", ::apache::thrift::protocol::T_LIST, 4);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partValues.size()));
+    std::vector<std::string> ::const_iterator _iter22;
+    for (_iter22 = this->partValues.begin(); _iter22 != this->partValues.end(); ++_iter22)
+    {
+      xfer += oprot->writeString((*_iter22));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("columnName", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeString(this->columnName);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1033,52 +1285,72 @@ uint32_t HiveObjectPrivilege::write(::apache::thrift::protocol::TProtocol* oprot
   return xfer;
 }
 
-void swap(HiveObjectPrivilege &a, HiveObjectPrivilege &b) {
+void swap(HiveObjectRef &a, HiveObjectRef &b) {
   using ::std::swap;
-  swap(a.hiveObject, b.hiveObject);
-  swap(a.principalName, b.principalName);
-  swap(a.principalType, b.principalType);
-  swap(a.grantInfo, b.grantInfo);
+  swap(a.objectType, b.objectType);
+  swap(a.dbName, b.dbName);
+  swap(a.objectName, b.objectName);
+  swap(a.partValues, b.partValues);
+  swap(a.columnName, b.columnName);
   swap(a.__isset, b.__isset);
 }
 
-HiveObjectPrivilege::HiveObjectPrivilege(const HiveObjectPrivilege& other25) {
-  hiveObject = other25.hiveObject;
-  principalName = other25.principalName;
-  principalType = other25.principalType;
-  grantInfo = other25.grantInfo;
-  __isset = other25.__isset;
+HiveObjectRef::HiveObjectRef(const HiveObjectRef& other23) {
+  objectType = other23.objectType;
+  dbName = other23.dbName;
+  objectName = other23.objectName;
+  partValues = other23.partValues;
+  columnName = other23.columnName;
+  __isset = other23.__isset;
 }
-HiveObjectPrivilege& HiveObjectPrivilege::operator=(const HiveObjectPrivilege& other26) {
-  hiveObject = other26.hiveObject;
-  principalName = other26.principalName;
-  principalType = other26.principalType;
-  grantInfo = other26.grantInfo;
-  __isset = other26.__isset;
+HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other24) {
+  objectType = other24.objectType;
+  dbName = other24.dbName;
+  objectName = other24.objectName;
+  partValues = other24.partValues;
+  columnName = other24.columnName;
+  __isset = other24.__isset;
   return *this;
 }
-void HiveObjectPrivilege::printTo(std::ostream& out) const {
+void HiveObjectRef::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "HiveObjectPrivilege(";
-  out << "hiveObject=" << to_string(hiveObject);
-  out << ", " << "principalName=" << to_string(principalName);
-  out << ", " << "principalType=" << to_string(principalType);
-  out << ", " << "grantInfo=" << to_string(grantInfo);
+  out << "HiveObjectRef(";
+  out << "objectType=" << to_string(objectType);
+  out << ", " << "dbName=" << to_string(dbName);
+  out << ", " << "objectName=" << to_string(objectName);
+  out << ", " << "partValues=" << to_string(partValues);
+  out << ", " << "columnName=" << to_string(columnName);
   out << ")";
 }
 
 
-PrivilegeBag::~PrivilegeBag() throw() {
+PrivilegeGrantInfo::~PrivilegeGrantInfo() throw() {
 }
 
 
-void PrivilegeBag::__set_privileges(const std::vector<HiveObjectPrivilege> & val) {
-  this->privileges = val;
+void PrivilegeGrantInfo::__set_privilege(const std::string& val) {
+  this->privilege = val;
 }
 
-uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
+void PrivilegeGrantInfo::__set_createTime(const int32_t val) {
+  this->createTime = val;
+}
 
-  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+void PrivilegeGrantInfo::__set_grantor(const std::string& val) {
+  this->grantor = val;
+}
+
+void PrivilegeGrantInfo::__set_grantorType(const PrincipalType::type val) {
+  this->grantorType = val;
+}
+
+void PrivilegeGrantInfo::__set_grantOption(const bool val) {
+  this->grantOption = val;
+}
+
+uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
   std::string fname;
   ::apache::thrift::protocol::TType ftype;
@@ -1098,21 +1370,43 @@ uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->privileges.clear();
-            uint32_t _size27;
-            ::apache::thrift::protocol::TType _etype30;
-            xfer += iprot->readListBegin(_etype30, _size27);
-            this->privileges.resize(_size27);
-            uint32_t _i31;
-            for (_i31 = 0; _i31 < _size27; ++_i31)
-            {
-              xfer += this->privileges[_i31].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.privileges = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->privilege);
+          this->__isset.privilege = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->createTime);
+          this->__isset.createTime = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->grantor);
+          this->__isset.grantor = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast25;
+          xfer += iprot->readI32(ecast25);
+          this->grantorType = (PrincipalType::type)ecast25;
+          this->__isset.grantorType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->grantOption);
+          this->__isset.grantOption = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1129,21 +1423,29 @@ uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
   return xfer;
 }
 
-uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t PrivilegeGrantInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("PrivilegeBag");
+  xfer += oprot->writeStructBegin("PrivilegeGrantInfo");
 
-  xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_LIST, 1);
-  {
-    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->privileges.size()));
-    std::vector<HiveObjectPrivilege> ::const_iterator _iter32;
-    for (_iter32 = this->privileges.begin(); _iter32 != this->privileges.end(); ++_iter32)
-    {
-      xfer += (*_iter32).write(oprot);
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("privilege", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->privilege);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2);
+  xfer += oprot->writeI32(this->createTime);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->grantor);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 4);
+  xfer += oprot->writeI32((int32_t)this->grantorType);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 5);
+  xfer += oprot->writeBool(this->grantOption);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1151,46 +1453,66 @@ uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const
   return xfer;
 }
 
-void swap(PrivilegeBag &a, PrivilegeBag &b) {
+void swap(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b) {
   using ::std::swap;
-  swap(a.privileges, b.privileges);
+  swap(a.privilege, b.privilege);
+  swap(a.createTime, b.createTime);
+  swap(a.grantor, b.grantor);
+  swap(a.grantorType, b.grantorType);
+  swap(a.grantOption, b.grantOption);
   swap(a.__isset, b.__isset);
 }
 
-PrivilegeBag::PrivilegeBag(const PrivilegeBag& other33) {
-  privileges = other33.privileges;
-  __isset = other33.__isset;
+PrivilegeGrantInfo::PrivilegeGrantInfo(const PrivilegeGrantInfo& other26) {
+  privilege = other26.privilege;
+  createTime = other26.createTime;
+  grantor = other26.grantor;
+  grantorType = other26.grantorType;
+  grantOption = other26.grantOption;
+  __isset = other26.__isset;
 }
-PrivilegeBag& PrivilegeBag::operator=(const PrivilegeBag& other34) {
-  privileges = other34.privileges;
-  __isset = other34.__isset;
+PrivilegeGrantInfo& PrivilegeGrantInfo::operator=(const PrivilegeGrantInfo& other27) {
+  privilege = other27.privilege;
+  createTime = other27.createTime;
+  grantor = other27.grantor;
+  grantorType = other27.grantorType;
+  grantOption = other27.grantOption;
+  __isset = other27.__isset;
   return *this;
 }
-void PrivilegeBag::printTo(std::ostream& out) const {
+void PrivilegeGrantInfo::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "PrivilegeBag(";
-  out << "privileges=" << to_string(privileges);
+  out << "PrivilegeGrantInfo(";
+  out << "privilege=" << to_string(privilege);
+  out << ", " << "createTime=" << to_string(createTime);
+  out << ", " << "grantor=" << to_string(grantor);
+  out << ", " << "grantorType=" << to_string(grantorType);
+  out << ", " << "grantOption=" << to_string(grantOption);
   out << ")";
 }
 
 
-PrincipalPrivilegeSet::~PrincipalPrivilegeSet() throw() {
+HiveObjectPrivilege::~HiveObjectPrivilege() throw() {
 }
 
 
-void PrincipalPrivilegeSet::__set_userPrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
-  this->userPrivileges = val;
+void HiveObjectPrivilege::__set_hiveObject(const HiveObjectRef& val) {
+  this->hiveObject = val;
 }
 
-void PrincipalPrivilegeSet::__set_groupPrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
-  this->groupPrivileges = val;
+void HiveObjectPrivilege::__set_principalName(const std::string& val) {
+  this->principalName = val;
 }
 
-void PrincipalPrivilegeSet::__set_rolePrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
-  this->rolePrivileges = val;
+void HiveObjectPrivilege::__set_principalType(const PrincipalType::type val) {
+  this->principalType = val;
 }
 
-uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* iprot) {
+void HiveObjectPrivilege::__set_grantInfo(const PrivilegeGrantInfo& val) {
+  this->grantInfo = val;
+}
+
+uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1212,106 +1534,35 @@ uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* ipro
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            this->userPrivileges.clear();
-            uint32_t _size35;
-            ::apache::thrift::protocol::TType _ktype36;
-            ::apache::thrift::protocol::TType _vtype37;
-            xfer += iprot->readMapBegin(_ktype36, _vtype37, _size35);
-            uint32_t _i39;
-            for (_i39 = 0; _i39 < _size35; ++_i39)
-            {
-              std::string _key40;
-              xfer += iprot->readString(_key40);
-              std::vector<PrivilegeGrantInfo> & _val41 = this->userPrivileges[_key40];
-              {
-                _val41.clear();
-                uint32_t _size42;
-                ::apache::thrift::protocol::TType _etype45;
-                xfer += iprot->readListBegin(_etype45, _size42);
-                _val41.resize(_size42);
-                uint32_t _i46;
-                for (_i46 = 0; _i46 < _size42; ++_i46)
-                {
-                  xfer += _val41[_i46].read(iprot);
-                }
-                xfer += iprot->readListEnd();
-              }
-            }
-            xfer += iprot->readMapEnd();
-          }
-          this->__isset.userPrivileges = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->hiveObject.read(iprot);
+          this->__isset.hiveObject = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            this->groupPrivileges.clear();
-            uint32_t _size47;
-            ::apache::thrift::protocol::TType _ktype48;
-            ::apache::thrift::protocol::TType _vtype49;
-            xfer += iprot->readMapBegin(_ktype48, _vtype49, _size47);
-            uint32_t _i51;
-            for (_i51 = 0; _i51 < _size47; ++_i51)
-            {
-              std::string _key52;
-              xfer += iprot->readString(_key52);
-              std::vector<PrivilegeGrantInfo> & _val53 = this->groupPrivileges[_key52];
-              {
-                _val53.clear();
-                uint32_t _size54;
-                ::apache::thrift::protocol::TType _etype57;
-                xfer += iprot->readListBegin(_etype57, _size54);
-                _val53.resize(_size54);
-                uint32_t _i58;
-                for (_i58 = 0; _i58 < _size54; ++_i58)
-                {
-                  xfer += _val53[_i58].read(iprot);
-                }
-                xfer += iprot->readListEnd();
-              }
-            }
-            xfer += iprot->readMapEnd();
-          }
-          this->__isset.groupPrivileges = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->principalName);
+          this->__isset.principalName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            this->rolePrivileges.clear();
-            uint32_t _size59;
-            ::apache::thrift::protocol::TType _ktype60;
-            ::apache::thrift::protocol::TType _vtype61;
-            xfer += iprot->readMapBegin(_ktype60, _vtype61, _size59);
-            uint32_t _i63;
-            for (_i63 = 0; _i63 < _size59; ++_i63)
-            {
-              std::string _key64;
-              xfer += iprot->readString(_key64);
-              std::vector<PrivilegeGrantInfo> & _val65 = this->rolePrivileges[_key64];
-              {
-                _val65.clear();
-                uint32_t _size66;
-                ::apache::thrift::protocol::TType _etype69;
-                xfer += iprot->readListBegin(_etype69, _size66);
-                _val65.resize(_size66);
-                uint32_t _i70;
-                for (_i70 = 0; _i70 < _size66; ++_i70)
-                {
-                  xfer += _val65[_i70].read(iprot);
-                }
-                xfer += iprot->readListEnd();
-              }
-            }
-            xfer += iprot->readMapEnd();
-          }
-          this->__isset.rolePrivileges = true;
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast28;
+          xfer += iprot->readI32(ecast28);
+          this->principalType = (PrincipalType::type)ecast28;
+          this->__isset.principalType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->grantInfo.read(iprot);
+          this->__isset.grantInfo = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1328,72 +1579,25 @@ uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* ipro
   return xfer;
 }
 
-uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t HiveObjectPrivilege::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("PrincipalPrivilegeSet");
+  xfer += oprot->writeStructBegin("HiveObjectPrivilege");
 
-  xfer += oprot->writeFieldBegin("userPrivileges", ::apache::thrift::protocol::T_MAP, 1);
-  {
-    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->userPrivileges.size()));
-    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter71;
-    for (_iter71 = this->userPrivileges.begin(); _iter71 != this->userPrivileges.end(); ++_iter71)
-    {
-      xfer += oprot->writeString(_iter71->first);
-      {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter71->second.size()));
-        std::vector<PrivilegeGrantInfo> ::const_iterator _iter72;
-        for (_iter72 = _iter71->second.begin(); _iter72 != _iter71->second.end(); ++_iter72)
-        {
-          xfer += (*_iter72).write(oprot);
-        }
-        xfer += oprot->writeListEnd();
-      }
-    }
-    xfer += oprot->writeMapEnd();
-  }
+  xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->hiveObject.write(oprot);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("groupPrivileges", ::apache::thrift::protocol::T_MAP, 2);
-  {
-    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->groupPrivileges.size()));
-    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter73;
-    for (_iter73 = this->groupPrivileges.begin(); _iter73 != this->groupPrivileges.end(); ++_iter73)
-    {
-      xfer += oprot->writeString(_iter73->first);
-      {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter73->second.size()));
-        std::vector<PrivilegeGrantInfo> ::const_iterator _iter74;
-        for (_iter74 = _iter73->second.begin(); _iter74 != _iter73->second.end(); ++_iter74)
-        {
-          xfer += (*_iter74).write(oprot);
-        }
-        xfer += oprot->writeListEnd();
-      }
-    }
-    xfer += oprot->writeMapEnd();
-  }
+  xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->principalName);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("rolePrivileges", ::apache::thrift::protocol::T_MAP, 3);
-  {
-    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->rolePrivileges.size()));
-    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter75;
-    for (_iter75 = this->rolePrivileges.begin(); _iter75 != this->rolePrivileges.end(); ++_iter75)
-    {
-      xfer += oprot->writeString(_iter75->first);
-      {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter75->second.size()));
-        std::vector<PrivilegeGrantInfo> ::const_iterator _iter76;
-        for (_iter76 = _iter75->second.begin(); _iter76 != _iter75->second.end(); ++_iter76)
-        {
-          xfer += (*_iter76).write(oprot);
-        }
-        xfer += oprot->writeListEnd();
-      }
-    }
-    xfer += oprot->writeMapEnd();
-  }
+  xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 3);
+  xfer += oprot->writeI32((int32_t)this->principalType);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantInfo", ::apache::thrift::protocol::T_STRUCT, 4);
+  xfer += this->grantInfo.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1401,55 +1605,50 @@ uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* opr
   return xfer;
 }
 
-void swap(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b) {
+void swap(HiveObjectPrivilege &a, HiveObjectPrivilege &b) {
   using ::std::swap;
-  swap(a.userPrivileges, b.userPrivileges);
-  swap(a.groupPrivileges, b.groupPrivileges);
-  swap(a.rolePrivileges, b.rolePrivileges);
+  swap(a.hiveObject, b.hiveObject);
+  swap(a.principalName, b.principalName);
+  swap(a.principalType, b.principalType);
+  swap(a.grantInfo, b.grantInfo);
   swap(a.__isset, b.__isset);
 }
 
-PrincipalPrivilegeSet::PrincipalPrivilegeSet(const PrincipalPrivilegeSet& other77) {
-  userPrivileges = other77.userPrivileges;
-  groupPrivileges = other77.groupPrivileges;
-  rolePrivileges = other77.rolePrivileges;
-  __isset = other77.__isset;
+HiveObjectPrivilege::HiveObjectPrivilege(const HiveObjectPrivilege& other29) {
+  hiveObject = other29.hiveObject;
+  principalName = other29.principalName;
+  principalType = other29.principalType;
+  grantInfo = other29.grantInfo;
+  __isset = other29.__isset;
 }
-PrincipalPrivilegeSet& PrincipalPrivilegeSet::operator=(const PrincipalPrivilegeSet& other78) {
-  userPrivileges = other78.userPrivileges;
-  groupPrivileges = other78.groupPrivileges;
-  rolePrivileges = other78.rolePrivileges;
-  __isset = other78.__isset;
+HiveObjectPrivilege& HiveObjectPrivilege::operator=(const HiveObjectPrivilege& other30) {
+  hiveObject = other30.hiveObject;
+  principalName = other30.principalName;
+  principalType = other30.principalType;
+  grantInfo = other30.grantInfo;
+  __isset = other30.__isset;
   return *this;
 }
-void PrincipalPrivilegeSet::printTo(std::ostream& out) const {
+void HiveObjectPrivilege::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "PrincipalPrivilegeSet(";
-  out << "userPrivileges=" << to_string(userPrivileges);
-  out << ", " << "groupPrivileges=" << to_string(groupPrivileges);
-  out << ", " << "rolePrivileges=" << to_string(rolePrivileges);
+  out << "HiveObjectPrivilege(";
+  out << "hiveObject=" << to_string(hiveObject);
+  out << ", " << "principalName=" << to_string(principalName);
+  out << ", " << "principalType=" << to_string(principalType);
+  out << ", " << "grantInfo=" << to_string(grantInfo);
   out << ")";
 }
 
 
-GrantRevokePrivilegeRequest::~GrantRevokePrivilegeRequest() throw() {
+PrivilegeBag::~PrivilegeBag() throw() {
 }
 
 
-void GrantRevokePrivilegeRequest::__set_requestType(const GrantRevokeType::type val) {
-  this->requestType = val;
-}
-
-void GrantRevokePrivilegeRequest::__set_privileges(const PrivilegeBag& val) {
+void PrivilegeBag::__set_privileges(const std::vector<HiveObjectPrivilege> & val) {
   this->privileges = val;
 }
 
-void GrantRevokePrivilegeRequest::__set_revokeGrantOption(const bool val) {
-  this->revokeGrantOption = val;
-__isset.revokeGrantOption = true;
-}
-
-uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1471,31 +1670,25 @@ uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast79;
-          xfer += iprot->readI32(ecast79);
-          this->requestType = (GrantRevokeType::type)ecast79;
-          this->__isset.requestType = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->privileges.read(iprot);
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->privileges.clear();
+            uint32_t _size31;
+            ::apache::thrift::protocol::TType _etype34;
+            xfer += iprot->readListBegin(_etype34, _size31);
+            this->privileges.resize(_size31);
+            uint32_t _i35;
+            for (_i35 = 0; _i35 < _size31; ++_i35)
+            {
+              xfer += this->privileges[_i35].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
           this->__isset.privileges = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->revokeGrantOption);
-          this->__isset.revokeGrantOption = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -1508,70 +1701,68 @@ uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol
   return xfer;
 }
 
-uint32_t GrantRevokePrivilegeRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("GrantRevokePrivilegeRequest");
-
-  xfer += oprot->writeFieldBegin("requestType", ::apache::thrift::protocol::T_I32, 1);
-  xfer += oprot->writeI32((int32_t)this->requestType);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("PrivilegeBag");
 
-  xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 2);
-  xfer += this->privileges.write(oprot);
+  xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->privileges.size()));
+    std::vector<HiveObjectPrivilege> ::const_iterator _iter36;
+    for (_iter36 = this->privileges.begin(); _iter36 != this->privileges.end(); ++_iter36)
+    {
+      xfer += (*_iter36).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
   xfer += oprot->writeFieldEnd();
 
-  if (this->__isset.revokeGrantOption) {
-    xfer += oprot->writeFieldBegin("revokeGrantOption", ::apache::thrift::protocol::T_BOOL, 3);
-    xfer += oprot->writeBool(this->revokeGrantOption);
-    xfer += oprot->writeFieldEnd();
-  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b) {
+void swap(PrivilegeBag &a, PrivilegeBag &b) {
   using ::std::swap;
-  swap(a.requestType, b.requestType);
   swap(a.privileges, b.privileges);
-  swap(a.revokeGrantOption, b.revokeGrantOption);
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokePrivilegeRequest::GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest& other80) {
-  requestType = other80.requestType;
-  privileges = other80.privileges;
-  revokeGrantOption = other80.revokeGrantOption;
-  __isset = other80.__isset;
+PrivilegeBag::PrivilegeBag(const PrivilegeBag& other37) {
+  privileges = other37.privileges;
+  __isset = other37.__isset;
 }
-GrantRevokePrivilegeRequest& GrantRevokePrivilegeRequest::operator=(const GrantRevokePrivilegeRequest& other81) {
-  requestType = other81.requestType;
-  privileges = other81.privileges;
-  revokeGrantOption = other81.revokeGrantOption;
-  __isset = other81.__isset;
+PrivilegeBag& PrivilegeBag::operator=(const PrivilegeBag& other38) {
+  privileges = other38.privileges;
+  __isset = other38.__isset;
   return *this;
 }
-void GrantRevokePrivilegeRequest::printTo(std::ostream& out) const {
+void PrivilegeBag::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "GrantRevokePrivilegeRequest(";
-  out << "requestType=" << to_string(requestType);
-  out << ", " << "privileges=" << to_string(privileges);
-  out << ", " << "revokeGrantOption="; (__isset.revokeGrantOption ? (out << to_string(revokeGrantOption)) : (out << "<null>"));
+  out << "PrivilegeBag(";
+  out << "privileges=" << to_string(privileges);
   out << ")";
 }
 
 
-GrantRevokePrivilegeResponse::~GrantRevokePrivilegeResponse() throw() {
+PrincipalPrivilegeSet::~PrincipalPrivilegeSet() throw() {
 }
 
 
-void GrantRevokePrivilegeResponse::__set_success(const bool val) {
-  this->success = val;
-__isset.success = true;
+void PrincipalPrivilegeSet::__set_userPrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
+  this->userPrivileges = val;
 }
 
-uint32_t GrantRevokePrivilegeResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
+void PrincipalPrivilegeSet::__set_groupPrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
+  this->groupPrivileges = val;
+}
+
+void PrincipalPrivilegeSet::__set_rolePrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
+  this->rolePrivileges = val;
+}
+
+uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1593,9 +1784,106 @@ uint32_t GrantRevokePrivilegeResponse::read(::apache::thrift::protocol::TProtoco
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->success);
-          this->__isset.success = true;
+        if (ftype == ::apache::thrift::protocol::T_MAP) {
+          {
+            this->userPrivileges.clear();
+            uint32_t _size39;
+            ::apache::thrift::protocol::TType _ktype40;
+            ::apache::thrift::protocol::TType _vtype41;
+            xfer += iprot->readMapBegin(_ktype40, _vtype41, _size39);
+            uint32_t _i43;
+            for (_i43 = 0; _i43 < _size39; ++_i43)
+            {
+              std::string _key44;
+              xfer += iprot->readString(_key44);
+              std::vector<PrivilegeGrantInfo> & _val45 = this->userPrivileges[_key44];
+              {
+                _val45.clear();
+                uint32_t _size46;
+                ::apache::thrift::protocol::TType _etype49;
+                xfer += iprot->readListBegin(_etype49, _size46);
+                _val45.resize(_size46);
+                uint32_t _i50;
+                for (_i50 = 0; _i50 < _size46; ++_i50)
+                {
+                  xfer += _val45[_i50].read(iprot);
+                }
+                xfer += iprot->readListEnd();
+              }
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.userPrivileges = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_MAP) {
+          {
+            this->groupPrivileges.clear();
+            uint32_t _size51;
+            ::apache::thrift::protocol::TType _ktype52;
+            ::apache::thrift::protocol::TType _vtype53;
+            xfer += iprot->readMapBegin(_ktype52, _vtype53, _size51);
+            uint32_t _i55;
+            for (_i55 = 0; _i55 < _size51; ++_i55)
+            {
+              std::string _key56;
+              xfer += iprot->readString(_key56);
+              std::vector<PrivilegeGrantInfo> & _val57 = this->groupPrivileges[_key56];
+              {
+                _val57.clear();
+                uint32_t _size58;
+                ::apache::thrift::protocol::TType _etype61;
+                xfer += iprot->readListBegin(_etype61, _size58);
+                _val57.resize(_size58);
+                uint32_t _i62;
+                for (_i62 = 0; _i62 < _size58; ++_i62)
+                {
+                  xfer += _val57[_i62].read(iprot);
+                }
+                xfer += iprot->readListEnd();
+              }
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.groupPrivileges = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_MAP) {
+          {
+            this->rolePrivileges.clear();
+            uint32_t _size63;
+            ::apache::thrift::protocol::TType _ktype64;
+            ::apache::thrift::protocol::TType _vtype65;
+            xfer += iprot->readMapBegin(_ktype64, _vtype65, _size63);
+            uint32_t _i67;
+            for (_i67 = 0; _i67 < _size63; ++_i67)
+            {
+              std::string _key68;
+              xfer += iprot->readString(_key68);
+              std::vector<PrivilegeGrantInfo> & _val69 = this->rolePrivileges[_key68];
+              {
+                _val69.clear();
+                uint32_t _size70;
+                ::apache::thrift::protocol::TType _etype73;
+                xfer += iprot->readListBegin(_etype73, _size70);
+                _val69.resize(_size70);
+                uint32_t _i74;
+                for (_i74 = 0; _i74 < _size70; ++_i74)
+                {
+                  xfer += _val69[_i74].read(iprot);
+                }
+                xfer += iprot->readListEnd();
+              }
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.rolePrivileges = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1612,61 +1900,128 @@ uint32_t GrantRevokePrivilegeResponse::read(::apache::thrift::protocol::TProtoco
   return xfer;
 }
 
-uint32_t GrantRevokePrivilegeResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("GrantRevokePrivilegeResponse");
+  xfer += oprot->writeStructBegin("PrincipalPrivilegeSet");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1);
-    xfer += oprot->writeBool(this->success);
-    xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeFieldBegin("userPrivileges", ::apache::thrift::protocol::T_MAP, 1);
+  {
+    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->userPrivileges.size()));
+    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter75;
+    for (_iter75 = this->userPrivileges.begin(); _iter75 != this->userPrivileges.end(); ++_iter75)
+    {
+      xfer += oprot->writeString(_iter75->first);
+      {
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter75->second.size()));
+        std::vector<PrivilegeGrantInfo> ::const_iterator _iter76;
+        for (_iter76 = _iter75->second.begin(); _iter76 != _iter75->second.end(); ++_iter76)
+        {
+          xfer += (*_iter76).write(oprot);
+        }
+        xfer += oprot->writeListEnd();
+      }
+    }
+    xfer += oprot->writeMapEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("groupPrivileges", ::apache::thrift::protocol::T_MAP, 2);
+  {
+    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->groupPrivileges.size()));
+    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter77;
+    for (_iter77 = this->groupPrivileges.begin(); _iter77 != this->groupPrivileges.end(); ++_iter77)
+    {
+      xfer += oprot->writeString(_iter77->first);
+      {
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter77->second.size()));
+        std::vector<PrivilegeGrantInfo> ::const_iterator _iter78;
+        for (_iter78 = _iter77->second.begin(); _iter78 != _iter77->second.end(); ++_iter78)
+        {
+          xfer += (*_iter78).write(oprot);
+        }
+        xfer += oprot->writeListEnd();
+      }
+    }
+    xfer += oprot->writeMapEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("rolePrivileges", ::apache::thrift::protocol::T_MAP, 3);
+  {
+    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->rolePrivileges.size()));
+    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter79;
+    for (_iter79 = this->rolePrivileges.begin(); _iter79 != this->rolePrivileges.end(); ++_iter79)
+    {
+      xfer += oprot->writeString(_iter79->first);
+      {
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter79->second.size()));
+        std::vector<PrivilegeGrantInfo> ::const_iterator _iter80;
+        for (_iter80 = _iter79->second.begin(); _iter80 != _iter79->second.end(); ++_iter80)
+        {
+          xfer += (*_iter80).write(oprot);
+        }
+        xfer += oprot->writeListEnd();
+      }
+    }
+    xfer += oprot->writeMapEnd();
   }
+  xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b) {
+void swap(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b) {
   using ::std::swap;
-  swap(a.success, b.success);
+  swap(a.userPrivileges, b.userPrivileges);
+  swap(a.groupPrivileges, b.groupPrivileges);
+  swap(a.rolePrivileges, b.rolePrivileges);
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokePrivilegeResponse::GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse& other82) {
-  success = other82.success;
-  __isset = other82.__isset;
+PrincipalPrivilegeSet::PrincipalPrivilegeSet(const PrincipalPrivilegeSet& other81) {
+  userPrivileges = other81.userPrivileges;
+  groupPrivileges = other81.groupPrivileges;
+  rolePrivileges = other81.rolePrivileges;
+  __isset = other81.__isset;
 }
-GrantRevokePrivilegeResponse& GrantRevokePrivilegeResponse::operator=(const GrantRevokePrivilegeResponse& other83) {
-  success = other83.success;
-  __isset = other83.__isset;
+PrincipalPrivilegeSet& PrincipalPrivilegeSet::operator=(const PrincipalPrivilegeSet& other82) {
+  userPrivileges = other82.userPrivileges;
+  groupPrivileges = other82.groupPrivileges;
+  rolePrivileges = other82.rolePrivileges;
+  __isset = other82.__isset;
   return *this;
 }
-void GrantRevokePrivilegeResponse::printTo(std::ostream& out) const {
+void PrincipalPrivilegeSet::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "GrantRevokePrivilegeResponse(";
-  out << "success="; (__isset.success ? (out << to_string(success)) : (out << "<null>"));
+  out << "PrincipalPrivilegeSet(";
+  out << "userPrivileges=" << to_string(userPrivileges);
+  out << ", " << "groupPrivileges=" << to_string(groupPrivileges);
+  out << ", " << "rolePrivileges=" << to_string(rolePrivileges);
   out << ")";
 }
 
 
-Role::~Role() throw() {
+GrantRevokePrivilegeRequest::~GrantRevokePrivilegeRequest() throw() {
 }
 
 
-void Role::__set_roleName(const std::string& val) {
-  this->roleName = val;
+void GrantRevokePrivilegeRequest::__set_requestType(const GrantRevokeType::type val) {
+  this->requestType = val;
 }
 
-void Role::__set_createTime(const int32_t val) {
-  this->createTime = val;
+void GrantRevokePrivilegeRequest::__set_privileges(const PrivilegeBag& val) {
+  this->privileges = val;
 }
 
-void Role::__set_ownerName(const std::string& val) {
-  this->ownerName = val;
+void GrantRevokePrivilegeRequest::__set_revokeGrantOption(const bool val) {
+  this->revokeGrantOption = val;
+__isset.revokeGrantOption = true;
 }
 
-uint32_t Role::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1688,25 +2043,27 @@ uint32_t Role::read(::apache::thrift::protocol::TProtocol* iprot) {
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->roleName);
-          this->__isset.roleName = true;
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast83;
+          xfer += iprot->readI32(ecast83);
+          this->requestType = (GrantRevokeType::type)ecast83;
+          this->__isset.requestType = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->createTime);
-          this->__isset.createTime = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->privileges.read(iprot);
+          this->__isset.privileges = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->ownerName);
-          this->__isset.ownerName = true;
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->revokeGrantOption);
+          this->__isset.revokeGrantOption = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1723,14 +2080,229 @@ uint32_t Role::read(::apache::thrift::protocol::TProtocol* iprot) {
   return xfer;
 }
 
-uint32_t Role::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t GrantRevokePrivilegeRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("Role");
+  xfer += oprot->writeStructBegin("GrantRevokePrivilegeRequest");
 
-  xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->roleName);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeFieldBegin("requestType", ::apache::thrift::protocol::T_I32, 1);
+  xfer += oprot->writeI32((int32_t)this->requestType);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 2);
+  xfer += this->privileges.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.revokeGrantOption) {
+    xfer += oprot->writeFieldBegin("revokeGrantOption", ::apache::thrift::protocol::T_BOOL, 3);
+    xfer += oprot->writeBool(this->revokeGrantOption);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b) {
+  using ::std::swap;
+  swap(a.requestType, b.requestType);
+  swap(a.privileges, b.privileges);
+  swap(a.revokeGrantOption, b.revokeGrantOption);
+  swap(a.__isset, b.__isset);
+}
+
+GrantRevokePrivilegeRequest::GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest& other84) {
+  requestType = other84.requestType;
+  privileges = other84.privileges;
+  revokeGrantOption = other84.revokeGrantOption;
+  __isset = other84.__isset;
+}
+GrantRevokePrivilegeRequest& GrantRevokePrivilegeRequest::operator=(const GrantRevokePrivilegeRequest& other85) {
+  requestType = other85.requestType;
+  privileges = other85.privileges;
+  revokeGrantOption = other85.revokeGrantOption;
+  __isset = other85.__isset;
+  return *this;
+}
+void GrantRevokePrivilegeRequest::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GrantRevokePrivilegeRequest(";
+  out << "requestType=" << to_string(requestType);
+  out << ", " << "privileges=" << to_string(privileges);
+  out << ", " << "revokeGrantOption="; (__isset.revokeGrantOption ? (out << to_string(revokeGrantOption)) : (out << "<null>"));
+  out << ")";
+}
+
+
+GrantRevokePrivilegeResponse::~GrantRevokePrivilegeResponse() throw() {
+}
+
+
+void GrantRevokePrivilegeResponse::__set_success(const bool val) {
+  this->success = val;
+__isset.success = true;
+}
+
+uint32_t GrantRevokePrivilegeResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*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_BOOL) {
+          xfer += iprot->readBool(this->success);
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t GrantRevokePrivilegeResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GrantRevokePrivilegeResponse");
+
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1);
+    xfer += oprot->writeBool(this->success);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b) {
+  using ::std::swap;
+  swap(a.success, b.success);
+  swap(a.__isset, b.__isset);
+}
+
+GrantRevokePrivilegeResponse::GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse& other86) {
+  success = other86.success;
+  __isset = other86.__isset;
+}
+GrantRevokePrivilegeResponse& GrantRevokePrivilegeResponse::operator=(const GrantRevokePrivilegeResponse& other87) {
+  success = other87.success;
+  __isset = other87.__isset;
+  return *this;
+}
+void GrantRevokePrivilegeResponse::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GrantRevokePrivilegeResponse(";
+  out << "success="; (__isset.success ? (out << to_string(success)) : (out << "<null>"));
+  out << ")";
+}
+
+
+Role::~Role() throw() {
+}
+
+
+void Role::__set_roleName(const std::string& val) {
+  this->roleName = val;
+}
+
+void Role::__set_createTime(const int32_t val) {
+  this->createTime = val;
+}
+
+void Role::__set_ownerName(const std::string& val) {
+  this->ownerName = val;
+}
+
+uint32_t Role::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*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->roleName);
+          this->__isset.roleName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->createTime);
+          this->__isset.createTime = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->ownerName);
+          this->__isset.ownerName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t Role::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("Role");
+
+  xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->roleName);
+  xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2);
   xfer += oprot->writeI32(this->createTime);
@@ -1753,17 +2325,17 @@ void swap(Role &a, Role &b) {
   swap(a.__isset, b.__isset);
 }
 
-Role::Role(const Role& other84) {
-  roleName = other84.roleName;
-  createTime = other84.createTime;
-  ownerName = other84.ownerName;
-  __isset = other84.__isset;
+Role::Role(const Role& other88) {
+  roleName = other88.roleName;
+  createTime = other88.createTime;
+  ownerName = other88.ownerName;
+  __isset = other88.__isset;
 }
-Role& Role::operator=(const Role& other85) {
-  roleName = other85.roleName;
-  createTime = other85.createTime;
-  ownerName = other85.ownerName;
-  __isset = other85.__isset;
+Role& Role::operator=(const Role& other89) {
+  roleName = other89.roleName;
+  createTime = other89.createTime;
+  ownerName = other89.ownerName;
+  __isset = other89.__isset;
   return *this;
 }
 void Role::printTo(std::ostream& out) const {
@@ -1847,9 +2419,9 @@ uint32_t RolePrincipalGrant::read(::apache::thrift::protocol::TProtocol* iprot)
         break;
  

<TRUNCATED>

[16/30] hive git commit: HIVE-13349: Metastore Changes : API calls for retrieving primary keys and foreign keys information (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/55375ec1/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 516b926..3ec46f1 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -42,6 +42,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   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 create_table_with_constraints(Table tbl,  primaryKeys,  foreignKeys)')
   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)')
@@ -101,6 +102,8 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   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('  PrimaryKeysResponse get_primary_keys(PrimaryKeysRequest request)')
+  print('  ForeignKeysResponse get_foreign_keys(ForeignKeysRequest request)')
   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)')
@@ -344,6 +347,12 @@ elif cmd == 'create_table_with_environment_context':
     sys.exit(1)
   pp.pprint(client.create_table_with_environment_context(eval(args[0]),eval(args[1]),))
 
+elif cmd == 'create_table_with_constraints':
+  if len(args) != 3:
+    print('create_table_with_constraints requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.create_table_with_constraints(eval(args[0]),eval(args[1]),eval(args[2]),))
+
 elif cmd == 'drop_table':
   if len(args) != 3:
     print('drop_table requires 3 args')
@@ -698,6 +707,18 @@ elif cmd == 'get_index_names':
     sys.exit(1)
   pp.pprint(client.get_index_names(args[0],args[1],eval(args[2]),))
 
+elif cmd == 'get_primary_keys':
+  if len(args) != 1:
+    print('get_primary_keys requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_primary_keys(eval(args[0]),))
+
+elif cmd == 'get_foreign_keys':
+  if len(args) != 1:
+    print('get_foreign_keys requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_foreign_keys(eval(args[0]),))
+
 elif cmd == 'update_table_column_statistics':
   if len(args) != 1:
     print('update_table_column_statistics requires 1 args')


[07/30] hive git commit: HIVE-13501: Invoke failure hooks if query fails on exception (Jimmy, reviewed by Szehon)

Posted by jd...@apache.org.
HIVE-13501: Invoke failure hooks if query fails on exception (Jimmy, reviewed by Szehon)


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

Branch: refs/heads/llap
Commit: ff6ddfb3cf4640f65ebd184e89e587c94fa0131e
Parents: 88cea1c
Author: Jimmy Xiang <jx...@apache.org>
Authored: Tue Apr 12 14:26:43 2016 -0700
Committer: Jimmy Xiang <jx...@apache.org>
Committed: Fri Apr 15 17:29:10 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 39 ++++++++++++++------
 1 file changed, 27 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ff6ddfb3/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 92c2c76..6062ac0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -98,11 +98,11 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
-import org.apache.hadoop.hive.ql.security.authorization.plugin.QueryContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivObjectActionType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.QueryContext;
 import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hadoop.hive.ql.session.OperationLog.LoggingLevel;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -1505,6 +1505,8 @@ public class Driver implements CommandProcessor {
 
     maxthreads = HiveConf.getIntVar(conf, HiveConf.ConfVars.EXECPARALLETHREADNUMBER);
 
+    HookContext hookContext = null;
+
     try {
       LOG.info("Executing command(queryId=" + queryId + "): " + queryStr);
       // compile and execute can get called from different threads in case of HS2
@@ -1521,7 +1523,7 @@ public class Driver implements CommandProcessor {
       resStream = null;
 
       SessionState ss = SessionState.get();
-      HookContext hookContext = new HookContext(plan, conf, ctx.getPathToCS(), ss.getUserName(),
+      hookContext = new HookContext(plan, conf, ctx.getPathToCS(), ss.getUserName(),
           ss.getUserIpAddress(), operationId);
       hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK);
 
@@ -1632,16 +1634,7 @@ public class Driver implements CommandProcessor {
 
           } else {
             setErrorMsgAndDetail(exitVal, result.getTaskError(), tsk);
-            hookContext.setHookType(HookContext.HookType.ON_FAILURE_HOOK);
-            hookContext.setErrorMessage(errorMessage);
-            // Get all the failure execution hooks and execute them.
-            for (Hook ofh : getHooks(HiveConf.ConfVars.ONFAILUREHOOKS)) {
-              perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
-
-              ((ExecuteWithHookContext) ofh).run(hookContext);
-
-              perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
-            }
+            invokeFailureHooks(perfLogger, hookContext);
             SQLState = "08S01";
             console.printError(errorMessage);
             driverCxt.shutdown();
@@ -1677,6 +1670,7 @@ public class Driver implements CommandProcessor {
       if (driverCxt.isShutdown()) {
         SQLState = "HY008";
         errorMessage = "FAILED: Operation cancelled";
+        invokeFailureHooks(perfLogger, hookContext);
         console.printError(errorMessage);
         return 1000;
       }
@@ -1731,6 +1725,13 @@ public class Driver implements CommandProcessor {
       }
       // TODO: do better with handling types of Exception here
       errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e);
+      if (hookContext != null) {
+        try {
+          invokeFailureHooks(perfLogger, hookContext);
+        } catch (Exception t) {
+          LOG.warn("Failed to invoke failure hook", t);
+        }
+      }
       SQLState = "08S01";
       downstreamError = e;
       console.printError(errorMessage + "\n"
@@ -1804,6 +1805,20 @@ public class Driver implements CommandProcessor {
       }
     }
   }
+
+  private void invokeFailureHooks(PerfLogger perfLogger, HookContext hookContext) throws Exception {
+    hookContext.setHookType(HookContext.HookType.ON_FAILURE_HOOK);
+    hookContext.setErrorMessage(errorMessage);
+    // Get all the failure execution hooks and execute them.
+    for (Hook ofh : getHooks(HiveConf.ConfVars.ONFAILUREHOOKS)) {
+      perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
+
+      ((ExecuteWithHookContext) ofh).run(hookContext);
+
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
+    }
+  }
+
   /**
    * Launches a new task
    *


[03/30] hive git commit: Revert "HIVE-12159: Create vectorized readers for the complex types (Owen O'Malley, reviewed by Matt McCline)"

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d559b347/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
index 460c925..a82d672 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
@@ -32,14 +32,40 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
 import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
+import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hive.common.util.HiveTestUtils;
 import org.apache.orc.BinaryColumnStatistics;
@@ -91,10 +117,6 @@ public class TestVectorOrcFile {
   public static class InnerStruct {
     int int1;
     Text string1 = new Text();
-    InnerStruct(int int1, Text string1) {
-      this.int1 = int1;
-      this.string1.set(string1);
-    }
     InnerStruct(int int1, String string1) {
       this.int1 = int1;
       this.string1.set(string1);
@@ -114,6 +136,50 @@ public class TestVectorOrcFile {
     }
   }
 
+  public static class BigRow {
+    Boolean boolean1;
+    Byte byte1;
+    Short short1;
+    Integer int1;
+    Long long1;
+    Float float1;
+    Double double1;
+    BytesWritable bytes1;
+    Text string1;
+    MiddleStruct middle;
+    List<InnerStruct> list = new ArrayList<InnerStruct>();
+    Map<Text, InnerStruct> map = new HashMap<Text, InnerStruct>();
+
+    BigRow(Boolean b1, Byte b2, Short s1, Integer i1, Long l1, Float f1,
+           Double d1,
+           BytesWritable b3, String s2, MiddleStruct m1,
+           List<InnerStruct> l2, Map<String, InnerStruct> m2) {
+      this.boolean1 = b1;
+      this.byte1 = b2;
+      this.short1 = s1;
+      this.int1 = i1;
+      this.long1 = l1;
+      this.float1 = f1;
+      this.double1 = d1;
+      this.bytes1 = b3;
+      if (s2 == null) {
+        this.string1 = null;
+      } else {
+        this.string1 = new Text(s2);
+      }
+      this.middle = m1;
+      this.list = l2;
+      if (m2 != null) {
+        this.map = new HashMap<Text, InnerStruct>();
+        for (Map.Entry<String, InnerStruct> item : m2.entrySet()) {
+          this.map.put(new Text(item.getKey()), item.getValue());
+        }
+      } else {
+        this.map = null;
+      }
+    }
+  }
+
   private static InnerStruct inner(int i, String s) {
     return new InnerStruct(i, s);
   }
@@ -235,115 +301,206 @@ public class TestVectorOrcFile {
     assertEquals("count: 7500 hasNull: true min: bye max: hi sum: 0", stats[9].toString());
 
     // check the inspectors
-    TypeDescription schema = reader.getSchema();
-    assertEquals(TypeDescription.Category.STRUCT, schema.getCategory());
+    StructObjectInspector readerInspector = (StructObjectInspector) reader
+        .getObjectInspector();
+    assertEquals(ObjectInspector.Category.STRUCT, readerInspector.getCategory());
     assertEquals("struct<boolean1:boolean,byte1:tinyint,short1:smallint,"
         + "int1:int,long1:bigint,float1:float,double1:double,bytes1:"
         + "binary,string1:string,middle:struct<list:array<struct<int1:int,"
         + "string1:string>>>,list:array<struct<int1:int,string1:string>>,"
         + "map:map<string,struct<int1:int,string1:string>>,ts:timestamp,"
-        + "decimal1:decimal(38,10)>", schema.toString());
-    VectorizedRowBatch batch = schema.createRowBatch();
-
+        + "decimal1:decimal(38,18)>", readerInspector.getTypeName());
+    List<? extends StructField> fields = readerInspector
+        .getAllStructFieldRefs();
+    BooleanObjectInspector bo = (BooleanObjectInspector) readerInspector
+        .getStructFieldRef("boolean1").getFieldObjectInspector();
+    ByteObjectInspector by = (ByteObjectInspector) readerInspector
+        .getStructFieldRef("byte1").getFieldObjectInspector();
+    ShortObjectInspector sh = (ShortObjectInspector) readerInspector
+        .getStructFieldRef("short1").getFieldObjectInspector();
+    IntObjectInspector in = (IntObjectInspector) readerInspector
+        .getStructFieldRef("int1").getFieldObjectInspector();
+    LongObjectInspector lo = (LongObjectInspector) readerInspector
+        .getStructFieldRef("long1").getFieldObjectInspector();
+    FloatObjectInspector fl = (FloatObjectInspector) readerInspector
+        .getStructFieldRef("float1").getFieldObjectInspector();
+    DoubleObjectInspector dbl = (DoubleObjectInspector) readerInspector
+        .getStructFieldRef("double1").getFieldObjectInspector();
+    BinaryObjectInspector bi = (BinaryObjectInspector) readerInspector
+        .getStructFieldRef("bytes1").getFieldObjectInspector();
+    StringObjectInspector st = (StringObjectInspector) readerInspector
+        .getStructFieldRef("string1").getFieldObjectInspector();
+    StructObjectInspector mid = (StructObjectInspector) readerInspector
+        .getStructFieldRef("middle").getFieldObjectInspector();
+    List<? extends StructField> midFields = mid.getAllStructFieldRefs();
+    ListObjectInspector midli = (ListObjectInspector) midFields.get(0)
+        .getFieldObjectInspector();
+    StructObjectInspector inner = (StructObjectInspector) midli
+        .getListElementObjectInspector();
+    List<? extends StructField> inFields = inner.getAllStructFieldRefs();
+    ListObjectInspector li = (ListObjectInspector) readerInspector
+        .getStructFieldRef("list").getFieldObjectInspector();
+    MapObjectInspector ma = (MapObjectInspector) readerInspector
+        .getStructFieldRef("map").getFieldObjectInspector();
+    TimestampObjectInspector tso = (TimestampObjectInspector) readerInspector
+        .getStructFieldRef("ts").getFieldObjectInspector();
+    HiveDecimalObjectInspector dco = (HiveDecimalObjectInspector) readerInspector
+        .getStructFieldRef("decimal1").getFieldObjectInspector();
+    StringObjectInspector mk = (StringObjectInspector) ma
+        .getMapKeyObjectInspector();
     RecordReader rows = reader.rows();
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1024, batch.size);
-
+    Object row = rows.next(null);
+    assertNotNull(row);
     // check the contents of the first row
-    assertEquals(false, getBoolean(batch, 0));
-    assertEquals(1, getByte(batch, 0));
-    assertEquals(1024, getShort(batch, 0));
-    assertEquals(65536, getInt(batch, 0));
-    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
-    assertEquals(1.0, getFloat(batch, 0), 0.00001);
-    assertEquals(-15.0, getDouble(batch, 0), 0.00001);
-    assertEquals(bytes(0, 1, 2, 3, 4), getBinary(batch, 0));
-    assertEquals("hi", getText(batch, 0).toString());
-
-    StructColumnVector middle = (StructColumnVector) batch.cols[9];
-    ListColumnVector midList = (ListColumnVector) middle.fields[0];
-    StructColumnVector midListStruct = (StructColumnVector) midList.child;
-    LongColumnVector midListInt = (LongColumnVector) midListStruct.fields[0];
-    BytesColumnVector midListStr = (BytesColumnVector) midListStruct.fields[1];
-    ListColumnVector list = (ListColumnVector) batch.cols[10];
-    StructColumnVector listStruct = (StructColumnVector) list.child;
-    LongColumnVector listInts = (LongColumnVector) listStruct.fields[0];
-    BytesColumnVector listStrs = (BytesColumnVector) listStruct.fields[1];
-    MapColumnVector map = (MapColumnVector) batch.cols[11];
-    BytesColumnVector mapKey = (BytesColumnVector) map.keys;
-    StructColumnVector mapValue = (StructColumnVector) map.values;
-    LongColumnVector mapValueInts = (LongColumnVector) mapValue.fields[0];
-    BytesColumnVector mapValueStrs = (BytesColumnVector) mapValue.fields[1];
-    TimestampColumnVector timestamp = (TimestampColumnVector) batch.cols[12];
-    DecimalColumnVector decs = (DecimalColumnVector) batch.cols[13];
-
-    assertEquals(false, middle.isNull[0]);
-    assertEquals(2, midList.lengths[0]);
-    int start = (int) midList.offsets[0];
-    assertEquals(1, midListInt.vector[start]);
-    assertEquals("bye", midListStr.toString(start));
-    assertEquals(2, midListInt.vector[start + 1]);
-    assertEquals("sigh", midListStr.toString(start + 1));
-
-    assertEquals(2, list.lengths[0]);
-    start = (int) list.offsets[0];
-    assertEquals(3, listInts.vector[start]);
-    assertEquals("good", listStrs.toString(start));
-    assertEquals(4, listInts.vector[start + 1]);
-    assertEquals("bad", listStrs.toString(start + 1));
-    assertEquals(0, map.lengths[0]);
+    assertEquals(false,
+        bo.get(readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals(1,
+        by.get(readerInspector.getStructFieldData(row, fields.get(1))));
+    assertEquals(1024,
+        sh.get(readerInspector.getStructFieldData(row, fields.get(2))));
+    assertEquals(65536,
+        in.get(readerInspector.getStructFieldData(row, fields.get(3))));
+    assertEquals(Long.MAX_VALUE,
+        lo.get(readerInspector.getStructFieldData(row, fields.get(4))));
+    assertEquals(1.0,
+        fl.get(readerInspector.getStructFieldData(row, fields.get(5))), 0.00001);
+    assertEquals(-15.0,
+        dbl.get(readerInspector.getStructFieldData(row, fields.get(6))),
+        0.00001);
+    assertEquals(bytes(0, 1, 2, 3, 4),
+        bi.getPrimitiveWritableObject(readerInspector.getStructFieldData(row,
+            fields.get(7))));
+    assertEquals("hi", st.getPrimitiveJavaObject(readerInspector
+        .getStructFieldData(row, fields.get(8))));
+    List<?> midRow = midli.getList(mid.getStructFieldData(
+        readerInspector.getStructFieldData(row, fields.get(9)),
+        midFields.get(0)));
+    assertNotNull(midRow);
+    assertEquals(2, midRow.size());
+    assertEquals(1,
+        in.get(inner.getStructFieldData(midRow.get(0), inFields.get(0))));
+    assertEquals("bye", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        midRow.get(0), inFields.get(1))));
+    assertEquals(2,
+        in.get(inner.getStructFieldData(midRow.get(1), inFields.get(0))));
+    assertEquals("sigh", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        midRow.get(1), inFields.get(1))));
+    List<?> list = li.getList(readerInspector.getStructFieldData(row,
+        fields.get(10)));
+    assertEquals(2, list.size());
+    assertEquals(3,
+        in.get(inner.getStructFieldData(list.get(0), inFields.get(0))));
+    assertEquals("good", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        list.get(0), inFields.get(1))));
+    assertEquals(4,
+        in.get(inner.getStructFieldData(list.get(1), inFields.get(0))));
+    assertEquals("bad", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        list.get(1), inFields.get(1))));
+    Map<?, ?> map = ma.getMap(readerInspector.getStructFieldData(row,
+        fields.get(11)));
+    assertEquals(0, map.size());
     assertEquals(Timestamp.valueOf("2000-03-12 15:00:00"),
-        timestamp.asScratchTimestamp(0));
-    assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547456")),
-        decs.vector[0]);
+        tso.getPrimitiveJavaObject(readerInspector.getStructFieldData(row,
+            fields.get(12))));
+    assertEquals(HiveDecimal.create("12345678.6547456"),
+        dco.getPrimitiveJavaObject(readerInspector.getStructFieldData(row,
+            fields.get(13))));
 
-    // check the contents of row 7499
+    // check the contents of second row
+    assertEquals(true, rows.hasNext());
     rows.seekToRow(7499);
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(true, getBoolean(batch, 0));
-    assertEquals(100, getByte(batch, 0));
-    assertEquals(2048, getShort(batch, 0));
-    assertEquals(65536, getInt(batch, 0));
-    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
-    assertEquals(2.0, getFloat(batch, 0), 0.00001);
-    assertEquals(-5.0, getDouble(batch, 0), 0.00001);
-    assertEquals(bytes(), getBinary(batch, 0));
-    assertEquals("bye", getText(batch, 0).toString());
-    assertEquals(false, middle.isNull[0]);
-    assertEquals(2, midList.lengths[0]);
-    start = (int) midList.offsets[0];
-    assertEquals(1, midListInt.vector[start]);
-    assertEquals("bye", midListStr.toString(start));
-    assertEquals(2, midListInt.vector[start + 1]);
-    assertEquals("sigh", midListStr.toString(start + 1));
-    assertEquals(3, list.lengths[0]);
-    start = (int) list.offsets[0];
-    assertEquals(100000000, listInts.vector[start]);
-    assertEquals("cat", listStrs.toString(start));
-    assertEquals(-100000, listInts.vector[start + 1]);
-    assertEquals("in", listStrs.toString(start + 1));
-    assertEquals(1234, listInts.vector[start + 2]);
-    assertEquals("hat", listStrs.toString(start + 2));
-    assertEquals(2, map.lengths[0]);
-    start = (int) map.offsets[0];
-    assertEquals("chani", mapKey.toString(start));
-    assertEquals(5, mapValueInts.vector[start]);
-    assertEquals("chani", mapValueStrs.toString(start));
-    assertEquals("mauddib", mapKey.toString(start + 1));
-    assertEquals(1, mapValueInts.vector[start + 1]);
-    assertEquals("mauddib", mapValueStrs.toString(start + 1));
+    row = rows.next(null);
+    assertEquals(true,
+        bo.get(readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals(100,
+        by.get(readerInspector.getStructFieldData(row, fields.get(1))));
+    assertEquals(2048,
+        sh.get(readerInspector.getStructFieldData(row, fields.get(2))));
+    assertEquals(65536,
+        in.get(readerInspector.getStructFieldData(row, fields.get(3))));
+    assertEquals(Long.MAX_VALUE,
+        lo.get(readerInspector.getStructFieldData(row, fields.get(4))));
+    assertEquals(2.0,
+        fl.get(readerInspector.getStructFieldData(row, fields.get(5))), 0.00001);
+    assertEquals(-5.0,
+        dbl.get(readerInspector.getStructFieldData(row, fields.get(6))),
+        0.00001);
+    assertEquals(bytes(), bi.getPrimitiveWritableObject(readerInspector
+        .getStructFieldData(row, fields.get(7))));
+    assertEquals("bye", st.getPrimitiveJavaObject(readerInspector
+        .getStructFieldData(row, fields.get(8))));
+    midRow = midli.getList(mid.getStructFieldData(
+        readerInspector.getStructFieldData(row, fields.get(9)),
+        midFields.get(0)));
+    assertNotNull(midRow);
+    assertEquals(2, midRow.size());
+    assertEquals(1,
+        in.get(inner.getStructFieldData(midRow.get(0), inFields.get(0))));
+    assertEquals("bye", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        midRow.get(0), inFields.get(1))));
+    assertEquals(2,
+        in.get(inner.getStructFieldData(midRow.get(1), inFields.get(0))));
+    assertEquals("sigh", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        midRow.get(1), inFields.get(1))));
+    list = li.getList(readerInspector.getStructFieldData(row, fields.get(10)));
+    assertEquals(3, list.size());
+    assertEquals(100000000,
+        in.get(inner.getStructFieldData(list.get(0), inFields.get(0))));
+    assertEquals("cat", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        list.get(0), inFields.get(1))));
+    assertEquals(-100000,
+        in.get(inner.getStructFieldData(list.get(1), inFields.get(0))));
+    assertEquals("in", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        list.get(1), inFields.get(1))));
+    assertEquals(1234,
+        in.get(inner.getStructFieldData(list.get(2), inFields.get(0))));
+    assertEquals("hat", st.getPrimitiveJavaObject(inner.getStructFieldData(
+        list.get(2), inFields.get(1))));
+    map = ma.getMap(readerInspector.getStructFieldData(row, fields.get(11)));
+    assertEquals(2, map.size());
+    boolean[] found = new boolean[2];
+    for(Object key : map.keySet()) {
+      String str = mk.getPrimitiveJavaObject(key);
+      if (str.equals("chani")) {
+        assertEquals(false, found[0]);
+        assertEquals(5,
+            in.get(inner.getStructFieldData(map.get(key), inFields.get(0))));
+        assertEquals(str, st.getPrimitiveJavaObject(inner.getStructFieldData(
+            map.get(key), inFields.get(1))));
+        found[0] = true;
+      } else if (str.equals("mauddib")) {
+        assertEquals(false, found[1]);
+        assertEquals(1,
+            in.get(inner.getStructFieldData(map.get(key), inFields.get(0))));
+        assertEquals(str, st.getPrimitiveJavaObject(inner.getStructFieldData(
+            map.get(key), inFields.get(1))));
+        found[1] = true;
+      } else {
+        throw new IllegalArgumentException("Unknown key " + str);
+      }
+    }
+    assertEquals(true, found[0]);
+    assertEquals(true, found[1]);
     assertEquals(Timestamp.valueOf("2000-03-12 15:00:01"),
-        timestamp.asScratchTimestamp(0));
-    assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547457")),
-        decs.vector[0]);
+        tso.getPrimitiveJavaObject(readerInspector.getStructFieldData(row,
+            fields.get(12))));
+    assertEquals(HiveDecimal.create("12345678.6547457"),
+        dco.getPrimitiveJavaObject(readerInspector.getStructFieldData(row,
+            fields.get(13))));
 
     // handle the close up
-    assertEquals(false, rows.nextBatch(batch));
+    assertEquals(false, rows.hasNext());
     rows.close();
   }
 
   @Test
   public void testTimestamp() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestVectorOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Timestamp.class,
+          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
     TypeDescription schema = TypeDescription.createTimestamp();
     Writer writer = OrcFile.createWriter(testFilePath,
         OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
@@ -376,15 +533,11 @@ public class TestVectorOrcFile {
 
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch();
-    TimestampColumnVector timestamps = (TimestampColumnVector) batch.cols[0];
+    RecordReader rows = reader.rows(null);
     int idx = 0;
-    while (rows.nextBatch(batch)) {
-      for(int r=0; r < batch.size; ++r) {
-        assertEquals(tslist.get(idx++).getNanos(),
-            timestamps.asScratchTimestamp(r).getNanos());
-      }
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(tslist.get(idx++).getNanos(), ((TimestampWritable) row).getNanos());
     }
     assertEquals(tslist.size(), rows.getRowNumber());
     assertEquals(0, writer.getSchema().getMaximumId());
@@ -455,28 +608,50 @@ public class TestVectorOrcFile {
         stats[2].toString());
 
     // check the inspectors
-    batch = reader.getSchema().createRowBatch();
-    BytesColumnVector bytes = (BytesColumnVector) batch.cols[0];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
+    StructObjectInspector readerInspector =
+        (StructObjectInspector) reader.getObjectInspector();
+    assertEquals(ObjectInspector.Category.STRUCT,
+        readerInspector.getCategory());
+    assertEquals("struct<bytes1:binary,string1:string>",
+        readerInspector.getTypeName());
+    List<? extends StructField> fields =
+        readerInspector.getAllStructFieldRefs();
+    BinaryObjectInspector bi = (BinaryObjectInspector) readerInspector.
+        getStructFieldRef("bytes1").getFieldObjectInspector();
+    StringObjectInspector st = (StringObjectInspector) readerInspector.
+        getStructFieldRef("string1").getFieldObjectInspector();
     RecordReader rows = reader.rows();
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(4, batch.size);
-
+    Object row = rows.next(null);
+    assertNotNull(row);
     // check the contents of the first row
-    assertEquals(bytes(0,1,2,3,4), getBinary(bytes, 0));
-    assertEquals("foo", strs.toString(0));
+    assertEquals(bytes(0,1,2,3,4), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("foo", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
 
     // check the contents of second row
-    assertEquals(bytes(0,1,2,3), getBinary(bytes, 1));
-    assertEquals("bar", strs.toString(1));
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertEquals(bytes(0,1,2,3), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("bar", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
 
     // check the contents of third row
-    assertEquals(bytes(0,1,2,3,4,5), getBinary(bytes, 2));
-    assertNull(strs.toString(2));
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertEquals(bytes(0,1,2,3,4,5), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertNull(st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
 
     // check the contents of fourth row
-    assertNull(getBinary(bytes, 3));
-    assertEquals("hi", strs.toString(3));
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertNull(bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("hi", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
 
     // handle the close up
     assertEquals(false, rows.hasNext());
@@ -592,19 +767,6 @@ public class TestVectorOrcFile {
     }
   }
 
-  private static void checkInner(StructColumnVector inner, int rowId,
-                                 int rowInBatch, int i, String value) {
-    assertEquals("row " + rowId, i,
-        ((LongColumnVector) inner.fields[0]).vector[rowInBatch]);
-    if (value != null) {
-      assertEquals("row " + rowId, value,
-          ((BytesColumnVector) inner.fields[1]).toString(rowInBatch));
-    } else {
-      assertEquals("row " + rowId, true, inner.fields[1].isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, inner.fields[1].noNulls);
-    }
-  }
-
   private static void setInnerList(ListColumnVector list, int rowId,
                                    List<InnerStruct> value) {
     if (value != null) {
@@ -625,23 +787,6 @@ public class TestVectorOrcFile {
     }
   }
 
-  private static void checkInnerList(ListColumnVector list, int rowId,
-                                     int rowInBatch, List<InnerStruct> value) {
-    if (value != null) {
-      assertEquals("row " + rowId, value.size(), list.lengths[rowInBatch]);
-      int start = (int) list.offsets[rowInBatch];
-      for (int i = 0; i < list.lengths[rowInBatch]; ++i) {
-        InnerStruct inner = value.get(i);
-        checkInner((StructColumnVector) list.child, rowId, i + start,
-            inner.int1, inner.string1.toString());
-      }
-      list.childCount += value.size();
-    } else {
-      assertEquals("row " + rowId, true, list.isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, list.noNulls);
-    }
-  }
-
   private static void setInnerMap(MapColumnVector map, int rowId,
                                   Map<String, InnerStruct> value) {
     if (value != null) {
@@ -667,24 +812,6 @@ public class TestVectorOrcFile {
     }
   }
 
-  private static void checkInnerMap(MapColumnVector map, int rowId,
-                                    int rowInBatch,
-                                    Map<String, InnerStruct> value) {
-    if (value != null) {
-      assertEquals("row " + rowId, value.size(), map.lengths[rowInBatch]);
-      int offset = (int) map.offsets[rowInBatch];
-      for(int i=0; i < value.size(); ++i) {
-        String key = ((BytesColumnVector) map.keys).toString(offset + i);
-        InnerStruct expected = value.get(key);
-        checkInner((StructColumnVector) map.values, rowId, offset + i,
-            expected.int1, expected.string1.toString());
-      }
-    } else {
-      assertEquals("row " + rowId, true, map.isNull[rowId]);
-      assertEquals("row " + rowId, false, map.noNulls);
-    }
-  }
-
   private static void setMiddleStruct(StructColumnVector middle, int rowId,
                                       MiddleStruct value) {
     if (value != null) {
@@ -695,17 +822,6 @@ public class TestVectorOrcFile {
     }
   }
 
-  private static void checkMiddleStruct(StructColumnVector middle, int rowId,
-                                        int rowInBatch, MiddleStruct value) {
-    if (value != null) {
-      checkInnerList((ListColumnVector) middle.fields[0], rowId, rowInBatch,
-          value.list);
-    } else {
-      assertEquals("row " + rowId, true, middle.isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, middle.noNulls);
-    }
-  }
-
   private static void setBigRow(VectorizedRowBatch batch, int rowId,
                                 Boolean b1, Byte b2, Short s1,
                                 Integer i1, Long l1, Float f1,
@@ -737,160 +853,6 @@ public class TestVectorOrcFile {
     setInnerMap((MapColumnVector) batch.cols[11], rowId, m2);
   }
 
-  private static void checkBigRow(VectorizedRowBatch batch,
-                                  int rowInBatch,
-                                  int rowId,
-                                  boolean b1, byte b2, short s1,
-                                  int i1, long l1, float f1,
-                                  double d1, BytesWritable b3, String s2,
-                                  MiddleStruct m1, List<InnerStruct> l2,
-                                  Map<String, InnerStruct> m2) {
-    assertEquals("row " + rowId, b1, getBoolean(batch, rowInBatch));
-    assertEquals("row " + rowId, b2, getByte(batch, rowInBatch));
-    assertEquals("row " + rowId, s1, getShort(batch, rowInBatch));
-    assertEquals("row " + rowId, i1, getInt(batch, rowInBatch));
-    assertEquals("row " + rowId, l1, getLong(batch, rowInBatch));
-    assertEquals("row " + rowId, f1, getFloat(batch, rowInBatch), 0.0001);
-    assertEquals("row " + rowId, d1, getDouble(batch, rowInBatch), 0.0001);
-    if (b3 != null) {
-      BytesColumnVector bytes = (BytesColumnVector) batch.cols[7];
-      assertEquals("row " + rowId, b3.getLength(), bytes.length[rowInBatch]);
-      for(int i=0; i < b3.getLength(); ++i) {
-        assertEquals("row " + rowId + " byte " + i, b3.getBytes()[i],
-            bytes.vector[rowInBatch][bytes.start[rowInBatch] + i]);
-      }
-    } else {
-      assertEquals("row " + rowId, true, batch.cols[7].isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, batch.cols[7].noNulls);
-    }
-    if (s2 != null) {
-      assertEquals("row " + rowId, s2, getText(batch, rowInBatch).toString());
-    } else {
-      assertEquals("row " + rowId, true, batch.cols[8].isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, batch.cols[8].noNulls);
-    }
-    checkMiddleStruct((StructColumnVector) batch.cols[9], rowId, rowInBatch,
-        m1);
-    checkInnerList((ListColumnVector) batch.cols[10], rowId, rowInBatch, l2);
-    checkInnerMap((MapColumnVector) batch.cols[11], rowId, rowInBatch, m2);
-  }
-
-  private static boolean getBoolean(VectorizedRowBatch batch, int rowId) {
-    return ((LongColumnVector) batch.cols[0]).vector[rowId] != 0;
-  }
-
-  private static byte getByte(VectorizedRowBatch batch, int rowId) {
-    return (byte) ((LongColumnVector) batch.cols[1]).vector[rowId];
-  }
-
-  private static short getShort(VectorizedRowBatch batch, int rowId) {
-    return (short) ((LongColumnVector) batch.cols[2]).vector[rowId];
-  }
-
-  private static int getInt(VectorizedRowBatch batch, int rowId) {
-    return (int) ((LongColumnVector) batch.cols[3]).vector[rowId];
-  }
-
-  private static long getLong(VectorizedRowBatch batch, int rowId) {
-    return ((LongColumnVector) batch.cols[4]).vector[rowId];
-  }
-
-  private static float getFloat(VectorizedRowBatch batch, int rowId) {
-    return (float) ((DoubleColumnVector) batch.cols[5]).vector[rowId];
-  }
-
-  private static double getDouble(VectorizedRowBatch batch, int rowId) {
-    return ((DoubleColumnVector) batch.cols[6]).vector[rowId];
-  }
-
-  private static BytesWritable getBinary(BytesColumnVector column, int rowId) {
-    if (column.isRepeating) {
-      rowId = 0;
-    }
-    if (column.noNulls || !column.isNull[rowId]) {
-      return new BytesWritable(Arrays.copyOfRange(column.vector[rowId],
-          column.start[rowId], column.start[rowId] + column.length[rowId]));
-    } else {
-      return null;
-    }
-  }
-
-  private static BytesWritable getBinary(VectorizedRowBatch batch, int rowId) {
-    return getBinary((BytesColumnVector) batch.cols[7], rowId);
-  }
-
-  private static Text getText(BytesColumnVector vector, int rowId) {
-    if (vector.isRepeating) {
-      rowId = 0;
-    }
-    if (vector.noNulls || !vector.isNull[rowId]) {
-      return new Text(Arrays.copyOfRange(vector.vector[rowId],
-          vector.start[rowId], vector.start[rowId] + vector.length[rowId]));
-    } else {
-      return null;
-    }
-  }
-
-  private static Text getText(VectorizedRowBatch batch, int rowId) {
-    return getText((BytesColumnVector) batch.cols[8], rowId);
-  }
-
-  private static InnerStruct getInner(StructColumnVector vector,
-                                      int rowId) {
-    return new InnerStruct(
-        (int) ((LongColumnVector) vector.fields[0]).vector[rowId],
-        getText((BytesColumnVector) vector.fields[1], rowId));
-  }
-
-  private static List<InnerStruct> getList(ListColumnVector cv,
-                                           int rowId) {
-    if (cv.isRepeating) {
-      rowId = 0;
-    }
-    if (cv.noNulls || !cv.isNull[rowId]) {
-      List<InnerStruct> result =
-          new ArrayList<InnerStruct>((int) cv.lengths[rowId]);
-      for(long i=cv.offsets[rowId];
-          i < cv.offsets[rowId] + cv.lengths[rowId]; ++i) {
-        result.add(getInner((StructColumnVector) cv.child, (int) i));
-      }
-      return result;
-    } else {
-      return null;
-    }
-  }
-
-  private static List<InnerStruct> getMidList(VectorizedRowBatch batch,
-                                              int rowId) {
-    return getList((ListColumnVector) ((StructColumnVector) batch.cols[9])
-        .fields[0], rowId);
-  }
-
-  private static List<InnerStruct> getList(VectorizedRowBatch batch,
-                                           int rowId) {
-    return getList((ListColumnVector) batch.cols[10], rowId);
-  }
-
-  private static Map<Text, InnerStruct> getMap(VectorizedRowBatch batch,
-                                               int rowId) {
-    MapColumnVector cv = (MapColumnVector) batch.cols[11];
-    if (cv.isRepeating) {
-      rowId = 0;
-    }
-    if (cv.noNulls || !cv.isNull[rowId]) {
-      Map<Text, InnerStruct> result =
-          new HashMap<Text, InnerStruct>((int) cv.lengths[rowId]);
-      for(long i=cv.offsets[rowId];
-          i < cv.offsets[rowId] + cv.lengths[rowId]; ++i) {
-        result.put(getText((BytesColumnVector) cv.keys, (int) i),
-            getInner((StructColumnVector) cv.values, (int) i));
-      }
-      return result;
-    } else {
-      return null;
-    }
-  }
-
   private static TypeDescription createInnerSchema() {
     return TypeDescription.createStruct()
         .addField("int1", TypeDescription.createInt())
@@ -1019,114 +981,178 @@ public class TestVectorOrcFile {
 
     assertEquals("count: 2 hasNull: false min: bye max: hi sum: 5", stats[9].toString());
 
-    // check the schema
-    TypeDescription readerSchema = reader.getSchema();
-    assertEquals(TypeDescription.Category.STRUCT, readerSchema.getCategory());
+    // check the inspectors
+    StructObjectInspector readerInspector =
+        (StructObjectInspector) reader.getObjectInspector();
+    assertEquals(ObjectInspector.Category.STRUCT,
+        readerInspector.getCategory());
     assertEquals("struct<boolean1:boolean,byte1:tinyint,short1:smallint,"
         + "int1:int,long1:bigint,float1:float,double1:double,bytes1:"
         + "binary,string1:string,middle:struct<list:array<struct<int1:int,"
         + "string1:string>>>,list:array<struct<int1:int,string1:string>>,"
         + "map:map<string,struct<int1:int,string1:string>>>",
-        readerSchema.toString());
-    List<String> fieldNames = readerSchema.getFieldNames();
-    List<TypeDescription> fieldTypes = readerSchema.getChildren();
-    assertEquals("boolean1", fieldNames.get(0));
-    assertEquals(TypeDescription.Category.BOOLEAN, fieldTypes.get(0).getCategory());
-    assertEquals("byte1", fieldNames.get(1));
-    assertEquals(TypeDescription.Category.BYTE, fieldTypes.get(1).getCategory());
-    assertEquals("short1", fieldNames.get(2));
-    assertEquals(TypeDescription.Category.SHORT, fieldTypes.get(2).getCategory());
-    assertEquals("int1", fieldNames.get(3));
-    assertEquals(TypeDescription.Category.INT, fieldTypes.get(3).getCategory());
-    assertEquals("long1", fieldNames.get(4));
-    assertEquals(TypeDescription.Category.LONG, fieldTypes.get(4).getCategory());
-    assertEquals("float1", fieldNames.get(5));
-    assertEquals(TypeDescription.Category.FLOAT, fieldTypes.get(5).getCategory());
-    assertEquals("double1", fieldNames.get(6));
-    assertEquals(TypeDescription.Category.DOUBLE, fieldTypes.get(6).getCategory());
-    assertEquals("bytes1", fieldNames.get(7));
-    assertEquals(TypeDescription.Category.BINARY, fieldTypes.get(7).getCategory());
-    assertEquals("string1", fieldNames.get(8));
-    assertEquals(TypeDescription.Category.STRING, fieldTypes.get(8).getCategory());
-    assertEquals("middle", fieldNames.get(9));
-    TypeDescription middle = fieldTypes.get(9);
-    assertEquals(TypeDescription.Category.STRUCT, middle.getCategory());
-    TypeDescription midList = middle.getChildren().get(0);
-    assertEquals(TypeDescription.Category.LIST, midList.getCategory());
-    TypeDescription inner = midList.getChildren().get(0);
-    assertEquals(TypeDescription.Category.STRUCT, inner.getCategory());
-    assertEquals("int1", inner.getFieldNames().get(0));
-    assertEquals("string1", inner.getFieldNames().get(1));
-
+        readerInspector.getTypeName());
+    List<? extends StructField> fields =
+        readerInspector.getAllStructFieldRefs();
+    BooleanObjectInspector bo = (BooleanObjectInspector) readerInspector.
+        getStructFieldRef("boolean1").getFieldObjectInspector();
+    ByteObjectInspector by = (ByteObjectInspector) readerInspector.
+        getStructFieldRef("byte1").getFieldObjectInspector();
+    ShortObjectInspector sh = (ShortObjectInspector) readerInspector.
+        getStructFieldRef("short1").getFieldObjectInspector();
+    IntObjectInspector in = (IntObjectInspector) readerInspector.
+        getStructFieldRef("int1").getFieldObjectInspector();
+    LongObjectInspector lo = (LongObjectInspector) readerInspector.
+        getStructFieldRef("long1").getFieldObjectInspector();
+    FloatObjectInspector fl = (FloatObjectInspector) readerInspector.
+        getStructFieldRef("float1").getFieldObjectInspector();
+    DoubleObjectInspector dbl = (DoubleObjectInspector) readerInspector.
+        getStructFieldRef("double1").getFieldObjectInspector();
+    BinaryObjectInspector bi = (BinaryObjectInspector) readerInspector.
+        getStructFieldRef("bytes1").getFieldObjectInspector();
+    StringObjectInspector st = (StringObjectInspector) readerInspector.
+        getStructFieldRef("string1").getFieldObjectInspector();
+    StructObjectInspector mid = (StructObjectInspector) readerInspector.
+        getStructFieldRef("middle").getFieldObjectInspector();
+    List<? extends StructField> midFields =
+        mid.getAllStructFieldRefs();
+    ListObjectInspector midli =
+        (ListObjectInspector) midFields.get(0).getFieldObjectInspector();
+    StructObjectInspector inner = (StructObjectInspector)
+        midli.getListElementObjectInspector();
+    List<? extends StructField> inFields = inner.getAllStructFieldRefs();
+    ListObjectInspector li = (ListObjectInspector) readerInspector.
+        getStructFieldRef("list").getFieldObjectInspector();
+    MapObjectInspector ma = (MapObjectInspector) readerInspector.
+        getStructFieldRef("map").getFieldObjectInspector();
+    StringObjectInspector mk = (StringObjectInspector)
+        ma.getMapKeyObjectInspector();
     RecordReader rows = reader.rows();
-    // create a new batch
-    batch = readerSchema.createRowBatch();
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(2, batch.size);
-    assertEquals(false, rows.hasNext());
-
+    Object row = rows.next(null);
+    assertNotNull(row);
     // check the contents of the first row
-    assertEquals(false, getBoolean(batch, 0));
-    assertEquals(1, getByte(batch, 0));
-    assertEquals(1024, getShort(batch, 0));
-    assertEquals(65536, getInt(batch, 0));
-    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
-    assertEquals(1.0, getFloat(batch, 0), 0.00001);
-    assertEquals(-15.0, getDouble(batch, 0), 0.00001);
-    assertEquals(bytes(0,1,2,3,4), getBinary(batch, 0));
-    assertEquals("hi", getText(batch, 0).toString());
-    List<InnerStruct> midRow = getMidList(batch, 0);
+    assertEquals(false,
+        bo.get(readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals(1, by.get(readerInspector.getStructFieldData(row,
+        fields.get(1))));
+    assertEquals(1024, sh.get(readerInspector.getStructFieldData(row,
+        fields.get(2))));
+    assertEquals(65536, in.get(readerInspector.getStructFieldData(row,
+        fields.get(3))));
+    assertEquals(Long.MAX_VALUE, lo.get(readerInspector.
+        getStructFieldData(row, fields.get(4))));
+    assertEquals(1.0, fl.get(readerInspector.getStructFieldData(row,
+        fields.get(5))), 0.00001);
+    assertEquals(-15.0, dbl.get(readerInspector.getStructFieldData(row,
+        fields.get(6))), 0.00001);
+    assertEquals(bytes(0,1,2,3,4), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(7))));
+    assertEquals("hi", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(8))));
+    List<?> midRow = midli.getList(mid.getStructFieldData(readerInspector.
+        getStructFieldData(row, fields.get(9)), midFields.get(0)));
     assertNotNull(midRow);
     assertEquals(2, midRow.size());
-    assertEquals(1, midRow.get(0).int1);
-    assertEquals("bye", midRow.get(0).string1.toString());
-    assertEquals(2, midRow.get(1).int1);
-    assertEquals("sigh", midRow.get(1).string1.toString());
-    List<InnerStruct> list = getList(batch, 0);
+    assertEquals(1, in.get(inner.getStructFieldData(midRow.get(0),
+        inFields.get(0))));
+    assertEquals("bye", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (midRow.get(0), inFields.get(1))));
+    assertEquals(2, in.get(inner.getStructFieldData(midRow.get(1),
+        inFields.get(0))));
+    assertEquals("sigh", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (midRow.get(1), inFields.get(1))));
+    List<?> list = li.getList(readerInspector.getStructFieldData(row,
+        fields.get(10)));
     assertEquals(2, list.size());
-    assertEquals(3, list.get(0).int1);
-    assertEquals("good", list.get(0).string1.toString());
-    assertEquals(4, list.get(1).int1);
-    assertEquals("bad", list.get(1).string1.toString());
-    Map<Text, InnerStruct> map = getMap(batch, 0);
+    assertEquals(3, in.get(inner.getStructFieldData(list.get(0),
+        inFields.get(0))));
+    assertEquals("good", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (list.get(0), inFields.get(1))));
+    assertEquals(4, in.get(inner.getStructFieldData(list.get(1),
+        inFields.get(0))));
+    assertEquals("bad", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (list.get(1), inFields.get(1))));
+    Map<?,?> map = ma.getMap(readerInspector.getStructFieldData(row,
+        fields.get(11)));
     assertEquals(0, map.size());
 
     // check the contents of second row
-    assertEquals(true, getBoolean(batch, 1));
-    assertEquals(100, getByte(batch, 1));
-    assertEquals(2048, getShort(batch, 1));
-    assertEquals(65536, getInt(batch, 1));
-    assertEquals(Long.MAX_VALUE, getLong(batch, 1));
-    assertEquals(2.0, getFloat(batch, 1), 0.00001);
-    assertEquals(-5.0, getDouble(batch, 1), 0.00001);
-    assertEquals(bytes(), getBinary(batch, 1));
-    assertEquals("bye", getText(batch, 1).toString());
-    midRow = getMidList(batch, 1);
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertEquals(true,
+        bo.get(readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals(100, by.get(readerInspector.getStructFieldData(row,
+        fields.get(1))));
+    assertEquals(2048, sh.get(readerInspector.getStructFieldData(row,
+        fields.get(2))));
+    assertEquals(65536, in.get(readerInspector.getStructFieldData(row,
+        fields.get(3))));
+    assertEquals(Long.MAX_VALUE, lo.get(readerInspector.
+        getStructFieldData(row, fields.get(4))));
+    assertEquals(2.0, fl.get(readerInspector.getStructFieldData(row,
+        fields.get(5))), 0.00001);
+    assertEquals(-5.0, dbl.get(readerInspector.getStructFieldData(row,
+        fields.get(6))), 0.00001);
+    assertEquals(bytes(), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(7))));
+    assertEquals("bye", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(8))));
+    midRow = midli.getList(mid.getStructFieldData(readerInspector.
+        getStructFieldData(row, fields.get(9)), midFields.get(0)));
     assertNotNull(midRow);
     assertEquals(2, midRow.size());
-    assertEquals(1, midRow.get(0).int1);
-    assertEquals("bye", midRow.get(0).string1.toString());
-    assertEquals(2, midRow.get(1).int1);
-    assertEquals("sigh", midRow.get(1).string1.toString());
-    list = getList(batch, 1);
+    assertEquals(1, in.get(inner.getStructFieldData(midRow.get(0),
+        inFields.get(0))));
+    assertEquals("bye", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (midRow.get(0), inFields.get(1))));
+    assertEquals(2, in.get(inner.getStructFieldData(midRow.get(1),
+        inFields.get(0))));
+    assertEquals("sigh", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (midRow.get(1), inFields.get(1))));
+    list = li.getList(readerInspector.getStructFieldData(row,
+        fields.get(10)));
     assertEquals(3, list.size());
-    assertEquals(100000000, list.get(0).int1);
-    assertEquals("cat", list.get(0).string1.toString());
-    assertEquals(-100000, list.get(1).int1);
-    assertEquals("in", list.get(1).string1.toString());
-    assertEquals(1234, list.get(2).int1);
-    assertEquals("hat", list.get(2).string1.toString());
-    map = getMap(batch, 1);
+    assertEquals(100000000, in.get(inner.getStructFieldData(list.get(0),
+        inFields.get(0))));
+    assertEquals("cat", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (list.get(0), inFields.get(1))));
+    assertEquals(-100000, in.get(inner.getStructFieldData(list.get(1),
+        inFields.get(0))));
+    assertEquals("in", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (list.get(1), inFields.get(1))));
+    assertEquals(1234, in.get(inner.getStructFieldData(list.get(2),
+        inFields.get(0))));
+    assertEquals("hat", st.getPrimitiveJavaObject(inner.getStructFieldData
+        (list.get(2), inFields.get(1))));
+    map = ma.getMap(readerInspector.getStructFieldData(row,
+        fields.get(11)));
     assertEquals(2, map.size());
-    InnerStruct value = map.get(new Text("chani"));
-    assertEquals(5, value.int1);
-    assertEquals("chani", value.string1.toString());
-    value = map.get(new Text("mauddib"));
-    assertEquals(1, value.int1);
-    assertEquals("mauddib", value.string1.toString());
+    boolean[] found = new boolean[2];
+    for(Object key: map.keySet()) {
+      String str = mk.getPrimitiveJavaObject(key);
+      if (str.equals("chani")) {
+        assertEquals(false, found[0]);
+        assertEquals(5, in.get(inner.getStructFieldData(map.get(key),
+            inFields.get(0))));
+        assertEquals(str, st.getPrimitiveJavaObject(
+            inner.getStructFieldData(map.get(key), inFields.get(1))));
+        found[0] = true;
+      } else if (str.equals("mauddib")) {
+        assertEquals(false, found[1]);
+        assertEquals(1, in.get(inner.getStructFieldData(map.get(key),
+            inFields.get(0))));
+        assertEquals(str, st.getPrimitiveJavaObject(
+            inner.getStructFieldData(map.get(key), inFields.get(1))));
+        found[1] = true;
+      } else {
+        throw new IllegalArgumentException("Unknown key " + str);
+      }
+    }
+    assertEquals(true, found[0]);
+    assertEquals(true, found[1]);
 
     // handle the close up
-    assertEquals(false, rows.nextBatch(batch));
+    assertEquals(false, rows.hasNext());
     rows.close();
   }
 
@@ -1190,36 +1216,35 @@ public class TestVectorOrcFile {
     }
 
     // check out the types
-    TypeDescription type = reader.getSchema();
-    assertEquals(TypeDescription.Category.STRUCT, type.getCategory());
-    assertEquals(2, type.getChildren().size());
-    TypeDescription type1 = type.getChildren().get(0);
-    TypeDescription type2 = type.getChildren().get(1);
-    assertEquals(TypeDescription.Category.INT, type1.getCategory());
-    assertEquals(TypeDescription.Category.STRING, type2.getCategory());
-    assertEquals("struct<int1:int,string1:string>", type.toString());
+    List<OrcProto.Type> types = reader.getTypes();
+    assertEquals(3, types.size());
+    assertEquals(OrcProto.Type.Kind.STRUCT, types.get(0).getKind());
+    assertEquals(2, types.get(0).getSubtypesCount());
+    assertEquals(1, types.get(0).getSubtypes(0));
+    assertEquals(2, types.get(0).getSubtypes(1));
+    assertEquals(OrcProto.Type.Kind.INT, types.get(1).getKind());
+    assertEquals(0, types.get(1).getSubtypesCount());
+    assertEquals(OrcProto.Type.Kind.STRING, types.get(2).getKind());
+    assertEquals(0, types.get(2).getSubtypesCount());
 
     // read the contents and make sure they match
     RecordReader rows1 = reader.rows(new boolean[]{true, true, false});
     RecordReader rows2 = reader.rows(new boolean[]{true, false, true});
     r1 = new Random(1);
     r2 = new Random(2);
-    VectorizedRowBatch batch1 = reader.getSchema().createRowBatch(1000);
-    VectorizedRowBatch batch2 = reader.getSchema().createRowBatch(1000);
-    for(int i = 0; i < 21000; i += 1000) {
-      assertEquals(true, rows1.nextBatch(batch1));
-      assertEquals(true, rows2.nextBatch(batch2));
-      assertEquals(1000, batch1.size);
-      assertEquals(1000, batch2.size);
-      for(int j=0; j < 1000; ++j) {
-        assertEquals(r1.nextInt(),
-            ((LongColumnVector) batch1.cols[0]).vector[j]);
-        assertEquals(Long.toHexString(r2.nextLong()),
-            ((BytesColumnVector) batch2.cols[1]).toString(j));
-      }
-    }
-    assertEquals(false, rows1.nextBatch(batch1));
-    assertEquals(false, rows2.nextBatch(batch2));
+    OrcStruct row1 = null;
+    OrcStruct row2 = null;
+    for(int i = 0; i < 21000; ++i) {
+      assertEquals(true, rows1.hasNext());
+      assertEquals(true, rows2.hasNext());
+      row1 = (OrcStruct) rows1.next(row1);
+      row2 = (OrcStruct) rows2.next(row2);
+      assertEquals(r1.nextInt(), ((IntWritable) row1.getFieldValue(0)).get());
+      assertEquals(Long.toHexString(r2.nextLong()),
+          row2.getFieldValue(1).toString());
+    }
+    assertEquals(false, rows1.hasNext());
+    assertEquals(false, rows2.hasNext());
     rows1.close();
     rows2.close();
   }
@@ -1330,33 +1355,17 @@ public class TestVectorOrcFile {
     Reader reader = OrcFile.createReader(file,
         OrcFile.readerOptions(conf));
     RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch(1000);
-    TimestampColumnVector times = (TimestampColumnVector) batch.cols[0];
-    LongColumnVector dates = (LongColumnVector) batch.cols[1];
+    OrcStruct row = null;
     for (int year = minYear; year < maxYear; ++year) {
-      rows.nextBatch(batch);
-      assertEquals(1000, batch.size);
       for(int ms = 1000; ms < 2000; ++ms) {
-        StringBuilder buffer = new StringBuilder();
-        times.stringifyValue(buffer, ms - 1000);
-        String expected = Integer.toString(year) + "-05-05 12:34:56.";
-        // suppress the final zeros on the string by dividing by the largest
-        // power of 10 that divides evenly.
-        int roundedMs = ms;
-        for(int round = 1000; round > 0; round /= 10) {
-          if (ms % round == 0) {
-            roundedMs = ms / round;
-            break;
-          }
-        }
-        expected += roundedMs;
-        assertEquals(expected, buffer.toString());
-        assertEquals(Integer.toString(year) + "-12-25",
-            new DateWritable((int) dates.vector[ms - 1000]).toString());
+        row = (OrcStruct) rows.next(row);
+        assertEquals(new TimestampWritable
+                (Timestamp.valueOf(year + "-05-05 12:34:56." + ms)),
+            row.getFieldValue(0));
+        assertEquals(new DateWritable(new Date(year - 1900, 11, 25)),
+            row.getFieldValue(1));
       }
     }
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
   }
 
   @Test
@@ -1474,7 +1483,6 @@ public class TestVectorOrcFile {
     for(int c=0; c < batch.cols.length; ++c) {
       batch.cols[c].setRepeating(true);
     }
-    ((UnionColumnVector) batch.cols[1]).fields[0].isRepeating = true;
     setUnion(batch, 0, null, 0, 1732050807, null, null);
     for(int i=0; i < 5; ++i) {
       writer.addRowBatch(batch);
@@ -1532,115 +1540,83 @@ public class TestVectorOrcFile {
     RecordReader rows = reader.rows();
     assertEquals(0, rows.getRowNumber());
     assertEquals(0.0, rows.getProgress(), 0.000001);
-
-    schema = reader.getSchema();
-    batch = schema.createRowBatch(74);
-    assertEquals(0, rows.getRowNumber());
-    rows.nextBatch(batch);
-    assertEquals(74, batch.size);
-    assertEquals(74, rows.getRowNumber());
-    TimestampColumnVector ts = (TimestampColumnVector) batch.cols[0];
-    UnionColumnVector union = (UnionColumnVector) batch.cols[1];
-    LongColumnVector longs = (LongColumnVector) union.fields[0];
-    BytesColumnVector strs = (BytesColumnVector) union.fields[1];
-    DecimalColumnVector decs = (DecimalColumnVector) batch.cols[2];
-
+    assertEquals(true, rows.hasNext());
+    OrcStruct row = (OrcStruct) rows.next(null);
+    assertEquals(1, rows.getRowNumber());
+    ObjectInspector inspector = reader.getObjectInspector();
     assertEquals("struct<time:timestamp,union:uniontype<int,string>,decimal:decimal(38,18)>",
-        schema.toString());
-    assertEquals("2000-03-12 15:00:00.0", ts.asScratchTimestamp(0).toString());
-    assertEquals(0, union.tags[0]);
-    assertEquals(42, longs.vector[0]);
-    assertEquals("12345678.6547456", decs.vector[0].toString());
-
-    assertEquals("2000-03-20 12:00:00.123456789", ts.asScratchTimestamp(1).toString());
-    assertEquals(1, union.tags[1]);
-    assertEquals("hello", strs.toString(1));
-    assertEquals("-5643.234", decs.vector[1].toString());
-
-    assertEquals(false, ts.noNulls);
-    assertEquals(false, union.noNulls);
-    assertEquals(false, decs.noNulls);
-    assertEquals(true, ts.isNull[2]);
-    assertEquals(true, union.isNull[2]);
-    assertEquals(true, decs.isNull[2]);
-
-    assertEquals(true, ts.isNull[3]);
-    assertEquals(false, union.isNull[3]);
-    assertEquals(0, union.tags[3]);
-    assertEquals(true, longs.isNull[3]);
-    assertEquals(true, decs.isNull[3]);
-
-    assertEquals(true, ts.isNull[4]);
-    assertEquals(false, union.isNull[4]);
-    assertEquals(1, union.tags[4]);
-    assertEquals(true, strs.isNull[4]);
-    assertEquals(true, decs.isNull[4]);
-
-    assertEquals(false, ts.isNull[5]);
-    assertEquals("1970-01-01 00:00:00.0", ts.asScratchTimestamp(5).toString());
-    assertEquals(false, union.isNull[5]);
-    assertEquals(0, union.tags[5]);
-    assertEquals(false, longs.isNull[5]);
-    assertEquals(200000, longs.vector[5]);
-    assertEquals(false, decs.isNull[5]);
-    assertEquals("10000000000000000000", decs.vector[5].toString());
-
+        inspector.getTypeName());
+    assertEquals(new TimestampWritable(Timestamp.valueOf("2000-03-12 15:00:00")),
+        row.getFieldValue(0));
+    OrcUnion union = (OrcUnion) row.getFieldValue(1);
+    assertEquals(0, union.getTag());
+    assertEquals(new IntWritable(42), union.getObject());
+    assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547456")),
+        row.getFieldValue(2));
+    row = (OrcStruct) rows.next(row);
+    assertEquals(2, rows.getRowNumber());
+    assertEquals(new TimestampWritable(Timestamp.valueOf("2000-03-20 12:00:00.123456789")),
+        row.getFieldValue(0));
+    assertEquals(1, union.getTag());
+    assertEquals(new Text("hello"), union.getObject());
+    assertEquals(new HiveDecimalWritable(HiveDecimal.create("-5643.234")),
+        row.getFieldValue(2));
+    row = (OrcStruct) rows.next(row);
+    assertEquals(null, row.getFieldValue(0));
+    assertEquals(null, row.getFieldValue(1));
+    assertEquals(null, row.getFieldValue(2));
+    row = (OrcStruct) rows.next(row);
+    assertEquals(null, row.getFieldValue(0));
+    union = (OrcUnion) row.getFieldValue(1);
+    assertEquals(0, union.getTag());
+    assertEquals(null, union.getObject());
+    assertEquals(null, row.getFieldValue(2));
+    row = (OrcStruct) rows.next(row);
+    assertEquals(null, row.getFieldValue(0));
+    assertEquals(1, union.getTag());
+    assertEquals(null, union.getObject());
+    assertEquals(null, row.getFieldValue(2));
+    row = (OrcStruct) rows.next(row);
+    assertEquals(new TimestampWritable(Timestamp.valueOf("1970-01-01 00:00:00")),
+        row.getFieldValue(0));
+    assertEquals(new IntWritable(200000), union.getObject());
+    assertEquals(new HiveDecimalWritable(HiveDecimal.create("10000000000000000000")),
+                 row.getFieldValue(2));
     rand = new Random(42);
     for(int i=1970; i < 2038; ++i) {
-      int row = 6 + i - 1970;
-      assertEquals(Timestamp.valueOf(i + "-05-05 12:34:56." + i),
-          ts.asScratchTimestamp(row));
+      row = (OrcStruct) rows.next(row);
+      assertEquals(new TimestampWritable(Timestamp.valueOf(i + "-05-05 12:34:56." + i)),
+          row.getFieldValue(0));
       if ((i & 1) == 0) {
-        assertEquals(0, union.tags[row]);
-        assertEquals(i*i, longs.vector[row]);
+        assertEquals(0, union.getTag());
+        assertEquals(new IntWritable(i*i), union.getObject());
       } else {
-        assertEquals(1, union.tags[row]);
-        assertEquals(Integer.toString(i * i), strs.toString(row));
+        assertEquals(1, union.getTag());
+        assertEquals(new Text(Integer.toString(i * i)), union.getObject());
       }
       assertEquals(new HiveDecimalWritable(HiveDecimal.create(new BigInteger(64, rand),
-                                   rand.nextInt(18))), decs.vector[row]);
-    }
-
-    // rebuild the row batch, so that we can read by 1000 rows
-    batch = schema.createRowBatch(1000);
-    ts = (TimestampColumnVector) batch.cols[0];
-    union = (UnionColumnVector) batch.cols[1];
-    longs = (LongColumnVector) union.fields[0];
-    strs = (BytesColumnVector) union.fields[1];
-    decs = (DecimalColumnVector) batch.cols[2];
-
-    for(int i=0; i < 5; ++i) {
-      rows.nextBatch(batch);
-      assertEquals("batch " + i, 1000, batch.size);
-      assertEquals("batch " + i, false, union.isRepeating);
-      assertEquals("batch " + i, true, union.noNulls);
-      for(int r=0; r < batch.size; ++r) {
-        assertEquals("bad tag at " + i + "." +r, 0, union.tags[r]);
-      }
-      assertEquals("batch " + i, true, longs.isRepeating);
-      assertEquals("batch " + i, 1732050807, longs.vector[0]);
-    }
-
-    rows.nextBatch(batch);
-    assertEquals(3, batch.size);
-    assertEquals(0, union.tags[0]);
-    assertEquals(0, longs.vector[0]);
-    assertEquals(0, union.tags[1]);
-    assertEquals(10, longs.vector[1]);
-    assertEquals(0, union.tags[2]);
-    assertEquals(138, longs.vector[2]);
-
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
+                                   rand.nextInt(18))), row.getFieldValue(2));
+    }
+    for(int i=0; i < 5000; ++i) {
+      row = (OrcStruct) rows.next(row);
+      assertEquals(new IntWritable(1732050807), union.getObject());
+    }
+    row = (OrcStruct) rows.next(row);
+    assertEquals(new IntWritable(0), union.getObject());
+    row = (OrcStruct) rows.next(row);
+    assertEquals(new IntWritable(10), union.getObject());
+    row = (OrcStruct) rows.next(row);
+    assertEquals(new IntWritable(138), union.getObject());
+    assertEquals(false, rows.hasNext());
     assertEquals(1.0, rows.getProgress(), 0.00001);
     assertEquals(reader.getNumberOfRows(), rows.getRowNumber());
     rows.seekToRow(1);
-    rows.nextBatch(batch);
-    assertEquals(1000, batch.size);
-    assertEquals(Timestamp.valueOf("2000-03-20 12:00:00.123456789"), ts.asScratchTimestamp(0));
-    assertEquals(1, union.tags[0]);
-    assertEquals("hello", strs.toString(0));
-    assertEquals(new HiveDecimalWritable(HiveDecimal.create("-5643.234")), decs.vector[0]);
+    row = (OrcStruct) rows.next(row);
+    assertEquals(new TimestampWritable(Timestamp.valueOf("2000-03-20 12:00:00.123456789")),
+        row.getFieldValue(0));
+    assertEquals(1, union.getTag());
+    assertEquals(new Text("hello"), union.getObject());
+    assertEquals(new HiveDecimalWritable(HiveDecimal.create("-5643.234")), row.getFieldValue(2));
     rows.close();
   }
 
@@ -1671,22 +1647,17 @@ public class TestVectorOrcFile {
     writer.close();
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(CompressionKind.SNAPPY, reader.getCompressionKind());
     RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch(1000);
     rand = new Random(12);
-    LongColumnVector longs = (LongColumnVector) batch.cols[0];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
-    for(int b=0; b < 10; ++b) {
-      rows.nextBatch(batch);
-      assertEquals(1000, batch.size);
-      for(int r=0; r < batch.size; ++r) {
-        assertEquals(rand.nextInt(), longs.vector[r]);
-        assertEquals(Integer.toHexString(rand.nextInt()), strs.toString(r));
-      }
+    OrcStruct row = null;
+    for(int i=0; i < 10000; ++i) {
+      assertEquals(true, rows.hasNext());
+      row = (OrcStruct) rows.next(row);
+      assertEquals(rand.nextInt(), ((IntWritable) row.getFieldValue(0)).get());
+      assertEquals(Integer.toHexString(rand.nextInt()),
+          row.getFieldValue(1).toString());
     }
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
+    assertEquals(false, rows.hasNext());
     rows.close();
   }
 
@@ -1726,23 +1697,18 @@ public class TestVectorOrcFile {
     assertEquals(0, stripe.getIndexLength());
     RecordReader rows = reader.rows();
     rand = new Random(24);
-    batch = reader.getSchema().createRowBatch(1000);
-    LongColumnVector longs = (LongColumnVector) batch.cols[0];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
-    for(int i=0; i < 50; ++i) {
-      rows.nextBatch(batch);
-      assertEquals("batch " + i, 1000, batch.size);
-      for(int j=0; j < 200; ++j) {
-        int intVal = rand.nextInt();
-        String strVal = Integer.toBinaryString(rand.nextInt());
-        for (int k = 0; k < 5; ++k) {
-          assertEquals(intVal, longs.vector[j * 5 + k]);
-          assertEquals(strVal, strs.toString(j * 5 + k));
-        }
+    OrcStruct row = null;
+    for(int i=0; i < 10000; ++i) {
+      int intVal = rand.nextInt();
+      String strVal = Integer.toBinaryString(rand.nextInt());
+      for(int j=0; j < 5; ++j) {
+        assertEquals(true, rows.hasNext());
+        row = (OrcStruct) rows.next(row);
+        assertEquals(intVal, ((IntWritable) row.getFieldValue(0)).get());
+        assertEquals(strVal, row.getFieldValue(1).toString());
       }
     }
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
+    assertEquals(false, rows.hasNext());
     rows.close();
   }
 
@@ -1806,18 +1772,34 @@ public class TestVectorOrcFile {
       assertEquals(1000,
           colIndex.getEntry(0).getStatistics().getNumberOfValues());
     }
-    batch = reader.getSchema().createRowBatch();
-    int nextRowInBatch = -1;
-    for(int i=COUNT-1; i >= 0; --i, --nextRowInBatch) {
-      // if we have consumed the previous batch read a new one
-      if (nextRowInBatch < 0) {
-        long base = Math.max(i - 1023, 0);
-        rows.seekToRow(base);
-        assertEquals("row " + i, true, rows.nextBatch(batch));
-        nextRowInBatch = batch.size - 1;
-      }
-      checkRandomRow(batch, intValues, doubleValues,
-          stringValues, byteValues, words, i, nextRowInBatch);
+    OrcStruct row = null;
+    for(int i=COUNT-1; i >= 0; --i) {
+      rows.seekToRow(i);
+      row = (OrcStruct) rows.next(row);
+      BigRow expected = createRandomRow(intValues, doubleValues,
+          stringValues, byteValues, words, i);
+      assertEquals(expected.boolean1.booleanValue(),
+          ((BooleanWritable) row.getFieldValue(0)).get());
+      assertEquals(expected.byte1.byteValue(),
+          ((ByteWritable) row.getFieldValue(1)).get());
+      assertEquals(expected.short1.shortValue(),
+          ((ShortWritable) row.getFieldValue(2)).get());
+      assertEquals(expected.int1.intValue(),
+          ((IntWritable) row.getFieldValue(3)).get());
+      assertEquals(expected.long1.longValue(),
+          ((LongWritable) row.getFieldValue(4)).get());
+      assertEquals(expected.float1,
+          ((FloatWritable) row.getFieldValue(5)).get(), 0.0001);
+      assertEquals(expected.double1,
+          ((DoubleWritable) row.getFieldValue(6)).get(), 0.0001);
+      assertEquals(expected.bytes1, row.getFieldValue(7));
+      assertEquals(expected.string1, row.getFieldValue(8));
+      List<InnerStruct> expectedList = expected.middle.list;
+      List<OrcStruct> actualList =
+          (List<OrcStruct>) ((OrcStruct) row.getFieldValue(9)).getFieldValue(0);
+      compareList(expectedList, actualList, "middle list " + i);
+      compareList(expected.list, (List<OrcStruct>) row.getFieldValue(10),
+          "list " + i);
     }
     rows.close();
     Iterator<StripeInformation> stripeIterator =
@@ -1843,20 +1825,41 @@ public class TestVectorOrcFile {
         .range(offsetOfStripe2, offsetOfStripe4 - offsetOfStripe2)
         .include(columns));
     rows.seekToRow(lastRowOfStripe2);
-    // we only want two rows
-    batch = reader.getSchema().createRowBatch(2);
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1, batch.size);
-    assertEquals(intValues[(int) lastRowOfStripe2], getLong(batch, 0));
-    assertEquals(stringValues[(int) lastRowOfStripe2],
-        getText(batch, 0).toString());
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(intValues[(int) lastRowOfStripe2 + 1], getLong(batch, 0));
-    assertEquals(stringValues[(int) lastRowOfStripe2 + 1],
-        getText(batch, 0).toString());
+    for(int i = 0; i < 2; ++i) {
+      row = (OrcStruct) rows.next(row);
+      BigRow expected = createRandomRow(intValues, doubleValues,
+                                        stringValues, byteValues, words,
+                                        (int) (lastRowOfStripe2 + i));
+
+      assertEquals(expected.long1.longValue(),
+          ((LongWritable) row.getFieldValue(4)).get());
+      assertEquals(expected.string1, row.getFieldValue(8));
+    }
     rows.close();
   }
 
+  private void compareInner(InnerStruct expect,
+                            OrcStruct actual,
+                            String context) throws Exception {
+    if (expect == null || actual == null) {
+      assertEquals(context, null, expect);
+      assertEquals(context, null, actual);
+    } else {
+      assertEquals(context, expect.int1,
+          ((IntWritable) actual.getFieldValue(0)).get());
+      assertEquals(context, expect.string1, actual.getFieldValue(1));
+    }
+  }
+
+  private void compareList(List<InnerStruct> expect,
+                           List<OrcStruct> actual,
+                           String context) throws Exception {
+    assertEquals(context, expect.size(), actual.size());
+    for(int j=0; j < expect.size(); ++j) {
+      compareInner(expect.get(j), actual.get(j), context + " at " + j);
+    }
+  }
+
   private void appendRandomRow(VectorizedRowBatch batch,
                                long[] intValues, double[] doubleValues,
                                String[] stringValues,
@@ -1871,18 +1874,17 @@ public class TestVectorOrcFile {
         new MiddleStruct(inner, inner2), list(), map(inner, inner2));
   }
 
-  private void checkRandomRow(VectorizedRowBatch batch,
-                              long[] intValues, double[] doubleValues,
-                              String[] stringValues,
-                              BytesWritable[] byteValues,
-                              String[] words, int i, int rowInBatch) {
+  private BigRow createRandomRow(long[] intValues, double[] doubleValues,
+                                 String[] stringValues,
+                                 BytesWritable[] byteValues,
+                                 String[] words, int i) {
     InnerStruct inner = new InnerStruct((int) intValues[i], stringValues[i]);
     InnerStruct inner2 = new InnerStruct((int) (intValues[i] >> 32),
         words[i % words.length] + "-x");
-    checkBigRow(batch, rowInBatch, i, (intValues[i] & 1) == 0, (byte) intValues[i],
+    return new BigRow((intValues[i] & 1) == 0, (byte) intValues[i],
         (short) intValues[i], (int) intValues[i], intValues[i],
-        (float) doubleValues[i], doubleValues[i], byteValues[i], stringValues[i],
-        new MiddleStruct(inner, inner2), list(), map(inner, inner2));
+        (float) doubleValues[i], doubleValues[i], byteValues[i],stringValues[i],
+        new MiddleStruct(inner, inner2), list(), map(inner,inner2));
   }
 
   private static class MyMemoryManager extends MemoryManager {
@@ -2043,19 +2045,15 @@ public class TestVectorOrcFile {
         .range(0L, Long.MAX_VALUE)
         .include(new boolean[]{true, true, true})
         .searchArgument(sarg, new String[]{null, "int1", "string1"}));
-    batch = reader.getSchema().createRowBatch(2000);
-    LongColumnVector ints = (LongColumnVector) batch.cols[0];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
-
     assertEquals(1000L, rows.getRowNumber());
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1000, batch.size);
-
+    OrcStruct row = null;
     for(int i=1000; i < 2000; ++i) {
-      assertEquals(300 * i, ints.vector[i - 1000]);
-      assertEquals(Integer.toHexString(10*i), strs.toString(i - 1000));
+      assertTrue(rows.hasNext());
+      row = (OrcStruct) rows.next(row);
+      assertEquals(300 * i, ((IntWritable) row.getFieldValue(0)).get());
+      assertEquals(Integer.toHexString(10*i), row.getFieldValue(1).toString());
     }
-    assertEquals(false, rows.nextBatch(batch));
+    assertTrue(!rows.hasNext());
     assertEquals(3500, rows.getRowNumber());
 
     // look through the file with no rows selected
@@ -2084,26 +2082,40 @@ public class TestVectorOrcFile {
         .range(0L, Long.MAX_VALUE)
         .include(new boolean[]{true, true, true})
         .searchArgument(sarg, new String[]{null, "int1", "string1"}));
-    assertEquals(0, rows.getRowNumber());
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1000, batch.size);
-    assertEquals(3000, rows.getRowNumber());
+    row = null;
     for(int i=0; i < 1000; ++i) {
-      assertEquals(300 * i, ints.vector[i]);
-      assertEquals(Integer.toHexString(10*i), strs.toString(i));
+      assertTrue(rows.hasNext());
+      assertEquals(i, rows.getRowNumber());
+      row = (OrcStruct) rows.next(row);
+      assertEquals(300 * i, ((IntWritable) row.getFieldValue(0)).get());
+      assertEquals(Integer.toHexString(10*i), row.getFieldValue(1).toString());
     }
-
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(500, batch.size);
-    assertEquals(3500, rows.getRowNumber());
     for(int i=3000; i < 3500; ++i) {
-      assertEquals(300 * i, ints.vector[i - 3000]);
-      assertEquals(Integer.toHexString(10*i), strs.toString(i - 3000));
+      assertTrue(rows.hasNext());
+      assertEquals(i, rows.getRowNumber());
+      row = (OrcStruct) rows.next(row);
+      assertEquals(300 * i, ((IntWritable) row.getFieldValue(0)).get());
+      assertEquals(Integer.toHexString(10*i), row.getFieldValue(1).toString());
     }
-    assertEquals(false, rows.nextBatch(batch));
+    assertTrue(!rows.hasNext());
     assertEquals(3500, rows.getRowNumber());
   }
 
+  private static String pad(String value, int length) {
+    if (value.length() == length) {
+      return value;
+    } else if (value.length() > length) {
+      return value.substring(0, length);
+    } else {
+      StringBuilder buf = new StringBuilder();
+      buf.append(value);
+      for(int i=0; i < length - value.length(); ++i) {
+        buf.append(' ');
+      }
+      return buf.toString();
+    }
+  }
+
   /**
    * Test all of the types that have distinct ORC writers using the vectorized
    * writer with different combinations of repeating and null values.
@@ -2220,7 +2232,8 @@ public class TestVectorOrcFile {
       ((LongColumnVector) batch.cols[6]).vector[r] =
           new DateWritable(new Date(111, 6, 1)).getDays() + r;
 
-      Timestamp ts = new Timestamp(115, 9, 25, 10, 11, 59 + r, 999999999);
+      Timestamp ts = new Timestamp(115, 9, 23, 10, 11, 59, 999999999);
+      ts.setTime(ts.getTime() + r * 1000);
       ((TimestampColumnVector) batch.cols[7]).set(r, ts);
       ((DecimalColumnVector) batch.cols[8]).vector[r] =
           new HiveDecimalWritable("1.234567");
@@ -2289,125 +2302,118 @@ public class TestVectorOrcFile {
     assertEquals(14813, ((StringColumnStatistics) stats[12]).getSum());
 
     RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch(1024);
-    BytesColumnVector bins = (BytesColumnVector) batch.cols[0];
-    LongColumnVector bools = (LongColumnVector) batch.cols[1];
-    LongColumnVector bytes = (LongColumnVector) batch.cols[2];
-    LongColumnVector longs = (LongColumnVector) batch.cols[3];
-    DoubleColumnVector floats = (DoubleColumnVector) batch.cols[4];
-    DoubleColumnVector doubles = (DoubleColumnVector) batch.cols[5];
-    LongColumnVector dates = (LongColumnVector) batch.cols[6];
-    TimestampColumnVector times = (TimestampColumnVector) batch.cols[7];
-    DecimalColumnVector decs = (DecimalColumnVector) batch.cols[8];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[9];
-    BytesColumnVector chars = (BytesColumnVector) batch.cols[10];
-    BytesColumnVector vcs = (BytesColumnVector) batch.cols[11];
-    StructColumnVector structs = (StructColumnVector) batch.cols[12];
-    UnionColumnVector unions = (UnionColumnVector) batch.cols[13];
-    ListColumnVector lists = (ListColumnVector) batch.cols[14];
-    MapColumnVector maps = (MapColumnVector) batch.cols[15];
-    LongColumnVector structInts = (LongColumnVector) structs.fields[0];
-    LongColumnVector unionInts = (LongColumnVector) unions.fields[1];
-    LongColumnVector listInts = (LongColumnVector) lists.child;
-    BytesColumnVector mapKeys = (BytesColumnVector) maps.keys;
-    BytesColumnVector mapValues = (BytesColumnVector) maps.values;
-
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1024, batch.size);
+    OrcStruct row = null;
 
     // read the 1024 nulls
-    for(int f=0; f < batch.cols.length; ++f) {
-      assertEquals("field " + f,
-          true, batch.cols[f].isRepeating);
-      assertEquals("field " + f,
-          false, batch.cols[f].noNulls);
-      assertEquals("field " + f,
-          true, batch.cols[f].isNull[0]);
+    for(int r=0; r < 1024; ++r) {
+      assertEquals(true, rows.hasNext());
+      row = (OrcStruct) rows.next(row);
+      for(int f=0; f < row.getNumFields(); ++f) {
+        assertEquals("non-null on row " + r + " field " + f,
+            null, row.getFieldValue(f));
+      }
     }
 
     // read the 1024 repeat values
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1024, batch.size);
     for(int r=0; r < 1024; ++r) {
-      assertEquals("row " + r, "Horton", bins.toString(r));
-      assertEquals("row " + r, 1, bools.vector[r]);
-      assertEquals("row " + r, -126, bytes.vector[r]);
-      assertEquals("row " + r, 1311768467463790320L, longs.vector[r]);
-      assertEquals("row " + r, 1.125, floats.vector[r], 0.00001);
-      assertEquals("row " + r, 9.765625E-4, doubles.vector[r], 0.000001);
-      assertEquals("row " + r, "2011-07-01",
-          new DateWritable((int) dates.vector[r]).toString());
+      assertEquals(true, rows.hasNext());
+      row = (OrcStruct) rows.next(row);
+      assertEquals("row " + r, "48 6f 72 74 6f 6e",
+          row.getFieldValue(0).toString());
+      assertEquals("row " + r, "true", row.getFieldValue(1).toString());
+      assertEquals("row " + r, "-126", row.getFieldValue(2).toString());
+      assertEquals("row " + r, "1311768467463790320",
+          row.getFieldValue(3).toString());
+      assertEquals("row " + r, "1.125", row.getFieldValue(4).toString());
+      assertEquals("row " + r, "9.765625E-4", row.getFieldValue(5).toString());
+      assertEquals("row " + r, "2011-07-01", row.getFieldValue(6).toString());
       assertEquals("row " + r, "2015-10-23 10:11:59.999999999",
-          times.asScratchTimestamp(r).toString());
-      assertEquals("row " + r, "1.234567", decs.vector[r].toString());
-      assertEquals("row " + r, "Echelon", strs.toString(r));
-      assertEquals("row " + r, "Juggernaut", chars.toString(r));
-      assertEquals("row " + r, "Dreadnaugh", vcs.toString(r));
-      assertEquals("row " + r, 123, structInts.vector[r]);
-      assertEquals("row " + r, 1, unions.tags[r]);
-      assertEquals("row " + r, 1234, unionInts.vector[r]);
-      assertEquals("row " + r, 3, lists.lengths[r]);
-      assertEquals("row " + r, true, listInts.isRepeating);
-      assertEquals("row " + r, 31415, listInts.vector[0]);
-      assertEquals("row " + r, 3, maps.lengths[r]);
-      assertEquals("row " + r, "ORC", mapKeys.toString((int) maps.offsets[r]));
-      assertEquals("row " + r, "Hive", mapKeys.toString((int) maps.offsets[r] + 1));
-      assertEquals("row " + r, "LLAP", mapKeys.toString((int) maps.offsets[r] + 2));
-      assertEquals("row " + r, "fast", mapValues.toString((int) maps.offsets[r]));
-      assertEquals("row " + r, "fast", mapValues.toString((int) maps.offsets[r] + 1));
-      assertEquals("row " + r, "fast", mapValues.toString((int) maps.offsets[r] + 2));
+          row.getFieldValue(7).toString());
+      assertEquals("row " + r, "1.234567", row.getFieldValue(8).toString());
+      assertEquals("row " + r, "Echelon", row.getFieldValue(9).toString());
+      assertEquals("row " + r, "Juggernaut", row.getFieldValue(10).toString());
+      assertEquals("row " + r, "Dreadnaugh", row.getFieldValue(11).toString());
+      assertEquals("row " + r, "{123}", row.getFieldValue(12).toString());
+      assertEquals("row " + r, "union(1, 1234)",
+          row.getFieldValue(13).toString());
+      assertEquals("row " + r, "[31415, 31415, 31415]",
+          row.getFieldValue(14).toString());
+      assertEquals("row " + r, "{ORC=fast, Hive=fast, LLAP=fast}",
+          row.getFieldValue(15).toString());
     }
 
     // read the second set of 1024 nulls
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1024, batch.size);
-    for(int f=0; f < batch.cols.length; ++f) {
-      assertEquals("field " + f,
-          true, batch.cols[f].isRepeating);
-      assertEquals("field " + f,
-          false, batch.cols[f].noNulls);
-      assertEquals("field " + f,
-          true, batch.cols[f].isNull[0]);
+    for(int r=0; r < 1024; ++r) {
+      assertEquals(true, rows.hasNext());
+      row = (OrcStruct) rows.next(row);
+      for(int f=0; f < row.getNumFields(); ++f) {
+        assertEquals("non-null on row " + r + " field " + f,
+            null, row.getFieldValue(f));
+      }
     }
-
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1024, batch.size);
     for(int r=0; r < 1024; ++r) {
-      String hex = Integer.toHexString(r);
-
-      assertEquals("row " + r, hex, bins.toString(r));
-      assertEquals("row " + r, r % 2 == 1 ? 1 : 0, bools.vector[r]);
-      assertEquals("row " + r, (byte) (r % 255), bytes.vector[r]);
-      assertEquals("row " + r, 31415L * r, longs.vector[r]);
-      assertEquals("row " + r, 1.125F * r, floats.vector[r], 0.0001);
-      assertEquals("row " + r, 0.0009765625 * r, doubles.vector[r], 0.000001);
-      assertEquals("row " + r, new DateWritable(new Date(111, 6, 1 + r)),
-          new DateWritable((int) dates.vector[r]));
+      assertEquals(true, rows.hasNext());
+      row = (OrcStruct) rows.next(row);
+      byte[] hex = Integer.toHexString(r).getBytes();
+      StringBuilder expected = new StringBuilder();
+      for(int i=0; i < hex.length; ++i) {
+        if (i != 0) {
+          expected.append(' ');
+        }
+        expected.append(Integer.toHexString(hex[i]));
+      }
+      assertEquals("row " + r, expected.toString(),
+          row.getFieldValue(0).toString());
+      assertEquals("row " + r, r % 2 == 1 ? "true" : "false",
+          row.getFieldValue(1).toString());
+      assertEquals("row " + r, Integer.toString((byte) (r % 255)),
+          row.getFieldValue(2).toString());
+      assertEquals("row " + r, Long.toString(31415L * r),
+          row.getFieldValue(3).toString());
+      assertEquals("row " + r, Float.toString(1.125F * r),
+          row.getFieldValue(4).toString());
+      assertEquals("row " + r, Double.toString(0.0009765625 * r),
+          row.getFieldValue(5).toString());
+      assertEquals("row " + r, new Date(111, 6, 1 + r).toString(),
+          row.getFieldValue(6).toString());
+      Timestamp ts = new Timestamp(115, 9, 23, 10, 11, 59, 999999999);
+      ts.setTime(ts.getTime() + r * 1000);
       assertEquals("row " + r,
-          new Timestamp(115, 9, 25, 10, 11, 59 + r, 999999999),
-          times.asScratchTimestamp(r));
-      assertEquals("row " + r, "1.234567", decs.vector[r].toString());
-      assertEquals("row " + r, Integer.toString(r), strs.toString(r));
-      assertEquals("row " + r, Integer.toHexString(r), chars.toString(r));
-      assertEquals("row " + r, Integer.toHexString(r * 128), vcs.toString(r));
-      assertEquals("row " + r, r + 13, structInts.vector[r]);
-      assertEquals("row " + r, 1, unions.tags[r]);
-      assertEquals("row " + r, r + 42, unionInts.vector[r]);
-      assertEquals("row " + r, 3, lists.lengths[r]);
-      assertEquals("row " + r, 31415, listInts.vector[(int) lists.offsets[r]]);
-      assertEquals("row " + r, 31416, listInts.vector[(int) lists.offsets[r] + 1]);
-      assertEquals("row " + r, 31417, listInts.vector[(int) lists.offsets[r] + 2]);
-      assertEquals("row " + r, 3, maps.lengths[3]);
-      assertEquals("row " + r, Integer.toHexString(3 * r), mapKeys.toString((int) maps.offsets[r]));
-      assertEquals("row " + r, Integer.toString(3 * r), mapValues.toString((int) maps.offsets[r]));
-      assertEquals("row " + r, Integer.toHexString(3 * r + 1), mapKeys.toString((int) maps.offsets[r] + 1));
-      assertEquals("row " + r, Integer.toString(3 * r + 1), mapValues.toString((int) maps.offsets[r] + 1));
-      assertEquals("row " + r, Integer.toHexString(3 * r + 2), mapKeys.toString((int) maps.offsets[r] + 2));
-      assertEquals("row " + r, Integer.toString(3 * r + 2), mapValues.toString((int) maps.offsets[r] + 2));
+          ts.toString(),
+          row.getFieldValue(7).toString());
+      assertEquals("row " + r, "1.234567", row.getFieldValue(8).toString());
+      assertEquals("row " + r, Integer.toString(r),
+          row.getFieldValue(9).toString());
+      assertEquals("row " + r, pad(Integer.toHexString(r), 10),
+          row.getFieldValue(10).toString());
+      assertEquals("row " + r, Integer.toHexString(r * 128),
+          row.getFieldValue(11).toString());
+      assertEquals("row " + r, "{" + Integer.toString(r + 13) + "}",
+          row.getFieldValue(12).toString());
+      assertEquals("row " + r, "union(1, " + Integer.toString(r + 42) + ")",
+          row.getFieldValue(13).toString());
+      assertEquals("row " + r, "[31415, 31416, 31417]",
+          row.getFieldValue(14).toString());
+      expected = new StringBuilder();
+      expected.append('{');
+      expected.append(Integer.toHexString(3 * r));
+      expected.append('=');
+      expected.append(3 * r);
+      expected.append(", ");
+      expected.append(Integer.toHexString(3 * r + 1));
+      expected.append('=');
+      expected.append(3 * r + 1);
+      expected.append(", ");
+      expected.append(Integer.toHexString(3 * r + 2));
+      expected.append('=');
+      expected.append(3 * r + 2);
+      expected.append('}');
+      assertEquals("row " + r, expected.toString(),
+          row.getFieldValue(15).toString());
     }
 
     // should have no more rows
-    assertEquals(false, rows.nextBatch(batch));
+    assertEquals(false, rows.hasNext());
   }
 
   private static String makeString(BytesColumnVector vector, int row) {
@@ -2449,8 +2455,7 @@ public class TestVectorOrcFile {
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf));
     RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch();
-    assertEquals(true, rows.nextBatch(batch));
+    batch = rows.nextBatch(null);
     assertEquals(4, batch.size);
     // ORC currently trims the output strings. See HIVE-12286
     assertEquals("",
@@ -2499,20 +2504,19 @@ public class TestVectorOrcFile {
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf));
     RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch();
-    assertEquals(true, rows.nextBatch(batch));
+    batch = rows.nextBatch(null);
     assertEquals(1024, batch.size);
     for(int r=0; r < 1024; ++r) {
       assertEquals(Integer.toString(r * 10001),
           makeString((BytesColumnVector) batch.cols[0], r));
     }
-    assertEquals(true, rows.nextBatch(batch));
+    batch = rows.nextBatch(batch);
     assertEquals(1024, batch.size);
     for(int r=0; r < 1024; ++r) {
       assertEquals("Halloween",
           makeString((BytesColumnVector) batch.cols[0], r));
     }
-    assertEquals(false, rows.nextBatch(batch));
+    assertEquals(false, rows.hasNext());
   }
 
   @Test
@@ -2537,21 +2541,18 @@ public class TestVectorOrcFile {
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf));
     RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch();
-    rows.nextBatch(batch);
-    assertEquals(1024, batch.size);
-    StructColumnVector inner = (StructColumnVector) batch.cols[0];
-    LongColumnVector vec = (LongColumnVector) inner.fields[0];
+    OrcStruct row = null;
     for(int r=0; r < 1024; ++r) {
+      assertEquals(true, rows.hasNext());
+      row = (OrcStruct) rows.next(row);
+      OrcStruct inner = (OrcStruct) row.getFieldValue(0);
       if (r < 200 || (r >= 400 && r < 600) || r >= 800) {
-        assertEquals("row " + r, true, inner.isNull[r]);
+        assertEquals("row " + r, null, inner);
       } else {
-        assertEquals("row " + r, false, inner.isNull[r]);
-        assertEquals("row " + r, r, vec.vector[r]);
+        assertEquals("row " + r, "{" + r + "}", inner.toString());
       }
     }
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
+    assertEquals(false, rows.hasNext());
   }
 
   /**
@@ -2594,38 +2595,28 @@ public class TestVectorOrcFile {
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf));
     RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch(1024);
-    UnionColumnVector union = (UnionColumnVector) batch.cols[0];
-    LongColumnVector ints = (LongColumnVector) union.fields[0];
-    LongColumnVector longs = (LongColumnVector) union.fields[1];
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1024, batch.size);
+    OrcStruct row = null;
     for(int r=0; r < 1024; ++r) {
+      assertEquals(true, rows.hasNext());
+      row = (OrcStruct) rows.next(row);
+      OrcUnion inner = (OrcUnion) row.getFieldValue(0);
       if (r < 200) {
-        assertEquals("row " + r, true, union.isNull[r]);
+        assertEquals("row " + r, null, inner);
       } else if (r < 300) {
-        assertEquals("row " + r, false, union.isNull[r]);
-        assertEquals("row " + r, 0, union.tags[r]);
-        assertEquals("row " + r, r, ints.vector[r]);
+        assertEquals("row " + r, "union(0, " + r +")", inner.toString());
       } else if (r < 400) {
-        assertEquals("row " + r, false, union.isNull[r]);
-        assertEquals("row " + r, 1, union.tags[r]);
-        assertEquals("row " + r, -r, longs.vector[r]);
+        assertEquals("row " + r, "union(1, " + -r +")", inner.toString());
       } else if (r < 600) {
-        assertEquals("row " + r, true, union.isNull[r]);
+        assertEquals("row " + r, null, inner);
       } else if (r < 800) {
-        assertEquals("row " + r, false, union.isNull[r]);
-        assertEquals("row " + r, 1, union.tags[r]);
-        assertEquals("row " + r, -r, longs.vector[r]);
+        assertEquals("row " + r, "union(1, " + -r +")", inner.toString());
       } else if (r < 1000) {
-        assertEquals("row " + r, true, union.isNull[r]);
+        assertEquals("row " + r, null, inner);
       } else {
-        assertEquals("row " + r, false, union.isNull[r]);
-        assertEquals("row " + r, 1, union.tags[r]);
-        assertEquals("row " + r, -r, longs.vector[r]);
+        assertEquals("row " + r, "union(1, " + -r +")", inner.toString());
       }
     }
-    assertEquals(false, rows.nextBatch(batch));
+    assertEquals(false, rows.hasNext());
   }
 
   /**
@@ -2672,33 +2663,31 @@ public class TestVectorOrcFile {
     Reader r

<TRUNCATED>