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/29 00:15:19 UTC

[01/20] hive git commit: HIVE-13541: Pass view's ColumnAccessInfo to HiveAuthorizer (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/llap ffed821ca -> 390cb8cd0


HIVE-13541: Pass view's ColumnAccessInfo to HiveAuthorizer (Pengcheng Xiong, 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/15485012
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/15485012
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/15485012

Branch: refs/heads/llap
Commit: 15485012424b84e4205ac30c09c33548c81f8d79
Parents: 0ac424f
Author: Pengcheng Xiong <px...@apache.org>
Authored: Tue Apr 26 10:52:36 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Tue Apr 26 10:52:36 2016 -0700

----------------------------------------------------------------------
 .../TestHiveAuthorizerCheckInvocation.java      | 52 ++++++++++++++++++++
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  2 +-
 .../ql/optimizer/ColumnPrunerProcFactory.java   |  2 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      |  2 +-
 .../hive/ql/parse/ColumnAccessAnalyzer.java     | 31 +++++++-----
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 12 +++--
 .../hadoop/hive/ql/parse/TestColumnAccess.java  | 10 ++--
 7 files changed, 86 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/15485012/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
index acf2663..5e601c9 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
@@ -60,6 +60,8 @@ public class TestHiveAuthorizerCheckInvocation {
   protected static Driver driver;
   private static final String tableName = TestHiveAuthorizerCheckInvocation.class.getSimpleName()
       + "Table";
+  private static final String viewName = TestHiveAuthorizerCheckInvocation.class.getSimpleName()
+      + "View";
   private static final String inDbTableName = tableName + "_in_db";
   private static final String acidTableName = tableName + "_acid";
   private static final String dbName = TestHiveAuthorizerCheckInvocation.class.getSimpleName()
@@ -97,6 +99,7 @@ public class TestHiveAuthorizerCheckInvocation {
     driver = new Driver(conf);
     runCmd("create table " + tableName
         + " (i int, j int, k string) partitioned by (city string, `date` string) ");
+    runCmd("create view " + viewName + " as select * from " + tableName);
     runCmd("create database " + dbName);
     runCmd("create table " + dbName + "." + inDbTableName + "(i int)");
     // Need a separate table for ACID testing since it has to be bucketed and it has to be Acid
@@ -114,6 +117,7 @@ public class TestHiveAuthorizerCheckInvocation {
     // Drop the tables when we're done.  This makes the test work inside an IDE
     runCmd("drop table if exists " + acidTableName);
     runCmd("drop table if exists " + tableName);
+    runCmd("drop table if exists " + viewName);
     runCmd("drop table if exists " + dbName + "." + inDbTableName);
     runCmd("drop database if exists " + dbName );
     driver.close();
@@ -136,6 +140,46 @@ public class TestHiveAuthorizerCheckInvocation {
         getSortedList(tableObj.getColumns()));
   }
 
+  @Test
+  public void testInputSomeColumnsUsedView() throws HiveAuthzPluginException, HiveAccessControlException,
+  CommandNeedRetryException {
+
+    reset(mockedAuthorizer);
+    int status = driver.compile("select i from " + viewName
+        + " where k = 'X' and city = 'Scottsdale-AZ' ");
+    assertEquals(0, status);
+
+    List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+    checkSingleViewInput(inputs);
+    HivePrivilegeObject tableObj = inputs.get(0);
+    assertEquals("no of columns used", 3, tableObj.getColumns().size());
+    assertEquals("Columns used", Arrays.asList("city", "i", "k"),
+        getSortedList(tableObj.getColumns()));
+  }
+
+  @Test
+  public void testInputSomeColumnsUsedJoin() throws HiveAuthzPluginException, HiveAccessControlException,
+  CommandNeedRetryException {
+    
+    reset(mockedAuthorizer);
+    int status = driver.compile("select " + viewName + ".i, " + tableName + ".city from "
+        + viewName + " join " + tableName + " on " + viewName + ".city = " + tableName
+        + ".city where " + tableName + ".k = 'X'");
+    assertEquals(0, status);
+    
+    List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+    Collections.sort(inputs);
+    assertEquals(inputs.size(), 2);
+    HivePrivilegeObject tableObj = inputs.get(0);
+    assertEquals(tableObj.getObjectName().toLowerCase(), tableName.toLowerCase());
+    assertEquals("no of columns used", 2, tableObj.getColumns().size());
+    assertEquals("Columns used", Arrays.asList("city", "k"), getSortedList(tableObj.getColumns()));
+    tableObj = inputs.get(1);
+    assertEquals(tableObj.getObjectName().toLowerCase(), viewName.toLowerCase());
+    assertEquals("no of columns used", 2, tableObj.getColumns().size());
+    assertEquals("Columns used", Arrays.asList("city", "i"), getSortedList(tableObj.getColumns()));
+  }
+
   private List<String> getSortedList(List<String> columns) {
     List<String> sortedCols = new ArrayList<String>(columns);
     Collections.sort(sortedCols);
@@ -355,6 +399,14 @@ public class TestHiveAuthorizerCheckInvocation {
     assertTrue("table name", tableName.equalsIgnoreCase(tableObj.getObjectName()));
   }
 
+  private void checkSingleViewInput(List<HivePrivilegeObject> inputs) {
+    assertEquals("number of inputs", 1, inputs.size());
+
+    HivePrivilegeObject tableObj = inputs.get(0);
+    assertEquals("input type", HivePrivilegeObjectType.TABLE_OR_VIEW, tableObj.getType());
+    assertTrue("table name", viewName.equalsIgnoreCase(tableObj.getObjectName()));
+  }
+
   /**
    * @return pair with left value as inputs and right value as outputs,
    *  passed in current call to authorizer.checkPrivileges

http://git-wip-us.apache.org/repos/asf/hive/blob/15485012/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 7f72efb..dad43fb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -701,7 +701,7 @@ public class Driver implements CommandProcessor {
         Table tbl = read.getTable();
         if (tbl.isView() && sem instanceof SemanticAnalyzer) {
           tab2Cols.put(tbl,
-              sem.getColumnAccessInfo().getTableToColumnAccessMap().get(tbl.getTableName()));
+              sem.getColumnAccessInfo().getTableToColumnAccessMap().get(tbl.getCompleteName()));
         }
         if (read.getPartition() != null) {
           Partition partition = read.getPartition();

http://git-wip-us.apache.org/repos/asf/hive/blob/15485012/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
index 7638ba0..a2a7f00 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
@@ -790,7 +790,7 @@ public final class ColumnPrunerProcFactory {
           int index = originalOutputColumnNames.indexOf(col);
           Table tab = cppCtx.getParseContext().getViewProjectToTableSchema().get(op);
           cppCtx.getParseContext().getColumnAccessInfo()
-              .add(tab.getTableName(), tab.getCols().get(index).getName());
+              .add(tab.getCompleteName(), tab.getCols().get(index).getName());
         }
       }
       if (cols.size() < originalOutputColumnNames.size()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/15485012/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
index 03002cc..b0cb8df 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
@@ -387,7 +387,7 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
         if (this.columnAccessInfo != null && this.viewProjectToTableSchema != null
             && this.viewProjectToTableSchema.containsKey(project)) {
           Table tab = this.viewProjectToTableSchema.get(project);
-          this.columnAccessInfo.add(tab.getTableName(), tab.getCols().get(ord.i).getName());
+          this.columnAccessInfo.add(tab.getCompleteName(), tab.getCols().get(ord.i).getName());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/15485012/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessAnalyzer.java
index dcc8daf..777734b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessAnalyzer.java
@@ -34,21 +34,26 @@ public class ColumnAccessAnalyzer {
     pGraphContext = pactx;
   }
 
-  public ColumnAccessInfo analyzeColumnAccess() throws SemanticException {
-    ColumnAccessInfo columnAccessInfo = new ColumnAccessInfo();
+  public ColumnAccessInfo analyzeColumnAccess(ColumnAccessInfo columnAccessInfo) throws SemanticException {
+    if (columnAccessInfo == null) {
+      columnAccessInfo = new ColumnAccessInfo();
+    }
     Collection<TableScanOperator> topOps = pGraphContext.getTopOps().values();
     for (TableScanOperator top : topOps) {
-      Table table = top.getConf().getTableMetadata();
-      String tableName = table.getCompleteName();
-      List<String> referenced = top.getReferencedColumns();
-      for (String column : referenced) {
-        columnAccessInfo.add(tableName, column);
-      }
-      if (table.isPartitioned()) {
-        PrunedPartitionList parts = pGraphContext.getPrunedPartitions(table.getTableName(), top);
-        if (parts.getReferredPartCols() != null) {
-          for (String partKey : parts.getReferredPartCols()) {
-            columnAccessInfo.add(tableName, partKey);
+      // if a table is inside view, we do not care about its authorization.
+      if (!top.isInsideView()) {
+        Table table = top.getConf().getTableMetadata();
+        String tableName = table.getCompleteName();
+        List<String> referenced = top.getReferencedColumns();
+        for (String column : referenced) {
+          columnAccessInfo.add(tableName, column);
+        }
+        if (table.isPartitioned()) {
+          PrunedPartitionList parts = pGraphContext.getPrunedPartitions(table.getTableName(), top);
+          if (parts.getReferredPartCols() != null) {
+            for (String partKey : parts.getReferredPartCols()) {
+              columnAccessInfo.add(tableName, partKey);
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/15485012/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 197e8f1..cfe4497 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
@@ -2353,11 +2353,12 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     // if skip authorization, skip checking;
     // if it is inside a view, skip checking;
     // if authorization flag is not enabled, skip checking.
-    if (!this.skipAuthorization() && !qb.isInsideView()
-        && HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
+    // if HIVE_STATS_COLLECT_SCANCOLS is enabled, check.
+    if ((!this.skipAuthorization() && !qb.isInsideView() && HiveConf.getBoolVar(conf,
+        HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED))
+        || HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_STATS_COLLECT_SCANCOLS)) {
       qb.rewriteViewToSubq(alias, tab_name, qbexpr, tab);
-    }
-    else{
+    } else {
       qb.rewriteViewToSubq(alias, tab_name, qbexpr, null);
     }
   }
@@ -10753,7 +10754,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     if (isColumnInfoNeedForAuth
         || HiveConf.getBoolVar(this.conf, HiveConf.ConfVars.HIVE_STATS_COLLECT_SCANCOLS)) {
       ColumnAccessAnalyzer columnAccessAnalyzer = new ColumnAccessAnalyzer(pCtx);
-      setColumnAccessInfo(columnAccessAnalyzer.analyzeColumnAccess());
+      // view column access info is carried by this.getColumnAccessInfo().
+      setColumnAccessInfo(columnAccessAnalyzer.analyzeColumnAccess(this.getColumnAccessInfo()));
     }
 
     // 9. Optimize Physical op tree & Translate to target execution engine (MR,

http://git-wip-us.apache.org/repos/asf/hive/blob/15485012/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
index 5d22e27..11b20db 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
@@ -126,9 +126,11 @@ public class TestColumnAccess {
     QueryPlan plan = driver.getPlan();
     // check access columns from ColumnAccessInfo
     ColumnAccessInfo columnAccessInfo = plan.getColumnAccessInfo();
-    List<String> cols = columnAccessInfo.getTableToColumnAccessMap().get("default@v1");
+    // t1 is inside v1, we should not care about its access info.
+    List<String> cols = columnAccessInfo.getTableToColumnAccessMap().get("default@t1");
     Assert.assertNull(cols);
-    cols = columnAccessInfo.getTableToColumnAccessMap().get("default@t1");
+    // v1 is top level view, we should care about its access info.
+    cols = columnAccessInfo.getTableToColumnAccessMap().get("default@v1");
     Assert.assertNotNull(cols);
     Assert.assertEquals(2, cols.size());
     Assert.assertNotNull(cols.contains("id1"));
@@ -143,9 +145,9 @@ public class TestColumnAccess {
 
     // check access columns from readEntity
     Map<String, List<String>> tableColsMap = getColsFromReadEntity(plan.getInputs());
-    cols = tableColsMap.get("default@v1");
-    Assert.assertNull(cols);
     cols = tableColsMap.get("default@t1");
+    Assert.assertNull(cols);
+    cols = tableColsMap.get("default@v1");
     Assert.assertNotNull(cols);
     Assert.assertEquals(2, cols.size());
     Assert.assertNotNull(cols.contains("id1"));


[13/20] hive git commit: HIVE-13493 - Fix TransactionBatchImpl.getCurrentTxnId() and mis logging fixes (Eugene Koifman, reviewed by Wei Zheng)

Posted by jd...@apache.org.
HIVE-13493 - Fix TransactionBatchImpl.getCurrentTxnId() and mis logging fixes (Eugene Koifman, reviewed by Wei Zheng)


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

Branch: refs/heads/llap
Commit: ce457a4962a8f7a43c0da7647d958e5cc87b5dd8
Parents: 6a4e080
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Wed Apr 27 15:45:39 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Wed Apr 27 15:45:39 2016 -0700

----------------------------------------------------------------------
 .../src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java | 3 ++-
 .../test/org/apache/hive/hcatalog/streaming/TestStreaming.java    | 2 ++
 .../apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java    | 3 ++-
 .../src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java | 2 ++
 4 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ce457a49/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
index baeafad..db9fd72 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
@@ -622,7 +622,7 @@ public class HiveEndPoint {
 
     private void beginNextTransactionImpl() throws TransactionError {
       state = TxnState.INACTIVE;//clear state from previous txn
-      if ( currentTxnIndex >= txnIds.size() )
+      if ( currentTxnIndex + 1 >= txnIds.size() )
         throw new InvalidTrasactionState("No more transactions available in" +
                 " current batch for end point : " + endPt);
       ++currentTxnIndex;
@@ -874,6 +874,7 @@ public class HiveEndPoint {
               currentTxnIndex < txnIds.size(); currentTxnIndex++) {
             msClient.rollbackTxn(txnIds.get(currentTxnIndex));
           }
+          currentTxnIndex--;//since the loop left it == txnId.size()
         }
         else {
           if (getCurrentTxnId() > 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ce457a49/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
index bde78e4..f4ee208 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
@@ -1714,6 +1714,8 @@ public class TestStreaming {
     }
     catch(StreamingIOFailure ex) {
       expectedEx = ex;
+      txnBatch.getCurrentTransactionState();
+      txnBatch.getCurrentTxnId();//test it doesn't throw ArrayIndexOutOfBounds...
     }
     Assert.assertTrue("Wrong exception: " + (expectedEx != null ? expectedEx.getMessage() : "?"),
       expectedEx != null && expectedEx.getMessage().contains("Simulated fault occurred"));

http://git-wip-us.apache.org/repos/asf/hive/blob/ce457a49/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
index 67e661f..ab7da68 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
@@ -30,6 +30,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -451,7 +452,7 @@ class CompactionTxnHandler extends TxnHandler {
         if(txnids.size() <= 0) {
           return;
         }
-
+        Collections.sort(txnids);//easier to read logs
         List<String> queries = new ArrayList<String>();
         StringBuilder prefix = new StringBuilder();
         StringBuilder suffix = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hive/blob/ce457a49/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index df6591f..c32b0b0 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -2345,6 +2345,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         deletedLocks += stmt.executeUpdate(query);
       }
       if(deletedLocks > 0) {
+        Collections.sort(extLockIDs);////easier to read logs
         LOG.info("Deleted " + deletedLocks + " ext locks from HIVE_LOCKS due to timeout (vs. " +
             extLockIDs.size() + " found. List: " + extLockIDs + ") maxHeartbeatTime=" + maxHeartbeatTime);
       }
@@ -2444,6 +2445,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             dbConn.commit();
             numTxnsAborted += batchToAbort.size();
             //todo: add TXNS.COMMENT filed and set it to 'aborted by system due to timeout'
+            Collections.sort(batchToAbort);//easier to read logs
             LOG.info("Aborted the following transactions due to timeout: " + batchToAbort.toString());
           }
           else {


[11/20] hive git commit: HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-cpp/ThriftHive.cpp
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-cpp/ThriftHive.cpp b/service/src/gen/thrift/gen-cpp/ThriftHive.cpp
deleted file mode 100644
index a5448f0..0000000
--- a/service/src/gen/thrift/gen-cpp/ThriftHive.cpp
+++ /dev/null
@@ -1,3544 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-#include "ThriftHive.h"
-
-namespace Apache { namespace Hadoop { namespace Hive {
-
-
-ThriftHive_execute_args::~ThriftHive_execute_args() throw() {
-}
-
-
-uint32_t ThriftHive_execute_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->query);
-          this->__isset.query = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_execute_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_execute_args");
-
-  xfer += oprot->writeFieldBegin("query", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->query);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_execute_pargs::~ThriftHive_execute_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_execute_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_execute_pargs");
-
-  xfer += oprot->writeFieldBegin("query", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->query)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_execute_result::~ThriftHive_execute_result() throw() {
-}
-
-
-uint32_t ThriftHive_execute_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->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_execute_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_execute_result");
-
-  if (this->__isset.ex) {
-    xfer += oprot->writeFieldBegin("ex", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->ex.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_execute_presult::~ThriftHive_execute_presult() throw() {
-}
-
-
-uint32_t ThriftHive_execute_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->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHive_fetchOne_args::~ThriftHive_fetchOne_args() throw() {
-}
-
-
-uint32_t ThriftHive_fetchOne_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_fetchOne_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_fetchOne_args");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchOne_pargs::~ThriftHive_fetchOne_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_fetchOne_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_fetchOne_pargs");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchOne_result::~ThriftHive_fetchOne_result() throw() {
-}
-
-
-uint32_t ThriftHive_fetchOne_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->success);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_fetchOne_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_fetchOne_result");
-
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0);
-    xfer += oprot->writeString(this->success);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.ex) {
-    xfer += oprot->writeFieldBegin("ex", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->ex.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchOne_presult::~ThriftHive_fetchOne_presult() throw() {
-}
-
-
-uint32_t ThriftHive_fetchOne_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString((*(this->success)));
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHive_fetchN_args::~ThriftHive_fetchN_args() throw() {
-}
-
-
-uint32_t ThriftHive_fetchN_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_I32) {
-          xfer += iprot->readI32(this->numRows);
-          this->__isset.numRows = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_fetchN_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_fetchN_args");
-
-  xfer += oprot->writeFieldBegin("numRows", ::apache::thrift::protocol::T_I32, 1);
-  xfer += oprot->writeI32(this->numRows);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchN_pargs::~ThriftHive_fetchN_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_fetchN_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_fetchN_pargs");
-
-  xfer += oprot->writeFieldBegin("numRows", ::apache::thrift::protocol::T_I32, 1);
-  xfer += oprot->writeI32((*(this->numRows)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchN_result::~ThriftHive_fetchN_result() throw() {
-}
-
-
-uint32_t ThriftHive_fetchN_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 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size5;
-            ::apache::thrift::protocol::TType _etype8;
-            xfer += iprot->readListBegin(_etype8, _size5);
-            this->success.resize(_size5);
-            uint32_t _i9;
-            for (_i9 = 0; _i9 < _size5; ++_i9)
-            {
-              xfer += iprot->readString(this->success[_i9]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_fetchN_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_fetchN_result");
-
-  if (this->__isset.success) {
-    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 _iter10;
-      for (_iter10 = this->success.begin(); _iter10 != this->success.end(); ++_iter10)
-      {
-        xfer += oprot->writeString((*_iter10));
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.ex) {
-    xfer += oprot->writeFieldBegin("ex", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->ex.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchN_presult::~ThriftHive_fetchN_presult() throw() {
-}
-
-
-uint32_t ThriftHive_fetchN_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 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            (*(this->success)).clear();
-            uint32_t _size11;
-            ::apache::thrift::protocol::TType _etype14;
-            xfer += iprot->readListBegin(_etype14, _size11);
-            (*(this->success)).resize(_size11);
-            uint32_t _i15;
-            for (_i15 = 0; _i15 < _size11; ++_i15)
-            {
-              xfer += iprot->readString((*(this->success))[_i15]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHive_fetchAll_args::~ThriftHive_fetchAll_args() throw() {
-}
-
-
-uint32_t ThriftHive_fetchAll_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_fetchAll_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_fetchAll_args");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchAll_pargs::~ThriftHive_fetchAll_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_fetchAll_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_fetchAll_pargs");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchAll_result::~ThriftHive_fetchAll_result() throw() {
-}
-
-
-uint32_t ThriftHive_fetchAll_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 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size16;
-            ::apache::thrift::protocol::TType _etype19;
-            xfer += iprot->readListBegin(_etype19, _size16);
-            this->success.resize(_size16);
-            uint32_t _i20;
-            for (_i20 = 0; _i20 < _size16; ++_i20)
-            {
-              xfer += iprot->readString(this->success[_i20]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_fetchAll_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_fetchAll_result");
-
-  if (this->__isset.success) {
-    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 _iter21;
-      for (_iter21 = this->success.begin(); _iter21 != this->success.end(); ++_iter21)
-      {
-        xfer += oprot->writeString((*_iter21));
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.ex) {
-    xfer += oprot->writeFieldBegin("ex", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->ex.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_fetchAll_presult::~ThriftHive_fetchAll_presult() throw() {
-}
-
-
-uint32_t ThriftHive_fetchAll_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 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            (*(this->success)).clear();
-            uint32_t _size22;
-            ::apache::thrift::protocol::TType _etype25;
-            xfer += iprot->readListBegin(_etype25, _size22);
-            (*(this->success)).resize(_size22);
-            uint32_t _i26;
-            for (_i26 = 0; _i26 < _size22; ++_i26)
-            {
-              xfer += iprot->readString((*(this->success))[_i26]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHive_getSchema_args::~ThriftHive_getSchema_args() throw() {
-}
-
-
-uint32_t ThriftHive_getSchema_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_getSchema_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_getSchema_args");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getSchema_pargs::~ThriftHive_getSchema_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_getSchema_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_getSchema_pargs");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getSchema_result::~ThriftHive_getSchema_result() throw() {
-}
-
-
-uint32_t ThriftHive_getSchema_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->success.read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_getSchema_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_getSchema_result");
-
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
-    xfer += this->success.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.ex) {
-    xfer += oprot->writeFieldBegin("ex", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->ex.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getSchema_presult::~ThriftHive_getSchema_presult() throw() {
-}
-
-
-uint32_t ThriftHive_getSchema_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += (*(this->success)).read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHive_getThriftSchema_args::~ThriftHive_getThriftSchema_args() throw() {
-}
-
-
-uint32_t ThriftHive_getThriftSchema_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_getThriftSchema_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_getThriftSchema_args");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getThriftSchema_pargs::~ThriftHive_getThriftSchema_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_getThriftSchema_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_getThriftSchema_pargs");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getThriftSchema_result::~ThriftHive_getThriftSchema_result() throw() {
-}
-
-
-uint32_t ThriftHive_getThriftSchema_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->success.read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_getThriftSchema_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_getThriftSchema_result");
-
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
-    xfer += this->success.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.ex) {
-    xfer += oprot->writeFieldBegin("ex", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->ex.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getThriftSchema_presult::~ThriftHive_getThriftSchema_presult() throw() {
-}
-
-
-uint32_t ThriftHive_getThriftSchema_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += (*(this->success)).read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHive_getClusterStatus_args::~ThriftHive_getClusterStatus_args() throw() {
-}
-
-
-uint32_t ThriftHive_getClusterStatus_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_getClusterStatus_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_getClusterStatus_args");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getClusterStatus_pargs::~ThriftHive_getClusterStatus_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_getClusterStatus_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_getClusterStatus_pargs");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getClusterStatus_result::~ThriftHive_getClusterStatus_result() throw() {
-}
-
-
-uint32_t ThriftHive_getClusterStatus_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->success.read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_getClusterStatus_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_getClusterStatus_result");
-
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
-    xfer += this->success.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.ex) {
-    xfer += oprot->writeFieldBegin("ex", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->ex.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getClusterStatus_presult::~ThriftHive_getClusterStatus_presult() throw() {
-}
-
-
-uint32_t ThriftHive_getClusterStatus_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += (*(this->success)).read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHive_getQueryPlan_args::~ThriftHive_getQueryPlan_args() throw() {
-}
-
-
-uint32_t ThriftHive_getQueryPlan_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_getQueryPlan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_getQueryPlan_args");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getQueryPlan_pargs::~ThriftHive_getQueryPlan_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_getQueryPlan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_getQueryPlan_pargs");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getQueryPlan_result::~ThriftHive_getQueryPlan_result() throw() {
-}
-
-
-uint32_t ThriftHive_getQueryPlan_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->success.read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_getQueryPlan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_getQueryPlan_result");
-
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
-    xfer += this->success.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.ex) {
-    xfer += oprot->writeFieldBegin("ex", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->ex.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_getQueryPlan_presult::~ThriftHive_getQueryPlan_presult() throw() {
-}
-
-
-uint32_t ThriftHive_getQueryPlan_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += (*(this->success)).read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->ex.read(iprot);
-          this->__isset.ex = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHive_clean_args::~ThriftHive_clean_args() throw() {
-}
-
-
-uint32_t ThriftHive_clean_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_clean_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_clean_args");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_clean_pargs::~ThriftHive_clean_pargs() throw() {
-}
-
-
-uint32_t ThriftHive_clean_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHive_clean_pargs");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_clean_result::~ThriftHive_clean_result() throw() {
-}
-
-
-uint32_t ThriftHive_clean_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHive_clean_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHive_clean_result");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHive_clean_presult::~ThriftHive_clean_presult() throw() {
-}
-
-
-uint32_t ThriftHive_clean_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-void ThriftHiveClient::execute(const std::string& query)
-{
-  send_execute(query);
-  recv_execute();
-}
-
-void ThriftHiveClient::send_execute(const std::string& query)
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("execute", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_execute_pargs args;
-  args.query = &query;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_execute()
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("execute") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_execute_presult result;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  if (result.__isset.ex) {
-    throw result.ex;
-  }
-  return;
-}
-
-void ThriftHiveClient::fetchOne(std::string& _return)
-{
-  send_fetchOne();
-  recv_fetchOne(_return);
-}
-
-void ThriftHiveClient::send_fetchOne()
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("fetchOne", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_fetchOne_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_fetchOne(std::string& _return)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("fetchOne") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_fetchOne_presult result;
-  result.success = &_return;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  if (result.__isset.success) {
-    // _return pointer has now been filled
-    return;
-  }
-  if (result.__isset.ex) {
-    throw result.ex;
-  }
-  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fetchOne failed: unknown result");
-}
-
-void ThriftHiveClient::fetchN(std::vector<std::string> & _return, const int32_t numRows)
-{
-  send_fetchN(numRows);
-  recv_fetchN(_return);
-}
-
-void ThriftHiveClient::send_fetchN(const int32_t numRows)
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("fetchN", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_fetchN_pargs args;
-  args.numRows = &numRows;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_fetchN(std::vector<std::string> & _return)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("fetchN") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_fetchN_presult result;
-  result.success = &_return;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  if (result.__isset.success) {
-    // _return pointer has now been filled
-    return;
-  }
-  if (result.__isset.ex) {
-    throw result.ex;
-  }
-  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fetchN failed: unknown result");
-}
-
-void ThriftHiveClient::fetchAll(std::vector<std::string> & _return)
-{
-  send_fetchAll();
-  recv_fetchAll(_return);
-}
-
-void ThriftHiveClient::send_fetchAll()
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("fetchAll", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_fetchAll_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_fetchAll(std::vector<std::string> & _return)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("fetchAll") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_fetchAll_presult result;
-  result.success = &_return;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  if (result.__isset.success) {
-    // _return pointer has now been filled
-    return;
-  }
-  if (result.__isset.ex) {
-    throw result.ex;
-  }
-  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fetchAll failed: unknown result");
-}
-
-void ThriftHiveClient::getSchema( ::Apache::Hadoop::Hive::Schema& _return)
-{
-  send_getSchema();
-  recv_getSchema(_return);
-}
-
-void ThriftHiveClient::send_getSchema()
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("getSchema", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_getSchema_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_getSchema( ::Apache::Hadoop::Hive::Schema& _return)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("getSchema") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_getSchema_presult result;
-  result.success = &_return;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  if (result.__isset.success) {
-    // _return pointer has now been filled
-    return;
-  }
-  if (result.__isset.ex) {
-    throw result.ex;
-  }
-  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getSchema failed: unknown result");
-}
-
-void ThriftHiveClient::getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return)
-{
-  send_getThriftSchema();
-  recv_getThriftSchema(_return);
-}
-
-void ThriftHiveClient::send_getThriftSchema()
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("getThriftSchema", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_getThriftSchema_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("getThriftSchema") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_getThriftSchema_presult result;
-  result.success = &_return;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  if (result.__isset.success) {
-    // _return pointer has now been filled
-    return;
-  }
-  if (result.__isset.ex) {
-    throw result.ex;
-  }
-  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getThriftSchema failed: unknown result");
-}
-
-void ThriftHiveClient::getClusterStatus(HiveClusterStatus& _return)
-{
-  send_getClusterStatus();
-  recv_getClusterStatus(_return);
-}
-
-void ThriftHiveClient::send_getClusterStatus()
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("getClusterStatus", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_getClusterStatus_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_getClusterStatus(HiveClusterStatus& _return)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("getClusterStatus") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_getClusterStatus_presult result;
-  result.success = &_return;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  if (result.__isset.success) {
-    // _return pointer has now been filled
-    return;
-  }
-  if (result.__isset.ex) {
-    throw result.ex;
-  }
-  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getClusterStatus failed: unknown result");
-}
-
-void ThriftHiveClient::getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return)
-{
-  send_getQueryPlan();
-  recv_getQueryPlan(_return);
-}
-
-void ThriftHiveClient::send_getQueryPlan()
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("getQueryPlan", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_getQueryPlan_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("getQueryPlan") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_getQueryPlan_presult result;
-  result.success = &_return;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  if (result.__isset.success) {
-    // _return pointer has now been filled
-    return;
-  }
-  if (result.__isset.ex) {
-    throw result.ex;
-  }
-  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getQueryPlan failed: unknown result");
-}
-
-void ThriftHiveClient::clean()
-{
-  send_clean();
-  recv_clean();
-}
-
-void ThriftHiveClient::send_clean()
-{
-  int32_t cseqid = 0;
-  oprot_->writeMessageBegin("clean", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_clean_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-}
-
-void ThriftHiveClient::recv_clean()
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  iprot_->readMessageBegin(fname, mtype, rseqid);
-  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-    ::apache::thrift::TApplicationException x;
-    x.read(iprot_);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-    throw x;
-  }
-  if (mtype != ::apache::thrift::protocol::T_REPLY) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  if (fname.compare("clean") != 0) {
-    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-    iprot_->readMessageEnd();
-    iprot_->getTransport()->readEnd();
-  }
-  ThriftHive_clean_presult result;
-  result.read(iprot_);
-  iprot_->readMessageEnd();
-  iprot_->getTransport()->readEnd();
-
-  return;
-}
-
-bool ThriftHiveProcessor::dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext) {
-  ProcessMap::iterator pfn;
-  pfn = processMap_.find(fname);
-  if (pfn == processMap_.end()) {
-    return  ::Apache::Hadoop::Hive::ThriftHiveMetastoreProcessor::dispatchCall(iprot, oprot, fname, seqid, callContext);
-  }
-  (this->*(pfn->second))(seqid, iprot, oprot, callContext);
-  return true;
-}
-
-void ThriftHiveProcessor::process_execute(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.execute", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.execute");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.execute");
-  }
-
-  ThriftHive_execute_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.execute", bytes);
-  }
-
-  ThriftHive_execute_result result;
-  try {
-    iface_->execute(args.query);
-  } catch (HiveServerException &ex) {
-    result.ex = ex;
-    result.__isset.ex = true;
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.execute");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("execute", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.execute");
-  }
-
-  oprot->writeMessageBegin("execute", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.execute", bytes);
-  }
-}
-
-void ThriftHiveProcessor::process_fetchOne(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.fetchOne", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.fetchOne");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.fetchOne");
-  }
-
-  ThriftHive_fetchOne_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.fetchOne", bytes);
-  }
-
-  ThriftHive_fetchOne_result result;
-  try {
-    iface_->fetchOne(result.success);
-    result.__isset.success = true;
-  } catch (HiveServerException &ex) {
-    result.ex = ex;
-    result.__isset.ex = true;
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.fetchOne");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("fetchOne", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.fetchOne");
-  }
-
-  oprot->writeMessageBegin("fetchOne", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.fetchOne", bytes);
-  }
-}
-
-void ThriftHiveProcessor::process_fetchN(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.fetchN", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.fetchN");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.fetchN");
-  }
-
-  ThriftHive_fetchN_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.fetchN", bytes);
-  }
-
-  ThriftHive_fetchN_result result;
-  try {
-    iface_->fetchN(result.success, args.numRows);
-    result.__isset.success = true;
-  } catch (HiveServerException &ex) {
-    result.ex = ex;
-    result.__isset.ex = true;
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.fetchN");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("fetchN", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.fetchN");
-  }
-
-  oprot->writeMessageBegin("fetchN", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.fetchN", bytes);
-  }
-}
-
-void ThriftHiveProcessor::process_fetchAll(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.fetchAll", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.fetchAll");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.fetchAll");
-  }
-
-  ThriftHive_fetchAll_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.fetchAll", bytes);
-  }
-
-  ThriftHive_fetchAll_result result;
-  try {
-    iface_->fetchAll(result.success);
-    result.__isset.success = true;
-  } catch (HiveServerException &ex) {
-    result.ex = ex;
-    result.__isset.ex = true;
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.fetchAll");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("fetchAll", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.fetchAll");
-  }
-
-  oprot->writeMessageBegin("fetchAll", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.fetchAll", bytes);
-  }
-}
-
-void ThriftHiveProcessor::process_getSchema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.getSchema", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.getSchema");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.getSchema");
-  }
-
-  ThriftHive_getSchema_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.getSchema", bytes);
-  }
-
-  ThriftHive_getSchema_result result;
-  try {
-    iface_->getSchema(result.success);
-    result.__isset.success = true;
-  } catch (HiveServerException &ex) {
-    result.ex = ex;
-    result.__isset.ex = true;
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.getSchema");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("getSchema", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.getSchema");
-  }
-
-  oprot->writeMessageBegin("getSchema", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.getSchema", bytes);
-  }
-}
-
-void ThriftHiveProcessor::process_getThriftSchema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.getThriftSchema", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.getThriftSchema");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.getThriftSchema");
-  }
-
-  ThriftHive_getThriftSchema_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.getThriftSchema", bytes);
-  }
-
-  ThriftHive_getThriftSchema_result result;
-  try {
-    iface_->getThriftSchema(result.success);
-    result.__isset.success = true;
-  } catch (HiveServerException &ex) {
-    result.ex = ex;
-    result.__isset.ex = true;
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.getThriftSchema");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("getThriftSchema", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.getThriftSchema");
-  }
-
-  oprot->writeMessageBegin("getThriftSchema", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.getThriftSchema", bytes);
-  }
-}
-
-void ThriftHiveProcessor::process_getClusterStatus(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.getClusterStatus", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.getClusterStatus");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.getClusterStatus");
-  }
-
-  ThriftHive_getClusterStatus_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.getClusterStatus", bytes);
-  }
-
-  ThriftHive_getClusterStatus_result result;
-  try {
-    iface_->getClusterStatus(result.success);
-    result.__isset.success = true;
-  } catch (HiveServerException &ex) {
-    result.ex = ex;
-    result.__isset.ex = true;
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.getClusterStatus");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("getClusterStatus", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.getClusterStatus");
-  }
-
-  oprot->writeMessageBegin("getClusterStatus", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.getClusterStatus", bytes);
-  }
-}
-
-void ThriftHiveProcessor::process_getQueryPlan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.getQueryPlan", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.getQueryPlan");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.getQueryPlan");
-  }
-
-  ThriftHive_getQueryPlan_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.getQueryPlan", bytes);
-  }
-
-  ThriftHive_getQueryPlan_result result;
-  try {
-    iface_->getQueryPlan(result.success);
-    result.__isset.success = true;
-  } catch (HiveServerException &ex) {
-    result.ex = ex;
-    result.__isset.ex = true;
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.getQueryPlan");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("getQueryPlan", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.getQueryPlan");
-  }
-
-  oprot->writeMessageBegin("getQueryPlan", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.getQueryPlan", bytes);
-  }
-}
-
-void ThriftHiveProcessor::process_clean(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
-{
-  void* ctx = NULL;
-  if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHive.clean", callContext);
-  }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHive.clean");
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHive.clean");
-  }
-
-  ThriftHive_clean_args args;
-  args.read(iprot);
-  iprot->readMessageEnd();
-  uint32_t bytes = iprot->getTransport()->readEnd();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHive.clean", bytes);
-  }
-
-  ThriftHive_clean_result result;
-  try {
-    iface_->clean();
-  } catch (const std::exception& e) {
-    if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHive.clean");
-    }
-
-    ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("clean", ::apache::thrift::protocol::T_EXCEPTION, seqid);
-    x.write(oprot);
-    oprot->writeMessageEnd();
-    oprot->getTransport()->writeEnd();
-    oprot->getTransport()->flush();
-    return;
-  }
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHive.clean");
-  }
-
-  oprot->writeMessageBegin("clean", ::apache::thrift::protocol::T_REPLY, seqid);
-  result.write(oprot);
-  oprot->writeMessageEnd();
-  bytes = oprot->getTransport()->writeEnd();
-  oprot->getTransport()->flush();
-
-  if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHive.clean", bytes);
-  }
-}
-
-::boost::shared_ptr< ::apache::thrift::TProcessor > ThriftHiveProcessorFactory::getProcessor(const ::apache::thrift::TConnectionInfo& connInfo) {
-  ::apache::thrift::ReleaseHandler< ThriftHiveIfFactory > cleanup(handlerFactory_);
-  ::boost::shared_ptr< ThriftHiveIf > handler(handlerFactory_->getHandler(connInfo), cleanup);
-  ::boost::shared_ptr< ::apache::thrift::TProcessor > processor(new ThriftHiveProcessor(handler));
-  return processor;
-}
-
-void ThriftHiveConcurrentClient::execute(const std::string& query)
-{
-  int32_t seqid = send_execute(query);
-  recv_execute(seqid);
-}
-
-int32_t ThriftHiveConcurrentClient::send_execute(const std::string& query)
-{
-  int32_t cseqid = this->sync_.generateSeqId();
-  ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_);
-  oprot_->writeMessageBegin("execute", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_execute_pargs args;
-  args.query = &query;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-
-  sentry.commit();
-  return cseqid;
-}
-
-void ThriftHiveConcurrentClient::recv_execute(const int32_t seqid)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  // the read mutex gets dropped and reacquired as part of waitForWork()
-  // The destructor of this sentry wakes up other clients
-  ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid);
-
-  while(true) {
-    if(!this->sync_.getPending(fname, mtype, rseqid)) {
-      iprot_->readMessageBegin(fname, mtype, rseqid);
-    }
-    if(seqid == rseqid) {
-      if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-        ::apache::thrift::TApplicationException x;
-        x.read(iprot_);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-        sentry.commit();
-        throw x;
-      }
-      if (mtype != ::apache::thrift::protocol::T_REPLY) {
-        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-      }
-      if (fname.compare("execute") != 0) {
-        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-
-        // in a bad state, don't commit
-        using ::apache::thrift::protocol::TProtocolException;
-        throw TProtocolException(TProtocolException::INVALID_DATA);
-      }
-      ThriftHive_execute_presult result;
-      result.read(iprot_);
-      iprot_->readMessageEnd();
-      iprot_->getTransport()->readEnd();
-
-      if (result.__isset.ex) {
-        sentry.commit();
-        throw result.ex;
-      }
-      sentry.commit();
-      return;
-    }
-    // seqid != rseqid
-    this->sync_.updatePending(fname, mtype, rseqid);
-
-    // this will temporarily unlock the readMutex, and let other clients get work done
-    this->sync_.waitForWork(seqid);
-  } // end while(true)
-}
-
-void ThriftHiveConcurrentClient::fetchOne(std::string& _return)
-{
-  int32_t seqid = send_fetchOne();
-  recv_fetchOne(_return, seqid);
-}
-
-int32_t ThriftHiveConcurrentClient::send_fetchOne()
-{
-  int32_t cseqid = this->sync_.generateSeqId();
-  ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_);
-  oprot_->writeMessageBegin("fetchOne", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_fetchOne_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-
-  sentry.commit();
-  return cseqid;
-}
-
-void ThriftHiveConcurrentClient::recv_fetchOne(std::string& _return, const int32_t seqid)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  // the read mutex gets dropped and reacquired as part of waitForWork()
-  // The destructor of this sentry wakes up other clients
-  ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid);
-
-  while(true) {
-    if(!this->sync_.getPending(fname, mtype, rseqid)) {
-      iprot_->readMessageBegin(fname, mtype, rseqid);
-    }
-    if(seqid == rseqid) {
-      if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-        ::apache::thrift::TApplicationException x;
-        x.read(iprot_);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-        sentry.commit();
-        throw x;
-      }
-      if (mtype != ::apache::thrift::protocol::T_REPLY) {
-        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-      }
-      if (fname.compare("fetchOne") != 0) {
-        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-
-        // in a bad state, don't commit
-        using ::apache::thrift::protocol::TProtocolException;
-        throw TProtocolException(TProtocolException::INVALID_DATA);
-      }
-      ThriftHive_fetchOne_presult result;
-      result.success = &_return;
-      result.read(iprot_);
-      iprot_->readMessageEnd();
-      iprot_->getTransport()->readEnd();
-
-      if (result.__isset.success) {
-        // _return pointer has now been filled
-        sentry.commit();
-        return;
-      }
-      if (result.__isset.ex) {
-        sentry.commit();
-        throw result.ex;
-      }
-      // in a bad state, don't commit
-      throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fetchOne failed: unknown result");
-    }
-    // seqid != rseqid
-    this->sync_.updatePending(fname, mtype, rseqid);
-
-    // this will temporarily unlock the readMutex, and let other clients get work done
-    this->sync_.waitForWork(seqid);
-  } // end while(true)
-}
-
-void ThriftHiveConcurrentClient::fetchN(std::vector<std::string> & _return, const int32_t numRows)
-{
-  int32_t seqid = send_fetchN(numRows);
-  recv_fetchN(_return, seqid);
-}
-
-int32_t ThriftHiveConcurrentClient::send_fetchN(const int32_t numRows)
-{
-  int32_t cseqid = this->sync_.generateSeqId();
-  ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_);
-  oprot_->writeMessageBegin("fetchN", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_fetchN_pargs args;
-  args.numRows = &numRows;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-
-  sentry.commit();
-  return cseqid;
-}
-
-void ThriftHiveConcurrentClient::recv_fetchN(std::vector<std::string> & _return, const int32_t seqid)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  // the read mutex gets dropped and reacquired as part of waitForWork()
-  // The destructor of this sentry wakes up other clients
-  ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid);
-
-  while(true) {
-    if(!this->sync_.getPending(fname, mtype, rseqid)) {
-      iprot_->readMessageBegin(fname, mtype, rseqid);
-    }
-    if(seqid == rseqid) {
-      if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-        ::apache::thrift::TApplicationException x;
-        x.read(iprot_);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-        sentry.commit();
-        throw x;
-      }
-      if (mtype != ::apache::thrift::protocol::T_REPLY) {
-        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-      }
-      if (fname.compare("fetchN") != 0) {
-        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-
-        // in a bad state, don't commit
-        using ::apache::thrift::protocol::TProtocolException;
-        throw TProtocolException(TProtocolException::INVALID_DATA);
-      }
-      ThriftHive_fetchN_presult result;
-      result.success = &_return;
-      result.read(iprot_);
-      iprot_->readMessageEnd();
-      iprot_->getTransport()->readEnd();
-
-      if (result.__isset.success) {
-        // _return pointer has now been filled
-        sentry.commit();
-        return;
-      }
-      if (result.__isset.ex) {
-        sentry.commit();
-        throw result.ex;
-      }
-      // in a bad state, don't commit
-      throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fetchN failed: unknown result");
-    }
-    // seqid != rseqid
-    this->sync_.updatePending(fname, mtype, rseqid);
-
-    // this will temporarily unlock the readMutex, and let other clients get work done
-    this->sync_.waitForWork(seqid);
-  } // end while(true)
-}
-
-void ThriftHiveConcurrentClient::fetchAll(std::vector<std::string> & _return)
-{
-  int32_t seqid = send_fetchAll();
-  recv_fetchAll(_return, seqid);
-}
-
-int32_t ThriftHiveConcurrentClient::send_fetchAll()
-{
-  int32_t cseqid = this->sync_.generateSeqId();
-  ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_);
-  oprot_->writeMessageBegin("fetchAll", ::apache::thrift::protocol::T_CALL, cseqid);
-
-  ThriftHive_fetchAll_pargs args;
-  args.write(oprot_);
-
-  oprot_->writeMessageEnd();
-  oprot_->getTransport()->writeEnd();
-  oprot_->getTransport()->flush();
-
-  sentry.commit();
-  return cseqid;
-}
-
-void ThriftHiveConcurrentClient::recv_fetchAll(std::vector<std::string> & _return, const int32_t seqid)
-{
-
-  int32_t rseqid = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TMessageType mtype;
-
-  // the read mutex gets dropped and reacquired as part of waitForWork()
-  // The destructor of this sentry wakes up other clients
-  ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid);
-
-  while(true) {
-    if(!this->sync_.getPending(fname, mtype, rseqid)) {
-      iprot_->readMessageBegin(fname, mtype, rseqid);
-    }
-    if(seqid == rseqid) {
-      if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
-        ::apache::thrift::TApplicationException x;
-        x.read(iprot_);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-        sentry.commit();
-        throw x;
-      }
-      if (mtype != ::apache::thrift::protocol::T_REPLY) {
-        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-      }
-      if (fname.compare("fetchAll") != 0) {
-        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
-        iprot_->readMessageEnd();
-        iprot_->getTransport()->readEnd();
-
-        // in a bad state, don't commit
-        using ::apache::thrift::protocol::TProtocolException;
-        throw TProtocolException(TProtocolException::INVALID_DATA);
-      }
-      ThriftHive_fetchAll_presult result;
-      result.success = &_return;
-      result.read(iprot_);
-      iprot_->readMessageEnd();
-      iprot_->getTransport

<TRUNCATED>

[04/20] hive git commit: HIVE-13585: Add counter metric for direct sql failures (Mohit Sabharwal, reviewed by Aihua Xu, Sergey Shelukhin)

Posted by jd...@apache.org.
HIVE-13585: Add counter metric for direct sql failures (Mohit Sabharwal, reviewed by Aihua Xu, Sergey Shelukhin)


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

Branch: refs/heads/llap
Commit: 815499af9543687948b0330c1f8793bfbf2dea67
Parents: 972bcba
Author: Aihua Xu <ax...@cloudera.com>
Authored: Tue Apr 26 19:31:40 2016 -0400
Committer: Aihua Xu <ax...@cloudera.com>
Committed: Tue Apr 26 19:31:40 2016 -0400

----------------------------------------------------------------------
 .../common/metrics/common/MetricsConstant.java  |  2 +
 metastore/pom.xml                               |  8 ++++
 .../hadoop/hive/metastore/ObjectStore.java      | 19 +++++++-
 .../hadoop/hive/metastore/TestObjectStore.java  | 50 ++++++++++++++++++++
 4 files changed, 77 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/815499af/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
index 65b914c..b0d2b85 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
@@ -55,4 +55,6 @@ public class MetricsConstant {
   public static final String DELETE_TOTAL_DATABASES = "delete_total_count_dbs";
   public static final String DELETE_TOTAL_TABLES = "delete_total_count_tables";
   public static final String DELETE_TOTAL_PARTITIONS = "delete_total_count_partitions";
+
+  public static final String DIRECTSQL_ERRORS = "directsql_errors";
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/815499af/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index 18c1f9c..8816829 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -207,6 +207,14 @@
       <artifactId>tephra-hbase-compat-1.0</artifactId>
       <version>${tephra.version}</version>
     </dependency>
+    <!-- test intra-project -->
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-common</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
     <!-- test inter-project -->
     <dependency>
       <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/815499af/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 24fbf70..f651a13 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -63,6 +63,9 @@ import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
@@ -2556,7 +2559,8 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   /** Helper class for getting stuff w/transaction, direct SQL, perf logging, etc. */
-  private abstract class GetHelper<T> {
+  @VisibleForTesting
+  public abstract class GetHelper<T> {
     private final boolean isInTxn, doTrace, allowJdo;
     private boolean doUseDirectSql;
     private long start;
@@ -2668,6 +2672,16 @@ public class ObjectStore implements RawStore, Configurable {
       } else {
         start = doTrace ? System.nanoTime() : 0;
       }
+
+      Metrics metrics = MetricsFactory.getInstance();
+      if (metrics != null) {
+        try {
+          metrics.incrementCounter(MetricsConstant.DIRECTSQL_ERRORS);
+        } catch (Exception e) {
+          LOG.warn("Error reporting Direct SQL errors to metrics system", e);
+        }
+      }
+
       doUseDirectSql = false;
     }
 
@@ -2707,7 +2721,8 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
-  private abstract class GetDbHelper extends GetHelper<Database> {
+  @VisibleForTesting
+  public abstract class GetDbHelper extends GetHelper<Database> {
     /**
      * GetHelper for returning db info using directSql/JDO.
      * Since this is a db-level call, tblName is ignored, and null is passed irrespective of what is passed in.

http://git-wip-us.apache.org/repos/asf/hive/blob/815499af/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 2e1f5f4..6cb062a 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -21,6 +21,11 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
+import org.apache.hadoop.hive.common.metrics.metrics2.CodahaleMetrics;
+import org.apache.hadoop.hive.common.metrics.metrics2.MetricsReporting;
+import org.apache.hadoop.hive.common.metrics.MetricsTestUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -244,6 +249,51 @@ public class TestObjectStore {
     objectStore.removeRole(ROLE1);
   }
 
+  @Test
+  public void testDirectSqlErrorMetrics() throws Exception {
+    HiveConf conf = new HiveConf();
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, true);
+    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_REPORTER, MetricsReporting.JSON_FILE.name()
+        + "," + MetricsReporting.JMX.name());
+
+    MetricsFactory.init(conf);
+    CodahaleMetrics metrics = (CodahaleMetrics) MetricsFactory.getInstance();
+
+    objectStore.new GetDbHelper("foo", null, true, true) {
+      @Override
+      protected Database getSqlResult(ObjectStore.GetHelper<Database> ctx) throws MetaException {
+        return null;
+      }
+
+      @Override
+      protected Database getJdoResult(ObjectStore.GetHelper<Database> ctx) throws MetaException,
+          NoSuchObjectException {
+        return null;
+      }
+    }.run(false);
+
+    String json = metrics.dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.COUNTER,
+        MetricsConstant.DIRECTSQL_ERRORS, "");
+
+    objectStore.new GetDbHelper("foo", null, true, true) {
+      @Override
+      protected Database getSqlResult(ObjectStore.GetHelper<Database> ctx) throws MetaException {
+        throw new RuntimeException();
+      }
+
+      @Override
+      protected Database getJdoResult(ObjectStore.GetHelper<Database> ctx) throws MetaException,
+          NoSuchObjectException {
+        return null;
+      }
+    }.run(false);
+
+    json = metrics.dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.COUNTER,
+        MetricsConstant.DIRECTSQL_ERRORS, 1);
+  }
+
   public static void dropAllStoreObjects(RawStore store) throws MetaException, InvalidObjectException, InvalidInputException {
     try {
       Deadline.registerIfNot(100000);


[02/20] hive git commit: HIVE-13463 : Fix ImportSemanticAnalyzer to allow for different src/dst filesystems (Zach York, reviewed by Sergey Shelukhin)

Posted by jd...@apache.org.
HIVE-13463 : Fix ImportSemanticAnalyzer to allow for different src/dst filesystems (Zach York, reviewed by Sergey Shelukhin)


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

Branch: refs/heads/llap
Commit: 85ffd22af8ab460f545b137aff0592984aa4b4f7
Parents: 1548501
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Apr 26 15:17:57 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Apr 26 15:17:57 2016 -0700

----------------------------------------------------------------------
 .../hive/ql/parse/ImportSemanticAnalyzer.java   | 25 ++++++++++++--------
 1 file changed, 15 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/85ffd22a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index fbf1a3c..500c7ed 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@ -318,9 +318,9 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
     return tblDesc;
   }
 
-  private Task<?> loadTable(URI fromURI, Table table, boolean replace) {
+  private Task<?> loadTable(URI fromURI, Table table, boolean replace, Path tgtPath) {
     Path dataPath = new Path(fromURI.toString(), "data");
-    Path tmpPath = ctx.getExternalTmpPath(new Path(fromURI));
+    Path tmpPath = ctx.getExternalTmpPath(tgtPath);
     Task<?> copyTask = TaskFactory.get(new CopyWork(dataPath,
        tmpPath, false), conf);
     LoadTableDesc loadTableWork = new LoadTableDesc(tmpPath,
@@ -390,7 +390,8 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       LOG.debug("adding dependent CopyWork/AddPart/MoveWork for partition "
           + partSpecToString(partSpec.getPartSpec())
           + " with source location: " + srcLocation);
-      Path tmpPath = ctx.getExternalTmpPath(new Path(fromURI));
+      Path tgtLocation = new Path(partSpec.getLocation());
+      Path tmpPath = ctx.getExternalTmpPath(tgtLocation);
       Task<?> copyTask = TaskFactory.get(new CopyWork(new Path(srcLocation),
           tmpPath, false), conf);
       Task<?> addPartTask = TaskFactory.get(new DDLWork(getInputs(),
@@ -431,7 +432,8 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       tgtPath = new Path(tblDesc.getLocation(),
           Warehouse.makePartPath(partSpec.getPartSpec()));
     }
-    checkTargetLocationEmpty(fs, tgtPath, replicationSpec);
+    FileSystem tgtFs = FileSystem.get(tgtPath.toUri(), conf);
+    checkTargetLocationEmpty(tgtFs, tgtPath, replicationSpec);
     partSpec.setLocation(tgtPath.toString());
   }
 
@@ -707,8 +709,10 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       } else {
         LOG.debug("table non-partitioned");
         // ensure if destination is not empty only for regular import
-        checkTargetLocationEmpty(fs, new Path(table.getDataLocation().toString()), replicationSpec);
-        loadTable(fromURI, table, false);
+        Path tgtPath = new Path(table.getDataLocation().toString());
+        FileSystem tgtFs = FileSystem.get(tgtPath.toUri(), conf);
+        checkTargetLocationEmpty(tgtFs, tgtPath, replicationSpec);
+        loadTable(fromURI, table, false, tgtPath);
       }
       // Set this to read because we can't overwrite any existing partitions
       outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_NO_LOCK));
@@ -741,8 +745,9 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
           } else {
             tablePath = wh.getTablePath(parentDb, tblDesc.getTableName());
           }
-          checkTargetLocationEmpty(fs, tablePath, replicationSpec);
-          t.addDependentTask(loadTable(fromURI, table, false));
+          FileSystem tgtFs = FileSystem.get(tablePath.toUri(), conf);
+          checkTargetLocationEmpty(tgtFs, tablePath, replicationSpec);
+          t.addDependentTask(loadTable(fromURI, table, false, tablePath));
         }
       }
       rootTasks.add(t);
@@ -813,7 +818,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
           }
         } else {
           LOG.debug("adding dependent CopyWork/MoveWork for table");
-          t.addDependentTask(loadTable(fromURI, table, true));
+          t.addDependentTask(loadTable(fromURI, table, true, new Path(tblDesc.getLocation())));
         }
       }
       if (dr == null){
@@ -868,7 +873,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
           return; // silently return, table is newer than our replacement.
         }
         if (!replicationSpec.isMetadataOnly()) {
-          loadTable(fromURI, table, true); // repl-imports are replace-into
+          loadTable(fromURI, table, true, new Path(fromURI)); // repl-imports are replace-into
         } else {
           rootTasks.add(alterTableTask(tblDesc));
         }


[18/20] hive git commit: HIVE-13609: Fix UDTFs to allow local fetch task to fetch rows forwarded by GenericUDTF.close()

Posted by jd...@apache.org.
HIVE-13609: Fix UDTFs to allow local fetch task to fetch rows forwarded by GenericUDTF.close()


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

Branch: refs/heads/llap
Commit: 8bdf618f50bebd21580e5cce336540c33fe47d2b
Parents: 2fe47dc
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Apr 28 10:13:19 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Apr 28 10:13:19 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FetchOperator.java      |  16 +-
 .../apache/hadoop/hive/ql/exec/FetchTask.java   |   4 +
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |   2 +-
 .../clientpositive/lateral_view_noalias.q.out   | 120 ++++-------
 .../results/clientpositive/nonmr_fetch.q.out    |  78 +++----
 .../clientpositive/select_dummy_source.q.out    |  38 ++--
 .../clientpositive/tez/explainuser_1.q.out      |  14 +-
 .../clientpositive/tez/explainuser_3.q.out      |  38 ++--
 .../tez/select_dummy_source.q.out               |  76 ++-----
 .../results/clientpositive/udf_explode.q.out    | 208 +++----------------
 .../results/clientpositive/udf_inline.q.out     |  42 ++--
 .../results/clientpositive/udtf_explode.q.out   | 150 +++----------
 12 files changed, 221 insertions(+), 565 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
index a5489a9..d8ac6ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
@@ -410,6 +410,10 @@ public class FetchOperator implements Serializable {
    * Currently only used by FetchTask.
    **/
   public boolean pushRow() throws IOException, HiveException {
+    if (operator == null) {
+      return false;
+    }
+
     if (work.getRowsComputedUsingStats() != null) {
       for (List<Object> row : work.getRowsComputedUsingStats()) {
         operator.process(row, 0);
@@ -524,10 +528,7 @@ public class FetchOperator implements Serializable {
         currRecReader.close();
         currRecReader = null;
       }
-      if (operator != null) {
-        operator.close(false);
-        operator = null;
-      }
+      closeOperator();
       if (context != null) {
         context.clear();
         context = null;
@@ -542,6 +543,13 @@ public class FetchOperator implements Serializable {
     }
   }
 
+  public void closeOperator() throws HiveException {
+    if (operator != null) {
+      operator.close(false);
+      operator = null;
+    }
+  }
+
   /**
    * used for bucket map join
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
index ec9e98e..dff1815 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
@@ -145,6 +145,10 @@ public class FetchTask extends Task<FetchWork> implements Serializable {
           if (work.getLeastNumRows() > 0) {
             throw new CommandNeedRetryException();
           }
+
+          // Closing the operator can sometimes yield more rows (HIVE-11892)
+          fetch.closeOperator();
+
           return fetched;
         }
         fetched = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
index b5ceb14..eb0ba7b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
@@ -289,7 +289,7 @@ public class SimpleFetchOptimizer extends Transform {
 
   private boolean isConvertible(FetchData fetch, Operator<?> operator, Set<Operator<?>> traversed) {
     if (operator instanceof ReduceSinkOperator || operator instanceof CommonJoinOperator
-        || operator instanceof ScriptOperator || operator instanceof UDTFOperator) {
+        || operator instanceof ScriptOperator) {
       return false;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/lateral_view_noalias.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lateral_view_noalias.q.out b/ql/src/test/results/clientpositive/lateral_view_noalias.q.out
index 3604049..3532737 100644
--- a/ql/src/test/results/clientpositive/lateral_view_noalias.q.out
+++ b/ql/src/test/results/clientpositive/lateral_view_noalias.q.out
@@ -5,20 +5,38 @@ POSTHOOK: query: --HIVE-2608 Do not require AS a,b,c part in LATERAL VIEW
 EXPLAIN SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
+  Stage: Stage-0
+    Fetch Operator
+      limit: 2
+      Processor Tree:
+        TableScan
+          alias: src
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+          Lateral View Forward
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
-            Lateral View Forward
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                Statistics: Num rows: 500 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              Statistics: Num rows: 500 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
+              Lateral View Join Operator
+                outputColumnNames: _col5, _col6
+                Statistics: Num rows: 1000 Data size: 364000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col5 (type: string), _col6 (type: int)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1000 Data size: 364000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Limit
+                    Number of rows: 2
+                    Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    ListSink
+            Select Operator
+              expressions: map('key1':100,'key2':200) (type: map<string,int>)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
+              UDTF Operator
+                Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
+                function name: explode
                 Lateral View Join Operator
                   outputColumnNames: _col5, _col6
                   Statistics: Num rows: 1000 Data size: 364000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -29,43 +47,7 @@ STAGE PLANS:
                     Limit
                       Number of rows: 2
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                      File Output Operator
-                        compressed: false
-                        Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                        table:
-                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              Select Operator
-                expressions: map('key1':100,'key2':200) (type: map<string,int>)
-                outputColumnNames: _col0
-                Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
-                UDTF Operator
-                  Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
-                  function name: explode
-                  Lateral View Join Operator
-                    outputColumnNames: _col5, _col6
-                    Statistics: Num rows: 1000 Data size: 364000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: _col5 (type: string), _col6 (type: int)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1000 Data size: 364000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Limit
-                        Number of rows: 2
-                        Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                        File Output Operator
-                          compressed: false
-                          Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                          table:
-                              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: 2
-      Processor Tree:
-        ListSink
+                      ListSink
 
 PREHOOK: query: SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2
 PREHOOK: type: QUERY
@@ -82,39 +64,27 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN SELECT explode(map('key1', 100, 'key2', 200)) from src limit 2
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: map('key1':100,'key2':200) (type: map<string,int>)
-              outputColumnNames: _col0
-              Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
-              UDTF Operator
-                Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
-                function name: explode
-                Limit
-                  Number of rows: 2
-                  Statistics: Num rows: 2 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 2 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
   Stage: Stage-0
     Fetch Operator
       limit: 2
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: map('key1':100,'key2':200) (type: map<string,int>)
+            outputColumnNames: _col0
+            Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
+            UDTF Operator
+              Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
+              function name: explode
+              Limit
+                Number of rows: 2
+                Statistics: Num rows: 2 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE
+                ListSink
 
 PREHOOK: query: SELECT explode(map('key1', 100, 'key2', 200)) from src limit 2
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/nonmr_fetch.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/nonmr_fetch.q.out b/ql/src/test/results/clientpositive/nonmr_fetch.q.out
index 73bbdf5..8a3c2a7 100644
--- a/ql/src/test/results/clientpositive/nonmr_fetch.q.out
+++ b/ql/src/test/results/clientpositive/nonmr_fetch.q.out
@@ -841,22 +841,40 @@ explain
 select key,X from srcpart lateral view explode(array(key,value)) L as x where (ds='2008-04-08' AND hr='11') limit 20
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: srcpart
+  Stage: Stage-0
+    Fetch Operator
+      limit: 20
+      Processor Tree:
+        TableScan
+          alias: srcpart
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          Lateral View Forward
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-            Lateral View Forward
+            Select Operator
+              expressions: key (type: string)
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: key
+              Lateral View Join Operator
+                outputColumnNames: _col0, _col7
+                Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string), _col7 (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                  Limit
+                    Number of rows: 20
+                    Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
+                    ListSink
+            Select Operator
+              expressions: array(key,value) (type: array<string>)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              UDTF Operator
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                function name: explode
                 Lateral View Join Operator
                   outputColumnNames: _col0, _col7
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -867,43 +885,7 @@ STAGE PLANS:
                     Limit
                       Number of rows: 20
                       Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
-                      File Output Operator
-                        compressed: false
-                        Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
-                        table:
-                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              Select Operator
-                expressions: array(key,value) (type: array<string>)
-                outputColumnNames: _col0
-                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                UDTF Operator
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                  function name: explode
-                  Lateral View Join Operator
-                    outputColumnNames: _col0, _col7
-                    Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: _col0 (type: string), _col7 (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
-                      Limit
-                        Number of rows: 20
-                        Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
-                        File Output Operator
-                          compressed: false
-                          Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
-                          table:
-                              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: 20
-      Processor Tree:
-        ListSink
+                      ListSink
 
 PREHOOK: query: select key,X from srcpart lateral view explode(array(key,value)) L as x where (ds='2008-04-08' AND hr='11') limit 20
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/select_dummy_source.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/select_dummy_source.q.out b/ql/src/test/results/clientpositive/select_dummy_source.q.out
index 86c9d3c..a23c8ad 100644
--- a/ql/src/test/results/clientpositive/select_dummy_source.q.out
+++ b/ql/src/test/results/clientpositive/select_dummy_source.q.out
@@ -190,37 +190,25 @@ POSTHOOK: query: explain
 select explode(array('a', 'b'))
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: _dummy_table
-            Row Limit Per Split: 1
-            Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: array('a','b') (type: array<string>)
-              outputColumnNames: _col0
-              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-              UDTF Operator
-                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                function name: explode
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
   Stage: Stage-0
     Fetch Operator
       limit: -1
       Processor Tree:
-        ListSink
+        TableScan
+          alias: _dummy_table
+          Row Limit Per Split: 1
+          Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: array('a','b') (type: array<string>)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+            UDTF Operator
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              function name: explode
+              ListSink
 
 PREHOOK: query: select explode(array('a', 'b'))
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
index c70f104..bdb8830 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
@@ -5371,15 +5371,11 @@ Plan not optimized by CBO due to missing feature [Others].
 Stage-0
   Fetch Operator
     limit:-1
-    Stage-1
-      Map 1
-      File Output Operator [FS_3]
-        UDTF Operator [UDTF_2] (rows=1 width=0)
-          function name:explode
-          Select Operator [SEL_1] (rows=1 width=0)
-            Output:["_col0"]
-            TableScan [TS_0] (rows=1 width=1)
-              _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE
+    UDTF Operator [UDTF_2]
+      function name:explode
+      Select Operator [SEL_1]
+        Output:["_col0"]
+        TableScan [TS_0]
 
 PREHOOK: query: CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
index f4e21bd..f444058 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
@@ -51,28 +51,24 @@ Plan not optimized by CBO.
 Stage-0
   Fetch Operator
     limit:-1
-    Stage-1
-      Map 1
-      File Output Operator [FS_7]
-        Select Operator [SEL_6] (rows=4000 width=10)
-          Output:["_col0","_col1"]
-          Lateral View Join Operator [LVJ_5] (rows=4000 width=10)
-            Output:["_col0","_col1","_col7"]
-            Select Operator [SEL_2] (rows=2000 width=10)
+    Select Operator [SEL_6]
+      Output:["_col0","_col1"]
+      Lateral View Join Operator [LVJ_5]
+        Output:["_col0","_col1","_col7"]
+        Select Operator [SEL_2]
+          Output:["key","value"]
+          Lateral View Forward [LVF_1]
+            TableScan [TS_0]
               Output:["key","value"]
-              Lateral View Forward [LVF_1] (rows=2000 width=10)
-                TableScan [TS_0] (rows=2000 width=10)
-                  default@srcpart,srcpart,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-      File Output Operator [FS_7]
-        Select Operator [SEL_6] (rows=4000 width=10)
-          Output:["_col0","_col1"]
-          Lateral View Join Operator [LVJ_5] (rows=4000 width=10)
-            Output:["_col0","_col1","_col7"]
-            UDTF Operator [UDTF_4] (rows=2000 width=10)
-              function name:explode
-              Select Operator [SEL_3] (rows=2000 width=10)
-                Output:["_col0"]
-                 Please refer to the previous Lateral View Forward [LVF_1]
+    Select Operator [SEL_6]
+      Output:["_col0","_col1"]
+      Lateral View Join Operator [LVJ_5]
+        Output:["_col0","_col1","_col7"]
+        UDTF Operator [UDTF_4]
+          function name:explode
+          Select Operator [SEL_3]
+            Output:["_col0"]
+             Please refer to the previous Lateral View Forward [LVF_1]
 
 PREHOOK: query: explain show tables
 PREHOOK: type: SHOWTABLES

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/tez/select_dummy_source.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/select_dummy_source.q.out b/ql/src/test/results/clientpositive/tez/select_dummy_source.q.out
index 4f34611..fa99b76 100644
--- a/ql/src/test/results/clientpositive/tez/select_dummy_source.q.out
+++ b/ql/src/test/results/clientpositive/tez/select_dummy_source.q.out
@@ -71,40 +71,22 @@ explain
 select explode(array('a', 'b'))
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Tez
-#### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: _dummy_table
-                  Row Limit Per Split: 1
-                  Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: array('a','b') (type: array<string>)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                    UDTF Operator
-                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                      function name: explode
-                      File Output Operator
-                        compressed: false
-                        Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                        table:
-                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
   Stage: Stage-0
     Fetch Operator
       limit: -1
       Processor Tree:
-        ListSink
+        TableScan
+          alias: _dummy_table
+          Row Limit Per Split: 1
+          Select Operator
+            expressions: array('a','b') (type: array<string>)
+            outputColumnNames: _col0
+            UDTF Operator
+              function name: explode
+              ListSink
 
 PREHOOK: query: select explode(array('a', 'b'))
 PREHOOK: type: QUERY
@@ -185,40 +167,22 @@ POSTHOOK: query: explain
 select explode(array('a', 'b'))
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Tez
-#### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: _dummy_table
-                  Row Limit Per Split: 1
-                  Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: array('a','b') (type: array<string>)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                    UDTF Operator
-                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                      function name: explode
-                      File Output Operator
-                        compressed: false
-                        Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-                        table:
-                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
   Stage: Stage-0
     Fetch Operator
       limit: -1
       Processor Tree:
-        ListSink
+        TableScan
+          alias: _dummy_table
+          Row Limit Per Split: 1
+          Select Operator
+            expressions: array('a','b') (type: array<string>)
+            outputColumnNames: _col0
+            UDTF Operator
+              function name: explode
+              ListSink
 
 PREHOOK: query: select explode(array('a', 'b'))
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/udf_explode.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_explode.q.out b/ql/src/test/results/clientpositive/udf_explode.q.out
index 8db4375..ea12e80 100644
--- a/ql/src/test/results/clientpositive/udf_explode.q.out
+++ b/ql/src/test/results/clientpositive/udf_explode.q.out
@@ -13,102 +13,26 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN EXTENDED SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Row Limit Per Split: 1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
-            GatherStats: false
-            Select Operator
-              expressions: array(1,2,3) (type: array<int>)
-              outputColumnNames: _col0
-              Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
-              UDTF Operator
-                Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
-                function name: explode
-                File Output Operator
-                  compressed: false
-                  GlobalTableId: 0
-#### A masked pattern was here ####
-                  NumFilesPerFileSink: 1
-                  Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
-#### A masked pattern was here ####
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      properties:
-                        columns col
-                        columns.types int
-                        escape.delim \
-                        hive.serialization.extend.additional.nesting.levels true
-                        serialization.escape.crlf true
-                        serialization.format 1
-                        serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  TotalFiles: 1
-                  GatherStats: false
-                  MultiFileSpray: false
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: src
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            properties:
-              COLUMN_STATS_ACCURATE {"COLUMN_STATS":{"key":"true","value":"true"},"BASIC_STATS":"true"}
-              bucket_count -1
-              columns key,value
-              columns.comments 'default','default'
-              columns.types string:string
-#### A masked pattern was here ####
-              name default.src
-              numFiles 1
-              numRows 500
-              rawDataSize 5312
-              serialization.ddl struct src { string key, string value}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 5812
-#### A masked pattern was here ####
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                COLUMN_STATS_ACCURATE {"COLUMN_STATS":{"key":"true","value":"true"},"BASIC_STATS":"true"}
-                bucket_count -1
-                columns key,value
-                columns.comments 'default','default'
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.src
-                numFiles 1
-                numRows 500
-                rawDataSize 5312
-                serialization.ddl struct src { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 5812
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.src
-            name: default.src
-      Truncated Path -> Alias:
-        /src [src]
-
   Stage: Stage-0
     Fetch Operator
       limit: -1
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Row Limit Per Split: 1
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+          GatherStats: false
+          Select Operator
+            expressions: array(1,2,3) (type: array<int>)
+            outputColumnNames: _col0
+            Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
+            UDTF Operator
+              Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
+              function name: explode
+              ListSink
 
 PREHOOK: query: EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows)) a GROUP BY a.myCol
 PREHOOK: type: QUERY
@@ -273,102 +197,26 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN EXTENDED SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Row Limit Per Split: 1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
-            GatherStats: false
-            Select Operator
-              expressions: map(1:'one',2:'two',3:'three') (type: map<int,string>)
-              outputColumnNames: _col0
-              Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
-              UDTF Operator
-                Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
-                function name: explode
-                File Output Operator
-                  compressed: false
-                  GlobalTableId: 0
-#### A masked pattern was here ####
-                  NumFilesPerFileSink: 1
-                  Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
-#### A masked pattern was here ####
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      properties:
-                        columns key,value
-                        columns.types int:string
-                        escape.delim \
-                        hive.serialization.extend.additional.nesting.levels true
-                        serialization.escape.crlf true
-                        serialization.format 1
-                        serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  TotalFiles: 1
-                  GatherStats: false
-                  MultiFileSpray: false
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: src
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            properties:
-              COLUMN_STATS_ACCURATE {"COLUMN_STATS":{"key":"true","value":"true"},"BASIC_STATS":"true"}
-              bucket_count -1
-              columns key,value
-              columns.comments 'default','default'
-              columns.types string:string
-#### A masked pattern was here ####
-              name default.src
-              numFiles 1
-              numRows 500
-              rawDataSize 5312
-              serialization.ddl struct src { string key, string value}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 5812
-#### A masked pattern was here ####
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                COLUMN_STATS_ACCURATE {"COLUMN_STATS":{"key":"true","value":"true"},"BASIC_STATS":"true"}
-                bucket_count -1
-                columns key,value
-                columns.comments 'default','default'
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.src
-                numFiles 1
-                numRows 500
-                rawDataSize 5312
-                serialization.ddl struct src { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 5812
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.src
-            name: default.src
-      Truncated Path -> Alias:
-        /src [src]
-
   Stage: Stage-0
     Fetch Operator
       limit: -1
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Row Limit Per Split: 1
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+          GatherStats: false
+          Select Operator
+            expressions: map(1:'one',2:'two',3:'three') (type: map<int,string>)
+            outputColumnNames: _col0
+            Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
+            UDTF Operator
+              Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
+              function name: explode
+              ListSink
 
 PREHOOK: query: EXPLAIN EXTENDED SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows)) a GROUP BY a.key, a.val
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/udf_inline.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_inline.q.out b/ql/src/test/results/clientpositive/udf_inline.q.out
index 2c5268e..f986abf 100644
--- a/ql/src/test/results/clientpositive/udf_inline.q.out
+++ b/ql/src/test/results/clientpositive/udf_inline.q.out
@@ -20,39 +20,27 @@ POSTHOOK: query: explain SELECT inline(
 )  as (id, text) FROM SRC limit 2
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: array(const struct(1,'dude!'),const struct(2,'Wheres'),const struct(3,'my car?')) (type: array<struct<col1:int,col2:string>>)
-              outputColumnNames: _col0
-              Statistics: Num rows: 500 Data size: 32000 Basic stats: COMPLETE Column stats: COMPLETE
-              UDTF Operator
-                Statistics: Num rows: 500 Data size: 32000 Basic stats: COMPLETE Column stats: COMPLETE
-                function name: inline
-                Limit
-                  Number of rows: 2
-                  Statistics: Num rows: 2 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 2 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
   Stage: Stage-0
     Fetch Operator
       limit: 2
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: array(const struct(1,'dude!'),const struct(2,'Wheres'),const struct(3,'my car?')) (type: array<struct<col1:int,col2:string>>)
+            outputColumnNames: _col0
+            Statistics: Num rows: 500 Data size: 32000 Basic stats: COMPLETE Column stats: COMPLETE
+            UDTF Operator
+              Statistics: Num rows: 500 Data size: 32000 Basic stats: COMPLETE Column stats: COMPLETE
+              function name: inline
+              Limit
+                Number of rows: 2
+                Statistics: Num rows: 2 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
+                ListSink
 
 PREHOOK: query: SELECT inline( 
   ARRAY(

http://git-wip-us.apache.org/repos/asf/hive/blob/8bdf618f/ql/src/test/results/clientpositive/udtf_explode.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udtf_explode.q.out b/ql/src/test/results/clientpositive/udtf_explode.q.out
index 3304109..e067a0a 100644
--- a/ql/src/test/results/clientpositive/udtf_explode.q.out
+++ b/ql/src/test/results/clientpositive/udtf_explode.q.out
@@ -13,104 +13,28 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN EXTENDED SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
-            GatherStats: false
-            Select Operator
-              expressions: array(1,2,3) (type: array<int>)
-              outputColumnNames: _col0
-              Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
-              UDTF Operator
-                Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
-                function name: explode
-                Limit
-                  Number of rows: 3
-                  Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
-                  File Output Operator
-                    compressed: false
-                    GlobalTableId: 0
-#### A masked pattern was here ####
-                    NumFilesPerFileSink: 1
-                    Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
-#### A masked pattern was here ####
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        properties:
-                          columns col
-                          columns.types int
-                          escape.delim \
-                          hive.serialization.extend.additional.nesting.levels true
-                          serialization.escape.crlf true
-                          serialization.format 1
-                          serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    TotalFiles: 1
-                    GatherStats: false
-                    MultiFileSpray: false
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: src
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            properties:
-              COLUMN_STATS_ACCURATE {"COLUMN_STATS":{"key":"true","value":"true"},"BASIC_STATS":"true"}
-              bucket_count -1
-              columns key,value
-              columns.comments 'default','default'
-              columns.types string:string
-#### A masked pattern was here ####
-              name default.src
-              numFiles 1
-              numRows 500
-              rawDataSize 5312
-              serialization.ddl struct src { string key, string value}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 5812
-#### A masked pattern was here ####
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                COLUMN_STATS_ACCURATE {"COLUMN_STATS":{"key":"true","value":"true"},"BASIC_STATS":"true"}
-                bucket_count -1
-                columns key,value
-                columns.comments 'default','default'
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.src
-                numFiles 1
-                numRows 500
-                rawDataSize 5312
-                serialization.ddl struct src { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 5812
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.src
-            name: default.src
-      Truncated Path -> Alias:
-        /src [src]
-
   Stage: Stage-0
     Fetch Operator
       limit: 3
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+          GatherStats: false
+          Select Operator
+            expressions: array(1,2,3) (type: array<int>)
+            outputColumnNames: _col0
+            Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
+            UDTF Operator
+              Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
+              function name: explode
+              Limit
+                Number of rows: 3
+                Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                ListSink
 
 PREHOOK: query: EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3) a GROUP BY a.myCol
 PREHOOK: type: QUERY
@@ -342,39 +266,27 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src LIMIT 3
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: map(1:'one',2:'two',3:'three') (type: map<int,string>)
-              outputColumnNames: _col0
-              Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
-              UDTF Operator
-                Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
-                function name: explode
-                Limit
-                  Number of rows: 3
-                  Statistics: Num rows: 3 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 3 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
   Stage: Stage-0
     Fetch Operator
       limit: 3
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: map(1:'one',2:'two',3:'three') (type: map<int,string>)
+            outputColumnNames: _col0
+            Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
+            UDTF Operator
+              Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
+              function name: explode
+              Limit
+                Number of rows: 3
+                Statistics: Num rows: 3 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
+                ListSink
 
 PREHOOK: query: EXPLAIN EXTENDED SELECT a.myKey, a.myVal, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src LIMIT 3) a GROUP BY a.myKey, a.myVal
 PREHOOK: type: QUERY


[07/20] hive git commit: HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-php/ThriftHive.php
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-php/ThriftHive.php b/service/src/gen/thrift/gen-php/ThriftHive.php
deleted file mode 100644
index 23dc8fd..0000000
--- a/service/src/gen/thrift/gen-php/ThriftHive.php
+++ /dev/null
@@ -1,1943 +0,0 @@
-<?php
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-use Thrift\Base\TBase;
-use Thrift\Type\TType;
-use Thrift\Type\TMessageType;
-use Thrift\Exception\TException;
-use Thrift\Exception\TProtocolException;
-use Thrift\Protocol\TProtocol;
-use Thrift\Protocol\TBinaryProtocolAccelerated;
-use Thrift\Exception\TApplicationException;
-
-
-interface ThriftHiveIf extends \metastore\ThriftHiveMetastoreIf {
-  /**
-   * @param string $query
-   * @throws \HiveServerException
-   */
-  public function execute($query);
-  /**
-   * @return string
-   * @throws \HiveServerException
-   */
-  public function fetchOne();
-  /**
-   * @param int $numRows
-   * @return string[]
-   * @throws \HiveServerException
-   */
-  public function fetchN($numRows);
-  /**
-   * @return string[]
-   * @throws \HiveServerException
-   */
-  public function fetchAll();
-  /**
-   * @return \metastore\Schema
-   * @throws \HiveServerException
-   */
-  public function getSchema();
-  /**
-   * @return \metastore\Schema
-   * @throws \HiveServerException
-   */
-  public function getThriftSchema();
-  /**
-   * @return \HiveClusterStatus
-   * @throws \HiveServerException
-   */
-  public function getClusterStatus();
-  /**
-   * @return \QueryPlan
-   * @throws \HiveServerException
-   */
-  public function getQueryPlan();
-  /**
-   */
-  public function clean();
-}
-
-class ThriftHiveClient extends \metastore\ThriftHiveMetastoreClient implements \ThriftHiveIf {
-  public function __construct($input, $output=null) {
-    parent::__construct($input, $output);
-  }
-
-  public function execute($query)
-  {
-    $this->send_execute($query);
-    $this->recv_execute();
-  }
-
-  public function send_execute($query)
-  {
-    $args = new \ThriftHive_execute_args();
-    $args->query = $query;
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'execute', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('execute', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_execute()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_execute_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 \ThriftHive_execute_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->ex !== null) {
-      throw $result->ex;
-    }
-    return;
-  }
-
-  public function fetchOne()
-  {
-    $this->send_fetchOne();
-    return $this->recv_fetchOne();
-  }
-
-  public function send_fetchOne()
-  {
-    $args = new \ThriftHive_fetchOne_args();
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'fetchOne', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('fetchOne', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_fetchOne()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_fetchOne_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 \ThriftHive_fetchOne_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    if ($result->ex !== null) {
-      throw $result->ex;
-    }
-    throw new \Exception("fetchOne failed: unknown result");
-  }
-
-  public function fetchN($numRows)
-  {
-    $this->send_fetchN($numRows);
-    return $this->recv_fetchN();
-  }
-
-  public function send_fetchN($numRows)
-  {
-    $args = new \ThriftHive_fetchN_args();
-    $args->numRows = $numRows;
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'fetchN', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('fetchN', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_fetchN()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_fetchN_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 \ThriftHive_fetchN_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    if ($result->ex !== null) {
-      throw $result->ex;
-    }
-    throw new \Exception("fetchN failed: unknown result");
-  }
-
-  public function fetchAll()
-  {
-    $this->send_fetchAll();
-    return $this->recv_fetchAll();
-  }
-
-  public function send_fetchAll()
-  {
-    $args = new \ThriftHive_fetchAll_args();
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'fetchAll', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('fetchAll', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_fetchAll()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_fetchAll_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 \ThriftHive_fetchAll_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    if ($result->ex !== null) {
-      throw $result->ex;
-    }
-    throw new \Exception("fetchAll failed: unknown result");
-  }
-
-  public function getSchema()
-  {
-    $this->send_getSchema();
-    return $this->recv_getSchema();
-  }
-
-  public function send_getSchema()
-  {
-    $args = new \ThriftHive_getSchema_args();
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'getSchema', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('getSchema', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_getSchema()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_getSchema_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 \ThriftHive_getSchema_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    if ($result->ex !== null) {
-      throw $result->ex;
-    }
-    throw new \Exception("getSchema failed: unknown result");
-  }
-
-  public function getThriftSchema()
-  {
-    $this->send_getThriftSchema();
-    return $this->recv_getThriftSchema();
-  }
-
-  public function send_getThriftSchema()
-  {
-    $args = new \ThriftHive_getThriftSchema_args();
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'getThriftSchema', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('getThriftSchema', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_getThriftSchema()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_getThriftSchema_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 \ThriftHive_getThriftSchema_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    if ($result->ex !== null) {
-      throw $result->ex;
-    }
-    throw new \Exception("getThriftSchema failed: unknown result");
-  }
-
-  public function getClusterStatus()
-  {
-    $this->send_getClusterStatus();
-    return $this->recv_getClusterStatus();
-  }
-
-  public function send_getClusterStatus()
-  {
-    $args = new \ThriftHive_getClusterStatus_args();
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'getClusterStatus', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('getClusterStatus', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_getClusterStatus()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_getClusterStatus_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 \ThriftHive_getClusterStatus_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    if ($result->ex !== null) {
-      throw $result->ex;
-    }
-    throw new \Exception("getClusterStatus failed: unknown result");
-  }
-
-  public function getQueryPlan()
-  {
-    $this->send_getQueryPlan();
-    return $this->recv_getQueryPlan();
-  }
-
-  public function send_getQueryPlan()
-  {
-    $args = new \ThriftHive_getQueryPlan_args();
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'getQueryPlan', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('getQueryPlan', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_getQueryPlan()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_getQueryPlan_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 \ThriftHive_getQueryPlan_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    if ($result->ex !== null) {
-      throw $result->ex;
-    }
-    throw new \Exception("getQueryPlan failed: unknown result");
-  }
-
-  public function clean()
-  {
-    $this->send_clean();
-    $this->recv_clean();
-  }
-
-  public function send_clean()
-  {
-    $args = new \ThriftHive_clean_args();
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'clean', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('clean', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_clean()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\ThriftHive_clean_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 \ThriftHive_clean_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    return;
-  }
-
-}
-
-// HELPER FUNCTIONS AND STRUCTURES
-
-class ThriftHive_execute_args {
-  static $_TSPEC;
-
-  /**
-   * @var string
-   */
-  public $query = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'query',
-          'type' => TType::STRING,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['query'])) {
-        $this->query = $vals['query'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_execute_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->query);
-          } 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('ThriftHive_execute_args');
-    if ($this->query !== null) {
-      $xfer += $output->writeFieldBegin('query', TType::STRING, 1);
-      $xfer += $output->writeString($this->query);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_execute_result {
-  static $_TSPEC;
-
-  /**
-   * @var \HiveServerException
-   */
-  public $ex = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'ex',
-          'type' => TType::STRUCT,
-          'class' => '\HiveServerException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['ex'])) {
-        $this->ex = $vals['ex'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_execute_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->ex = new \HiveServerException();
-            $xfer += $this->ex->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('ThriftHive_execute_result');
-    if ($this->ex !== null) {
-      $xfer += $output->writeFieldBegin('ex', TType::STRUCT, 1);
-      $xfer += $this->ex->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_fetchOne_args {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_fetchOne_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)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHive_fetchOne_args');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_fetchOne_result {
-  static $_TSPEC;
-
-  /**
-   * @var string
-   */
-  public $success = null;
-  /**
-   * @var \HiveServerException
-   */
-  public $ex = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::STRING,
-          ),
-        1 => array(
-          'var' => 'ex',
-          'type' => TType::STRUCT,
-          'class' => '\HiveServerException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['ex'])) {
-        $this->ex = $vals['ex'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_fetchOne_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->success);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->ex = new \HiveServerException();
-            $xfer += $this->ex->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('ThriftHive_fetchOne_result');
-    if ($this->success !== null) {
-      $xfer += $output->writeFieldBegin('success', TType::STRING, 0);
-      $xfer += $output->writeString($this->success);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->ex !== null) {
-      $xfer += $output->writeFieldBegin('ex', TType::STRUCT, 1);
-      $xfer += $this->ex->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_fetchN_args {
-  static $_TSPEC;
-
-  /**
-   * @var int
-   */
-  public $numRows = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'numRows',
-          'type' => TType::I32,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['numRows'])) {
-        $this->numRows = $vals['numRows'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_fetchN_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::I32) {
-            $xfer += $input->readI32($this->numRows);
-          } 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('ThriftHive_fetchN_args');
-    if ($this->numRows !== null) {
-      $xfer += $output->writeFieldBegin('numRows', TType::I32, 1);
-      $xfer += $output->writeI32($this->numRows);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_fetchN_result {
-  static $_TSPEC;
-
-  /**
-   * @var string[]
-   */
-  public $success = null;
-  /**
-   * @var \HiveServerException
-   */
-  public $ex = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::LST,
-          'etype' => TType::STRING,
-          'elem' => array(
-            'type' => TType::STRING,
-            ),
-          ),
-        1 => array(
-          'var' => 'ex',
-          'type' => TType::STRUCT,
-          'class' => '\HiveServerException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['ex'])) {
-        $this->ex = $vals['ex'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_fetchN_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::LST) {
-            $this->success = array();
-            $_size0 = 0;
-            $_etype3 = 0;
-            $xfer += $input->readListBegin($_etype3, $_size0);
-            for ($_i4 = 0; $_i4 < $_size0; ++$_i4)
-            {
-              $elem5 = null;
-              $xfer += $input->readString($elem5);
-              $this->success []= $elem5;
-            }
-            $xfer += $input->readListEnd();
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->ex = new \HiveServerException();
-            $xfer += $this->ex->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('ThriftHive_fetchN_result');
-    if ($this->success !== null) {
-      if (!is_array($this->success)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
-      {
-        $output->writeListBegin(TType::STRING, count($this->success));
-        {
-          foreach ($this->success as $iter6)
-          {
-            $xfer += $output->writeString($iter6);
-          }
-        }
-        $output->writeListEnd();
-      }
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->ex !== null) {
-      $xfer += $output->writeFieldBegin('ex', TType::STRUCT, 1);
-      $xfer += $this->ex->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_fetchAll_args {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_fetchAll_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)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHive_fetchAll_args');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_fetchAll_result {
-  static $_TSPEC;
-
-  /**
-   * @var string[]
-   */
-  public $success = null;
-  /**
-   * @var \HiveServerException
-   */
-  public $ex = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::LST,
-          'etype' => TType::STRING,
-          'elem' => array(
-            'type' => TType::STRING,
-            ),
-          ),
-        1 => array(
-          'var' => 'ex',
-          'type' => TType::STRUCT,
-          'class' => '\HiveServerException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['ex'])) {
-        $this->ex = $vals['ex'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_fetchAll_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::LST) {
-            $this->success = array();
-            $_size7 = 0;
-            $_etype10 = 0;
-            $xfer += $input->readListBegin($_etype10, $_size7);
-            for ($_i11 = 0; $_i11 < $_size7; ++$_i11)
-            {
-              $elem12 = null;
-              $xfer += $input->readString($elem12);
-              $this->success []= $elem12;
-            }
-            $xfer += $input->readListEnd();
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->ex = new \HiveServerException();
-            $xfer += $this->ex->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('ThriftHive_fetchAll_result');
-    if ($this->success !== null) {
-      if (!is_array($this->success)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
-      {
-        $output->writeListBegin(TType::STRING, count($this->success));
-        {
-          foreach ($this->success as $iter13)
-          {
-            $xfer += $output->writeString($iter13);
-          }
-        }
-        $output->writeListEnd();
-      }
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->ex !== null) {
-      $xfer += $output->writeFieldBegin('ex', TType::STRUCT, 1);
-      $xfer += $this->ex->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_getSchema_args {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_getSchema_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)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHive_getSchema_args');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_getSchema_result {
-  static $_TSPEC;
-
-  /**
-   * @var \metastore\Schema
-   */
-  public $success = null;
-  /**
-   * @var \HiveServerException
-   */
-  public $ex = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\Schema',
-          ),
-        1 => array(
-          'var' => 'ex',
-          'type' => TType::STRUCT,
-          'class' => '\HiveServerException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['ex'])) {
-        $this->ex = $vals['ex'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_getSchema_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::STRUCT) {
-            $this->success = new \metastore\Schema();
-            $xfer += $this->success->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->ex = new \HiveServerException();
-            $xfer += $this->ex->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('ThriftHive_getSchema_result');
-    if ($this->success !== null) {
-      if (!is_object($this->success)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
-      $xfer += $this->success->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->ex !== null) {
-      $xfer += $output->writeFieldBegin('ex', TType::STRUCT, 1);
-      $xfer += $this->ex->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_getThriftSchema_args {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_getThriftSchema_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)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHive_getThriftSchema_args');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_getThriftSchema_result {
-  static $_TSPEC;
-
-  /**
-   * @var \metastore\Schema
-   */
-  public $success = null;
-  /**
-   * @var \HiveServerException
-   */
-  public $ex = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\Schema',
-          ),
-        1 => array(
-          'var' => 'ex',
-          'type' => TType::STRUCT,
-          'class' => '\HiveServerException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['ex'])) {
-        $this->ex = $vals['ex'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_getThriftSchema_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::STRUCT) {
-            $this->success = new \metastore\Schema();
-            $xfer += $this->success->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->ex = new \HiveServerException();
-            $xfer += $this->ex->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('ThriftHive_getThriftSchema_result');
-    if ($this->success !== null) {
-      if (!is_object($this->success)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
-      $xfer += $this->success->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->ex !== null) {
-      $xfer += $output->writeFieldBegin('ex', TType::STRUCT, 1);
-      $xfer += $this->ex->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_getClusterStatus_args {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_getClusterStatus_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)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHive_getClusterStatus_args');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_getClusterStatus_result {
-  static $_TSPEC;
-
-  /**
-   * @var \HiveClusterStatus
-   */
-  public $success = null;
-  /**
-   * @var \HiveServerException
-   */
-  public $ex = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::STRUCT,
-          'class' => '\HiveClusterStatus',
-          ),
-        1 => array(
-          'var' => 'ex',
-          'type' => TType::STRUCT,
-          'class' => '\HiveServerException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['ex'])) {
-        $this->ex = $vals['ex'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_getClusterStatus_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::STRUCT) {
-            $this->success = new \HiveClusterStatus();
-            $xfer += $this->success->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->ex = new \HiveServerException();
-            $xfer += $this->ex->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('ThriftHive_getClusterStatus_result');
-    if ($this->success !== null) {
-      if (!is_object($this->success)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
-      $xfer += $this->success->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->ex !== null) {
-      $xfer += $output->writeFieldBegin('ex', TType::STRUCT, 1);
-      $xfer += $this->ex->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_getQueryPlan_args {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_getQueryPlan_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)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHive_getQueryPlan_args');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_getQueryPlan_result {
-  static $_TSPEC;
-
-  /**
-   * @var \QueryPlan
-   */
-  public $success = null;
-  /**
-   * @var \HiveServerException
-   */
-  public $ex = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::STRUCT,
-          'class' => '\QueryPlan',
-          ),
-        1 => array(
-          'var' => 'ex',
-          'type' => TType::STRUCT,
-          'class' => '\HiveServerException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['ex'])) {
-        $this->ex = $vals['ex'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_getQueryPlan_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::STRUCT) {
-            $this->success = new \QueryPlan();
-            $xfer += $this->success->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->ex = new \HiveServerException();
-            $xfer += $this->ex->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('ThriftHive_getQueryPlan_result');
-    if ($this->success !== null) {
-      if (!is_object($this->success)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
-      $xfer += $this->success->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->ex !== null) {
-      $xfer += $output->writeFieldBegin('ex', TType::STRUCT, 1);
-      $xfer += $this->ex->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_clean_args {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_clean_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)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHive_clean_args');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHive_clean_result {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHive_clean_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)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHive_clean_result');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-php/Types.php
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-php/Types.php b/service/src/gen/thrift/gen-php/Types.php
deleted file mode 100644
index cbf4c29..0000000
--- a/service/src/gen/thrift/gen-php/Types.php
+++ /dev/null
@@ -1,338 +0,0 @@
-<?php
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-use Thrift\Base\TBase;
-use Thrift\Type\TType;
-use Thrift\Type\TMessageType;
-use Thrift\Exception\TException;
-use Thrift\Exception\TProtocolException;
-use Thrift\Protocol\TProtocol;
-use Thrift\Protocol\TBinaryProtocolAccelerated;
-use Thrift\Exception\TApplicationException;
-
-
-final class JobTrackerState {
-  const INITIALIZING = 1;
-  const RUNNING = 2;
-  static public $__names = array(
-    1 => 'INITIALIZING',
-    2 => 'RUNNING',
-  );
-}
-
-class HiveClusterStatus {
-  static $_TSPEC;
-
-  /**
-   * @var int
-   */
-  public $taskTrackers = null;
-  /**
-   * @var int
-   */
-  public $mapTasks = null;
-  /**
-   * @var int
-   */
-  public $reduceTasks = null;
-  /**
-   * @var int
-   */
-  public $maxMapTasks = null;
-  /**
-   * @var int
-   */
-  public $maxReduceTasks = null;
-  /**
-   * @var int
-   */
-  public $state = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'taskTrackers',
-          'type' => TType::I32,
-          ),
-        2 => array(
-          'var' => 'mapTasks',
-          'type' => TType::I32,
-          ),
-        3 => array(
-          'var' => 'reduceTasks',
-          'type' => TType::I32,
-          ),
-        4 => array(
-          'var' => 'maxMapTasks',
-          'type' => TType::I32,
-          ),
-        5 => array(
-          'var' => 'maxReduceTasks',
-          'type' => TType::I32,
-          ),
-        6 => array(
-          'var' => 'state',
-          'type' => TType::I32,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['taskTrackers'])) {
-        $this->taskTrackers = $vals['taskTrackers'];
-      }
-      if (isset($vals['mapTasks'])) {
-        $this->mapTasks = $vals['mapTasks'];
-      }
-      if (isset($vals['reduceTasks'])) {
-        $this->reduceTasks = $vals['reduceTasks'];
-      }
-      if (isset($vals['maxMapTasks'])) {
-        $this->maxMapTasks = $vals['maxMapTasks'];
-      }
-      if (isset($vals['maxReduceTasks'])) {
-        $this->maxReduceTasks = $vals['maxReduceTasks'];
-      }
-      if (isset($vals['state'])) {
-        $this->state = $vals['state'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'HiveClusterStatus';
-  }
-
-  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::I32) {
-            $xfer += $input->readI32($this->taskTrackers);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->mapTasks);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 3:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->reduceTasks);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 4:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->maxMapTasks);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 5:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->maxReduceTasks);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 6:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->state);
-          } 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('HiveClusterStatus');
-    if ($this->taskTrackers !== null) {
-      $xfer += $output->writeFieldBegin('taskTrackers', TType::I32, 1);
-      $xfer += $output->writeI32($this->taskTrackers);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->mapTasks !== null) {
-      $xfer += $output->writeFieldBegin('mapTasks', TType::I32, 2);
-      $xfer += $output->writeI32($this->mapTasks);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->reduceTasks !== null) {
-      $xfer += $output->writeFieldBegin('reduceTasks', TType::I32, 3);
-      $xfer += $output->writeI32($this->reduceTasks);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->maxMapTasks !== null) {
-      $xfer += $output->writeFieldBegin('maxMapTasks', TType::I32, 4);
-      $xfer += $output->writeI32($this->maxMapTasks);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->maxReduceTasks !== null) {
-      $xfer += $output->writeFieldBegin('maxReduceTasks', TType::I32, 5);
-      $xfer += $output->writeI32($this->maxReduceTasks);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->state !== null) {
-      $xfer += $output->writeFieldBegin('state', TType::I32, 6);
-      $xfer += $output->writeI32($this->state);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class HiveServerException extends TException {
-  static $_TSPEC;
-
-  /**
-   * @var string
-   */
-  public $message = null;
-  /**
-   * @var int
-   */
-  public $errorCode = null;
-  /**
-   * @var string
-   */
-  public $SQLState = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'message',
-          'type' => TType::STRING,
-          ),
-        2 => array(
-          'var' => 'errorCode',
-          'type' => TType::I32,
-          ),
-        3 => array(
-          'var' => 'SQLState',
-          'type' => TType::STRING,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['message'])) {
-        $this->message = $vals['message'];
-      }
-      if (isset($vals['errorCode'])) {
-        $this->errorCode = $vals['errorCode'];
-      }
-      if (isset($vals['SQLState'])) {
-        $this->SQLState = $vals['SQLState'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'HiveServerException';
-  }
-
-  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->message);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->errorCode);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 3:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->SQLState);
-          } 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('HiveServerException');
-    if ($this->message !== null) {
-      $xfer += $output->writeFieldBegin('message', TType::STRING, 1);
-      $xfer += $output->writeString($this->message);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->errorCode !== null) {
-      $xfer += $output->writeFieldBegin('errorCode', TType::I32, 2);
-      $xfer += $output->writeI32($this->errorCode);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->SQLState !== null) {
-      $xfer += $output->writeFieldBegin('SQLState', TType::STRING, 3);
-      $xfer += $output->writeString($this->SQLState);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-


[08/20] hive git commit: HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java
deleted file mode 100644
index 934a8a5..0000000
--- a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java
+++ /dev/null
@@ -1,7784 +0,0 @@
-/**
- * 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.service;
-
-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 ThriftHive {
-
-  public interface Iface extends org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface {
-
-    public void execute(String query) throws HiveServerException, org.apache.thrift.TException;
-
-    public String fetchOne() throws HiveServerException, org.apache.thrift.TException;
-
-    public List<String> fetchN(int numRows) throws HiveServerException, org.apache.thrift.TException;
-
-    public List<String> fetchAll() throws HiveServerException, org.apache.thrift.TException;
-
-    public org.apache.hadoop.hive.metastore.api.Schema getSchema() throws HiveServerException, org.apache.thrift.TException;
-
-    public org.apache.hadoop.hive.metastore.api.Schema getThriftSchema() throws HiveServerException, org.apache.thrift.TException;
-
-    public HiveClusterStatus getClusterStatus() throws HiveServerException, org.apache.thrift.TException;
-
-    public org.apache.hadoop.hive.ql.plan.api.QueryPlan getQueryPlan() throws HiveServerException, org.apache.thrift.TException;
-
-    public void clean() throws org.apache.thrift.TException;
-
-  }
-
-  public interface AsyncIface extends org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore .AsyncIface {
-
-    public void execute(String query, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void fetchOne(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void fetchN(int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void fetchAll(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void getSchema(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void getThriftSchema(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void getClusterStatus(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void getQueryPlan(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void clean(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-  }
-
-  public static class Client extends org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Client implements Iface {
-    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
-      public Factory() {}
-      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
-        return new Client(prot);
-      }
-      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
-        return new Client(iprot, oprot);
-      }
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol prot)
-    {
-      super(prot, prot);
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
-      super(iprot, oprot);
-    }
-
-    public void execute(String query) throws HiveServerException, org.apache.thrift.TException
-    {
-      send_execute(query);
-      recv_execute();
-    }
-
-    public void send_execute(String query) throws org.apache.thrift.TException
-    {
-      execute_args args = new execute_args();
-      args.setQuery(query);
-      sendBase("execute", args);
-    }
-
-    public void recv_execute() throws HiveServerException, org.apache.thrift.TException
-    {
-      execute_result result = new execute_result();
-      receiveBase(result, "execute");
-      if (result.ex != null) {
-        throw result.ex;
-      }
-      return;
-    }
-
-    public String fetchOne() throws HiveServerException, org.apache.thrift.TException
-    {
-      send_fetchOne();
-      return recv_fetchOne();
-    }
-
-    public void send_fetchOne() throws org.apache.thrift.TException
-    {
-      fetchOne_args args = new fetchOne_args();
-      sendBase("fetchOne", args);
-    }
-
-    public String recv_fetchOne() throws HiveServerException, org.apache.thrift.TException
-    {
-      fetchOne_result result = new fetchOne_result();
-      receiveBase(result, "fetchOne");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.ex != null) {
-        throw result.ex;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "fetchOne failed: unknown result");
-    }
-
-    public List<String> fetchN(int numRows) throws HiveServerException, org.apache.thrift.TException
-    {
-      send_fetchN(numRows);
-      return recv_fetchN();
-    }
-
-    public void send_fetchN(int numRows) throws org.apache.thrift.TException
-    {
-      fetchN_args args = new fetchN_args();
-      args.setNumRows(numRows);
-      sendBase("fetchN", args);
-    }
-
-    public List<String> recv_fetchN() throws HiveServerException, org.apache.thrift.TException
-    {
-      fetchN_result result = new fetchN_result();
-      receiveBase(result, "fetchN");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.ex != null) {
-        throw result.ex;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "fetchN failed: unknown result");
-    }
-
-    public List<String> fetchAll() throws HiveServerException, org.apache.thrift.TException
-    {
-      send_fetchAll();
-      return recv_fetchAll();
-    }
-
-    public void send_fetchAll() throws org.apache.thrift.TException
-    {
-      fetchAll_args args = new fetchAll_args();
-      sendBase("fetchAll", args);
-    }
-
-    public List<String> recv_fetchAll() throws HiveServerException, org.apache.thrift.TException
-    {
-      fetchAll_result result = new fetchAll_result();
-      receiveBase(result, "fetchAll");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.ex != null) {
-        throw result.ex;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "fetchAll failed: unknown result");
-    }
-
-    public org.apache.hadoop.hive.metastore.api.Schema getSchema() throws HiveServerException, org.apache.thrift.TException
-    {
-      send_getSchema();
-      return recv_getSchema();
-    }
-
-    public void send_getSchema() throws org.apache.thrift.TException
-    {
-      getSchema_args args = new getSchema_args();
-      sendBase("getSchema", args);
-    }
-
-    public org.apache.hadoop.hive.metastore.api.Schema recv_getSchema() throws HiveServerException, org.apache.thrift.TException
-    {
-      getSchema_result result = new getSchema_result();
-      receiveBase(result, "getSchema");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.ex != null) {
-        throw result.ex;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getSchema failed: unknown result");
-    }
-
-    public org.apache.hadoop.hive.metastore.api.Schema getThriftSchema() throws HiveServerException, org.apache.thrift.TException
-    {
-      send_getThriftSchema();
-      return recv_getThriftSchema();
-    }
-
-    public void send_getThriftSchema() throws org.apache.thrift.TException
-    {
-      getThriftSchema_args args = new getThriftSchema_args();
-      sendBase("getThriftSchema", args);
-    }
-
-    public org.apache.hadoop.hive.metastore.api.Schema recv_getThriftSchema() throws HiveServerException, org.apache.thrift.TException
-    {
-      getThriftSchema_result result = new getThriftSchema_result();
-      receiveBase(result, "getThriftSchema");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.ex != null) {
-        throw result.ex;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getThriftSchema failed: unknown result");
-    }
-
-    public HiveClusterStatus getClusterStatus() throws HiveServerException, org.apache.thrift.TException
-    {
-      send_getClusterStatus();
-      return recv_getClusterStatus();
-    }
-
-    public void send_getClusterStatus() throws org.apache.thrift.TException
-    {
-      getClusterStatus_args args = new getClusterStatus_args();
-      sendBase("getClusterStatus", args);
-    }
-
-    public HiveClusterStatus recv_getClusterStatus() throws HiveServerException, org.apache.thrift.TException
-    {
-      getClusterStatus_result result = new getClusterStatus_result();
-      receiveBase(result, "getClusterStatus");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.ex != null) {
-        throw result.ex;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterStatus failed: unknown result");
-    }
-
-    public org.apache.hadoop.hive.ql.plan.api.QueryPlan getQueryPlan() throws HiveServerException, org.apache.thrift.TException
-    {
-      send_getQueryPlan();
-      return recv_getQueryPlan();
-    }
-
-    public void send_getQueryPlan() throws org.apache.thrift.TException
-    {
-      getQueryPlan_args args = new getQueryPlan_args();
-      sendBase("getQueryPlan", args);
-    }
-
-    public org.apache.hadoop.hive.ql.plan.api.QueryPlan recv_getQueryPlan() throws HiveServerException, org.apache.thrift.TException
-    {
-      getQueryPlan_result result = new getQueryPlan_result();
-      receiveBase(result, "getQueryPlan");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.ex != null) {
-        throw result.ex;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getQueryPlan failed: unknown result");
-    }
-
-    public void clean() throws org.apache.thrift.TException
-    {
-      send_clean();
-      recv_clean();
-    }
-
-    public void send_clean() throws org.apache.thrift.TException
-    {
-      clean_args args = new clean_args();
-      sendBase("clean", args);
-    }
-
-    public void recv_clean() throws org.apache.thrift.TException
-    {
-      clean_result result = new clean_result();
-      receiveBase(result, "clean");
-      return;
-    }
-
-  }
-  public static class AsyncClient extends org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.AsyncClient implements AsyncIface {
-    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
-      private org.apache.thrift.async.TAsyncClientManager clientManager;
-      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
-      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
-        this.clientManager = clientManager;
-        this.protocolFactory = protocolFactory;
-      }
-      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
-        return new AsyncClient(protocolFactory, clientManager, transport);
-      }
-    }
-
-    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
-      super(protocolFactory, clientManager, transport);
-    }
-
-    public void execute(String query, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      execute_call method_call = new execute_call(query, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class execute_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String query;
-      public execute_call(String query, 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.query = query;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("execute", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        execute_args args = new execute_args();
-        args.setQuery(query);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public void getResult() throws HiveServerException, 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_execute();
-      }
-    }
-
-    public void fetchOne(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      fetchOne_call method_call = new fetchOne_call(resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class fetchOne_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public fetchOne_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("fetchOne", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        fetchOne_args args = new fetchOne_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public String getResult() throws HiveServerException, 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_fetchOne();
-      }
-    }
-
-    public void fetchN(int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      fetchN_call method_call = new fetchN_call(numRows, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class fetchN_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private int numRows;
-      public fetchN_call(int numRows, 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.numRows = numRows;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("fetchN", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        fetchN_args args = new fetchN_args();
-        args.setNumRows(numRows);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public List<String> getResult() throws HiveServerException, 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_fetchN();
-      }
-    }
-
-    public void fetchAll(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      fetchAll_call method_call = new fetchAll_call(resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class fetchAll_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public fetchAll_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("fetchAll", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        fetchAll_args args = new fetchAll_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public List<String> getResult() throws HiveServerException, 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_fetchAll();
-      }
-    }
-
-    public void getSchema(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getSchema_call method_call = new getSchema_call(resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getSchema_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public getSchema_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getSchema", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getSchema_args args = new getSchema_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public org.apache.hadoop.hive.metastore.api.Schema getResult() throws HiveServerException, 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_getSchema();
-      }
-    }
-
-    public void getThriftSchema(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getThriftSchema_call method_call = new getThriftSchema_call(resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getThriftSchema_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public getThriftSchema_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getThriftSchema", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getThriftSchema_args args = new getThriftSchema_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public org.apache.hadoop.hive.metastore.api.Schema getResult() throws HiveServerException, 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_getThriftSchema();
-      }
-    }
-
-    public void getClusterStatus(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getClusterStatus_call method_call = new getClusterStatus_call(resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getClusterStatus_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public getClusterStatus_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getClusterStatus", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getClusterStatus_args args = new getClusterStatus_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public HiveClusterStatus getResult() throws HiveServerException, 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_getClusterStatus();
-      }
-    }
-
-    public void getQueryPlan(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getQueryPlan_call method_call = new getQueryPlan_call(resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getQueryPlan_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public getQueryPlan_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getQueryPlan", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getQueryPlan_args args = new getQueryPlan_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public org.apache.hadoop.hive.ql.plan.api.QueryPlan getResult() throws HiveServerException, 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_getQueryPlan();
-      }
-    }
-
-    public void clean(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      clean_call method_call = new clean_call(resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class clean_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public clean_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("clean", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        clean_args args = new clean_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public void getResult() throws org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_clean();
-      }
-    }
-
-  }
-
-  public static class Processor<I extends Iface> extends org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Processor<I> implements org.apache.thrift.TProcessor {
-    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
-    public Processor(I iface) {
-      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
-    }
-
-    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      processMap.put("execute", new execute());
-      processMap.put("fetchOne", new fetchOne());
-      processMap.put("fetchN", new fetchN());
-      processMap.put("fetchAll", new fetchAll());
-      processMap.put("getSchema", new getSchema());
-      processMap.put("getThriftSchema", new getThriftSchema());
-      processMap.put("getClusterStatus", new getClusterStatus());
-      processMap.put("getQueryPlan", new getQueryPlan());
-      processMap.put("clean", new clean());
-      return processMap;
-    }
-
-    public static class execute<I extends Iface> extends org.apache.thrift.ProcessFunction<I, execute_args> {
-      public execute() {
-        super("execute");
-      }
-
-      public execute_args getEmptyArgsInstance() {
-        return new execute_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public execute_result getResult(I iface, execute_args args) throws org.apache.thrift.TException {
-        execute_result result = new execute_result();
-        try {
-          iface.execute(args.query);
-        } catch (HiveServerException ex) {
-          result.ex = ex;
-        }
-        return result;
-      }
-    }
-
-    public static class fetchOne<I extends Iface> extends org.apache.thrift.ProcessFunction<I, fetchOne_args> {
-      public fetchOne() {
-        super("fetchOne");
-      }
-
-      public fetchOne_args getEmptyArgsInstance() {
-        return new fetchOne_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public fetchOne_result getResult(I iface, fetchOne_args args) throws org.apache.thrift.TException {
-        fetchOne_result result = new fetchOne_result();
-        try {
-          result.success = iface.fetchOne();
-        } catch (HiveServerException ex) {
-          result.ex = ex;
-        }
-        return result;
-      }
-    }
-
-    public static class fetchN<I extends Iface> extends org.apache.thrift.ProcessFunction<I, fetchN_args> {
-      public fetchN() {
-        super("fetchN");
-      }
-
-      public fetchN_args getEmptyArgsInstance() {
-        return new fetchN_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public fetchN_result getResult(I iface, fetchN_args args) throws org.apache.thrift.TException {
-        fetchN_result result = new fetchN_result();
-        try {
-          result.success = iface.fetchN(args.numRows);
-        } catch (HiveServerException ex) {
-          result.ex = ex;
-        }
-        return result;
-      }
-    }
-
-    public static class fetchAll<I extends Iface> extends org.apache.thrift.ProcessFunction<I, fetchAll_args> {
-      public fetchAll() {
-        super("fetchAll");
-      }
-
-      public fetchAll_args getEmptyArgsInstance() {
-        return new fetchAll_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public fetchAll_result getResult(I iface, fetchAll_args args) throws org.apache.thrift.TException {
-        fetchAll_result result = new fetchAll_result();
-        try {
-          result.success = iface.fetchAll();
-        } catch (HiveServerException ex) {
-          result.ex = ex;
-        }
-        return result;
-      }
-    }
-
-    public static class getSchema<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getSchema_args> {
-      public getSchema() {
-        super("getSchema");
-      }
-
-      public getSchema_args getEmptyArgsInstance() {
-        return new getSchema_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public getSchema_result getResult(I iface, getSchema_args args) throws org.apache.thrift.TException {
-        getSchema_result result = new getSchema_result();
-        try {
-          result.success = iface.getSchema();
-        } catch (HiveServerException ex) {
-          result.ex = ex;
-        }
-        return result;
-      }
-    }
-
-    public static class getThriftSchema<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getThriftSchema_args> {
-      public getThriftSchema() {
-        super("getThriftSchema");
-      }
-
-      public getThriftSchema_args getEmptyArgsInstance() {
-        return new getThriftSchema_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public getThriftSchema_result getResult(I iface, getThriftSchema_args args) throws org.apache.thrift.TException {
-        getThriftSchema_result result = new getThriftSchema_result();
-        try {
-          result.success = iface.getThriftSchema();
-        } catch (HiveServerException ex) {
-          result.ex = ex;
-        }
-        return result;
-      }
-    }
-
-    public static class getClusterStatus<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getClusterStatus_args> {
-      public getClusterStatus() {
-        super("getClusterStatus");
-      }
-
-      public getClusterStatus_args getEmptyArgsInstance() {
-        return new getClusterStatus_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public getClusterStatus_result getResult(I iface, getClusterStatus_args args) throws org.apache.thrift.TException {
-        getClusterStatus_result result = new getClusterStatus_result();
-        try {
-          result.success = iface.getClusterStatus();
-        } catch (HiveServerException ex) {
-          result.ex = ex;
-        }
-        return result;
-      }
-    }
-
-    public static class getQueryPlan<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getQueryPlan_args> {
-      public getQueryPlan() {
-        super("getQueryPlan");
-      }
-
-      public getQueryPlan_args getEmptyArgsInstance() {
-        return new getQueryPlan_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public getQueryPlan_result getResult(I iface, getQueryPlan_args args) throws org.apache.thrift.TException {
-        getQueryPlan_result result = new getQueryPlan_result();
-        try {
-          result.success = iface.getQueryPlan();
-        } catch (HiveServerException ex) {
-          result.ex = ex;
-        }
-        return result;
-      }
-    }
-
-    public static class clean<I extends Iface> extends org.apache.thrift.ProcessFunction<I, clean_args> {
-      public clean() {
-        super("clean");
-      }
-
-      public clean_args getEmptyArgsInstance() {
-        return new clean_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public clean_result getResult(I iface, clean_args args) throws org.apache.thrift.TException {
-        clean_result result = new clean_result();
-        iface.clean();
-        return result;
-      }
-    }
-
-  }
-
-  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.AsyncProcessor<I> {
-    private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName());
-    public AsyncProcessor(I iface) {
-      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
-    }
-
-    protected AsyncProcessor(I iface, Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends AsyncIface> Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
-      processMap.put("execute", new execute());
-      processMap.put("fetchOne", new fetchOne());
-      processMap.put("fetchN", new fetchN());
-      processMap.put("fetchAll", new fetchAll());
-      processMap.put("getSchema", new getSchema());
-      processMap.put("getThriftSchema", new getThriftSchema());
-      processMap.put("getClusterStatus", new getClusterStatus());
-      processMap.put("getQueryPlan", new getQueryPlan());
-      processMap.put("clean", new clean());
-      return processMap;
-    }
-
-    public static class execute<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, execute_args, Void> {
-      public execute() {
-        super("execute");
-      }
-
-      public execute_args getEmptyArgsInstance() {
-        return new execute_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) {
-            execute_result result = new execute_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;
-            execute_result result = new execute_result();
-            if (e instanceof HiveServerException) {
-                        result.ex = (HiveServerException) e;
-                        result.setExIsSet(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, execute_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.execute(args.query,resultHandler);
-      }
-    }
-
-    public static class fetchOne<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fetchOne_args, String> {
-      public fetchOne() {
-        super("fetchOne");
-      }
-
-      public fetchOne_args getEmptyArgsInstance() {
-        return new fetchOne_args();
-      }
-
-      public AsyncMethodCallback<String> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<String>() { 
-          public void onComplete(String o) {
-            fetchOne_result result = new fetchOne_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;
-            fetchOne_result result = new fetchOne_result();
-            if (e instanceof HiveServerException) {
-                        result.ex = (HiveServerException) e;
-                        result.setExIsSet(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, fetchOne_args args, org.apache.thrift.async.AsyncMethodCallback<String> resultHandler) throws TException {
-        iface.fetchOne(resultHandler);
-      }
-    }
-
-    public static class fetchN<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fetchN_args, List<String>> {
-      public fetchN() {
-        super("fetchN");
-      }
-
-      public fetchN_args getEmptyArgsInstance() {
-        return new fetchN_args();
-      }
-
-      public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<List<String>>() { 
-          public void onComplete(List<String> o) {
-            fetchN_result result = new fetchN_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;
-            fetchN_result result = new fetchN_result();
-            if (e instanceof HiveServerException) {
-                        result.ex = (HiveServerException) e;
-                        result.setExIsSet(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, fetchN_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
-        iface.fetchN(args.numRows,resultHandler);
-      }
-    }
-
-    public static class fetchAll<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fetchAll_args, List<String>> {
-      public fetchAll() {
-        super("fetchAll");
-      }
-
-      public fetchAll_args getEmptyArgsInstance() {
-        return new fetchAll_args();
-      }
-
-      public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<List<String>>() { 
-          public void onComplete(List<String> o) {
-            fetchAll_result result = new fetchAll_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;
-            fetchAll_result result = new fetchAll_result();
-            if (e instanceof HiveServerException) {
-                        result.ex = (HiveServerException) e;
-                        result.setExIsSet(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, fetchAll_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
-        iface.fetchAll(resultHandler);
-      }
-    }
-
-    public static class getSchema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getSchema_args, org.apache.hadoop.hive.metastore.api.Schema> {
-      public getSchema() {
-        super("getSchema");
-      }
-
-      public getSchema_args getEmptyArgsInstance() {
-        return new getSchema_args();
-      }
-
-      public AsyncMethodCallback<org.apache.hadoop.hive.metastore.api.Schema> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<org.apache.hadoop.hive.metastore.api.Schema>() { 
-          public void onComplete(org.apache.hadoop.hive.metastore.api.Schema o) {
-            getSchema_result result = new getSchema_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;
-            getSchema_result result = new getSchema_result();
-            if (e instanceof HiveServerException) {
-                        result.ex = (HiveServerException) e;
-                        result.setExIsSet(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, getSchema_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.hadoop.hive.metastore.api.Schema> resultHandler) throws TException {
-        iface.getSchema(resultHandler);
-      }
-    }
-
-    public static class getThriftSchema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getThriftSchema_args, org.apache.hadoop.hive.metastore.api.Schema> {
-      public getThriftSchema() {
-        super("getThriftSchema");
-      }
-
-      public getThriftSchema_args getEmptyArgsInstance() {
-        return new getThriftSchema_args();
-      }
-
-      public AsyncMethodCallback<org.apache.hadoop.hive.metastore.api.Schema> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<org.apache.hadoop.hive.metastore.api.Schema>() { 
-          public void onComplete(org.apache.hadoop.hive.metastore.api.Schema o) {
-            getThriftSchema_result result = new getThriftSchema_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;
-            getThriftSchema_result result = new getThriftSchema_result();
-            if (e instanceof HiveServerException) {
-                        result.ex = (HiveServerException) e;
-                        result.setExIsSet(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, getThriftSchema_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.hadoop.hive.metastore.api.Schema> resultHandler) throws TException {
-        iface.getThriftSchema(resultHandler);
-      }
-    }
-
-    public static class getClusterStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getClusterStatus_args, HiveClusterStatus> {
-      public getClusterStatus() {
-        super("getClusterStatus");
-      }
-
-      public getClusterStatus_args getEmptyArgsInstance() {
-        return new getClusterStatus_args();
-      }
-
-      public AsyncMethodCallback<HiveClusterStatus> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<HiveClusterStatus>() { 
-          public void onComplete(HiveClusterStatus o) {
-            getClusterStatus_result result = new getClusterStatus_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;
-            getClusterStatus_result result = new getClusterStatus_result();
-            if (e instanceof HiveServerException) {
-                        result.ex = (HiveServerException) e;
-                        result.setExIsSet(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, getClusterStatus_args args, org.apache.thrift.async.AsyncMethodCallback<HiveClusterStatus> resultHandler) throws TException {
-        iface.getClusterStatus(resultHandler);
-      }
-    }
-
-    public static class getQueryPlan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getQueryPlan_args, org.apache.hadoop.hive.ql.plan.api.QueryPlan> {
-      public getQueryPlan() {
-        super("getQueryPlan");
-      }
-
-      public getQueryPlan_args getEmptyArgsInstance() {
-        return new getQueryPlan_args();
-      }
-
-      public AsyncMethodCallback<org.apache.hadoop.hive.ql.plan.api.QueryPlan> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<org.apache.hadoop.hive.ql.plan.api.QueryPlan>() { 
-          public void onComplete(org.apache.hadoop.hive.ql.plan.api.QueryPlan o) {
-            getQueryPlan_result result = new getQueryPlan_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;
-            getQueryPlan_result result = new getQueryPlan_result();
-            if (e instanceof HiveServerException) {
-                        result.ex = (HiveServerException) e;
-                        result.setExIsSet(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, getQueryPlan_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.hadoop.hive.ql.plan.api.QueryPlan> resultHandler) throws TException {
-        iface.getQueryPlan(resultHandler);
-      }
-    }
-
-    public static class clean<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, clean_args, Void> {
-      public clean() {
-        super("clean");
-      }
-
-      public clean_args getEmptyArgsInstance() {
-        return new clean_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) {
-            clean_result result = new clean_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;
-            clean_result result = new clean_result();
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, clean_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.clean(resultHandler);
-      }
-    }
-
-  }
-
-  public static class execute_args implements org.apache.thrift.TBase<execute_args, execute_args._Fields>, java.io.Serializable, Cloneable, Comparable<execute_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("execute_args");
-
-    private static final org.apache.thrift.protocol.TField QUERY_FIELD_DESC = new org.apache.thrift.protocol.TField("query", org.apache.thrift.protocol.TType.STRING, (short)1);
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new execute_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new execute_argsTupleSchemeFactory());
-    }
-
-    private String query; // 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 {
-      QUERY((short)1, "query");
-
-      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: // QUERY
-            return QUERY;
-          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.QUERY, new org.apache.thrift.meta_data.FieldMetaData("query", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(execute_args.class, metaDataMap);
-    }
-
-    public execute_args() {
-    }
-
-    public execute_args(
-      String query)
-    {
-      this();
-      this.query = query;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public execute_args(execute_args other) {
-      if (other.isSetQuery()) {
-        this.query = other.query;
-      }
-    }
-
-    public execute_args deepCopy() {
-      return new execute_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.query = null;
-    }
-
-    public String getQuery() {
-      return this.query;
-    }
-
-    public void setQuery(String query) {
-      this.query = query;
-    }
-
-    public void unsetQuery() {
-      this.query = null;
-    }
-
-    /** Returns true if field query is set (has been assigned a value) and false otherwise */
-    public boolean isSetQuery() {
-      return this.query != null;
-    }
-
-    public void setQueryIsSet(boolean value) {
-      if (!value) {
-        this.query = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case QUERY:
-        if (value == null) {
-          unsetQuery();
-        } else {
-          setQuery((String)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case QUERY:
-        return getQuery();
-
-      }
-      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 QUERY:
-        return isSetQuery();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof execute_args)
-        return this.equals((execute_args)that);
-      return false;
-    }
-
-    public boolean equals(execute_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_query = true && this.isSetQuery();
-      boolean that_present_query = true && that.isSetQuery();
-      if (this_present_query || that_present_query) {
-        if (!(this_present_query && that_present_query))
-          return false;
-        if (!this.query.equals(that.query))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      boolean present_query = true && (isSetQuery());
-      list.add(present_query);
-      if (present_query)
-        list.add(query);
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(execute_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = Boolean.valueOf(isSetQuery()).compareTo(other.isSetQuery());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetQuery()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.query, other.query);
-        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("execute_args(");
-      boolean first = true;
-
-      sb.append("query:");
-      if (this.query == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.query);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class execute_argsStandardSchemeFactory implements SchemeFactory {
-      public execute_argsStandardScheme getScheme() {
-        return new execute_argsStandardScheme();
-      }
-    }
-
-    private static class execute_argsStandardScheme extends StandardScheme<execute_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, execute_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // QUERY
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.query = iprot.readString();
-                struct.setQueryIsSet(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, execute_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.query != null) {
-          oprot.writeFieldBegin(QUERY_FIELD_DESC);
-          oprot.writeString(struct.query);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class execute_argsTupleSchemeFactory implements SchemeFactory {
-      public execute_argsTupleScheme getScheme() {
-        return new execute_argsTupleScheme();
-      }
-    }
-
-    private static class execute_argsTupleScheme extends TupleScheme<execute_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, execute_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetQuery()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetQuery()) {
-          oprot.writeString(struct.query);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, execute_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.query = iprot.readString();
-          struct.setQueryIsSet(true);
-        }
-      }
-    }
-
-  }
-
-  public static class execute_result implements org.apache.thrift.TBase<execute_result, execute_result._Fields>, java.io.Serializable, Cloneable, Comparable<execute_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("execute_result");
-
-    private static final org.apache.thrift.protocol.TField EX_FIELD_DESC = new org.apache.thrift.protocol.TField("ex", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new execute_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new execute_resultTupleSchemeFactory());
-    }
-
-    private HiveServerException ex; // 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 {
-      EX((short)1, "ex");
-
-      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: // EX
-            return EX;
-          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.EX, new org.apache.thrift.meta_data.FieldMetaData("ex", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(execute_result.class, metaDataMap);
-    }
-
-    public execute_result() {
-    }
-
-    public execute_result(
-      HiveServerException ex)
-    {
-      this();
-      this.ex = ex;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public execute_result(execute_result other) {
-      if (other.isSetEx()) {
-        this.ex = new HiveServerException(other.ex);
-      }
-    }
-
-    public execute_result deepCopy() {
-      return new execute_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.ex = null;
-    }
-
-    public HiveServerException getEx() {
-      return this.ex;
-    }
-
-    public void setEx(HiveServerException ex) {
-      this.ex = ex;
-    }
-
-    public void unsetEx() {
-      this.ex = null;
-    }
-
-    /** Returns true if field ex is set (has been assigned a value) and false otherwise */
-    public boolean isSetEx() {
-      return this.ex != null;
-    }
-
-    public void setExIsSet(boolean value) {
-      if (!value) {
-        this.ex = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case EX:
-        if (value == null) {
-          unsetEx();
-        } else {
-          setEx((HiveServerException)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case EX:
-        return getEx();
-
-      }
-      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 EX:
-        return isSetEx();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof execute_result)
-        return this.equals((execute_result)that);
-      return false;
-    }
-
-    public boolean equals(execute_result that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_ex = true && this.isSetEx();
-      boolean that_present_ex = true && that.isSetEx();
-      if (this_present_ex || that_present_ex) {
-        if (!(this_present_ex && that_present_ex))
-          return false;
-        if (!this.ex.equals(that.ex))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      boolean present_ex = true && (isSetEx());
-      list.add(present_ex);
-      if (present_ex)
-        list.add(ex);
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(execute_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = Boolean.valueOf(isSetEx()).compareTo(other.isSetEx());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetEx()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ex, other.ex);
-        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("execute_result(");
-      boolean first = true;
-
-      sb.append("ex:");
-      if (this.ex == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.ex);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class execute_resultStandardSchemeFactory implements SchemeFactory {
-      public execute_resultStandardScheme getScheme() {
-        return new execute_resultStandardScheme();
-      }
-    }
-
-    private static class execute_resultStandardScheme extends StandardScheme<execute_result> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, execute_result 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: // EX
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.ex = new HiveServerException();
-                struct.ex.read(iprot);
-                struct.setExIsSet(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, execute_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.ex != null) {
-          oprot.writeFieldBegin(EX_FIELD_DESC);
-          struct.ex.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class execute_resultTupleSchemeFactory implements SchemeFactory {
-      public execute_resultTupleScheme getScheme() {
-        return new execute_resultTupleScheme();
-      }
-    }
-
-    private static class execute_resultTupleScheme extends TupleScheme<execute_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, execute_result struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetEx()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetEx()) {
-          struct.ex.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, execute_result struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.ex = new HiveServerException();
-          struct.ex.read(iprot);
-          struct.setExIsSet(true);
-        }
-      }
-    }
-
-  }
-
-  public static class fetchOne_args implements org.apache.thrift.TBase<fetchOne_args, fetchOne_args._Fields>, java.io.Serializable, Cloneable, Comparable<fetchOne_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("fetchOne_args");
-
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new fetchOne_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new fetchOne_argsTupleSchemeFactory());
-    }
-
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
-
-      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-      static {
-        for (_Fields field : EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      public static _Fields findByName(String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final String _fieldName;
-
-      _Fields(short thriftId, String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      public String getFieldName() {
-        return _fieldName;
-      }
-    }
-    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(fetchOne_args.class, metaDataMap);
-    }
-
-    public fetchOne_args() {
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public fetchOne_args(fetchOne_args other) {
-    }
-
-    public fetchOne_args deepCopy() {
-      return new fetchOne_args(this);
-    }
-
-    @Override
-    public void clear() {
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      }
-      throw new IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new IllegalArgumentException();
-      }
-
-      switch (field) {
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof fetchOne_args)
-        return this.equals((fetchOne_args)that);
-      return false;
-    }
-
-    public boolean equals(fetchOne_args that) {
-      if (that == null)
-        return false;
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(fetchOne_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      return 0;
-    }
-
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder("fetchOne_args(");
-      boolean first = true;
-
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class fetchOne_argsStandardSchemeFactory implements SchemeFactory {
-      public fetchOne_argsStandardScheme getScheme() {
-        return new fetchOne_argsStandardScheme();
-      }
-    }
-
-    private static class fetchOne_argsStandardScheme extends StandardScheme<fetchOne_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, fetchOne_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          

<TRUNCATED>

[06/20] hive git commit: HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/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
deleted file mode 100755
index 4dfa83d..0000000
--- a/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
+++ /dev/null
@@ -1,1263 +0,0 @@
-#!/usr/bin/env python
-#
-# Autogenerated by Thrift Compiler (0.9.3)
-#
-# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-#
-#  options string: py
-#
-
-import sys
-import pprint
-from urlparse import urlparse
-from thrift.transport import TTransport
-from thrift.transport import TSocket
-from thrift.transport import TSSLSocket
-from thrift.transport import THttpClient
-from thrift.protocol import TBinaryProtocol
-
-from hive_service import ThriftHive
-from hive_service.ttypes import *
-
-if len(sys.argv) <= 1 or sys.argv[1] == '--help':
-  print('')
-  print('Usage: ' + sys.argv[0] + ' [-h host[:port]] [-u url] [-f[ramed]] [-s[sl]] function [arg1 [arg2...]]')
-  print('')
-  print('Functions:')
-  print('  void execute(string query)')
-  print('  string fetchOne()')
-  print('   fetchN(i32 numRows)')
-  print('   fetchAll()')
-  print('  Schema getSchema()')
-  print('  Schema getThriftSchema()')
-  print('  HiveClusterStatus getClusterStatus()')
-  print('  QueryPlan getQueryPlan()')
-  print('  void clean()')
-  print('  string getMetaConf(string key)')
-  print('  void setMetaConf(string key, string value)')
-  print('  void create_database(Database database)')
-  print('  Database get_database(string name)')
-  print('  void drop_database(string name, bool deleteData, bool cascade)')
-  print('   get_databases(string pattern)')
-  print('   get_all_databases()')
-  print('  void alter_database(string dbname, Database db)')
-  print('  Type get_type(string name)')
-  print('  bool create_type(Type type)')
-  print('  bool drop_type(string type)')
-  print('   get_type_all(string name)')
-  print('   get_fields(string db_name, string table_name)')
-  print('   get_fields_with_environment_context(string db_name, string table_name, EnvironmentContext environment_context)')
-  print('   get_schema(string db_name, string table_name)')
-  print('   get_schema_with_environment_context(string db_name, string table_name, EnvironmentContext environment_context)')
-  print('  void create_table(Table tbl)')
-  print('  void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context)')
-  print('  void 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)')
-  print('   get_table_meta(string db_patterns, string tbl_patterns,  tbl_types)')
-  print('   get_all_tables(string db_name)')
-  print('  Table get_table(string dbname, string tbl_name)')
-  print('   get_table_objects_by_name(string dbname,  tbl_names)')
-  print('   get_table_names_by_filter(string dbname, string filter, i16 max_tables)')
-  print('  void alter_table(string dbname, string tbl_name, Table new_tbl)')
-  print('  void alter_table_with_environment_context(string dbname, string tbl_name, Table new_tbl, EnvironmentContext environment_context)')
-  print('  void alter_table_with_cascade(string dbname, string tbl_name, Table new_tbl, bool cascade)')
-  print('  Partition add_partition(Partition new_part)')
-  print('  Partition add_partition_with_environment_context(Partition new_part, EnvironmentContext environment_context)')
-  print('  i32 add_partitions( new_parts)')
-  print('  i32 add_partitions_pspec( new_parts)')
-  print('  Partition append_partition(string db_name, string tbl_name,  part_vals)')
-  print('  AddPartitionsResult add_partitions_req(AddPartitionsRequest request)')
-  print('  Partition append_partition_with_environment_context(string db_name, string tbl_name,  part_vals, EnvironmentContext environment_context)')
-  print('  Partition append_partition_by_name(string db_name, string tbl_name, string part_name)')
-  print('  Partition append_partition_by_name_with_environment_context(string db_name, string tbl_name, string part_name, EnvironmentContext environment_context)')
-  print('  bool drop_partition(string db_name, string tbl_name,  part_vals, bool deleteData)')
-  print('  bool drop_partition_with_environment_context(string db_name, string tbl_name,  part_vals, bool deleteData, EnvironmentContext environment_context)')
-  print('  bool drop_partition_by_name(string db_name, string tbl_name, string part_name, bool deleteData)')
-  print('  bool drop_partition_by_name_with_environment_context(string db_name, string tbl_name, string part_name, bool deleteData, EnvironmentContext environment_context)')
-  print('  DropPartitionsResult drop_partitions_req(DropPartitionsRequest req)')
-  print('  Partition get_partition(string db_name, string tbl_name,  part_vals)')
-  print('  Partition exchange_partition( partitionSpecs, string source_db, string source_table_name, string dest_db, string dest_table_name)')
-  print('   exchange_partitions( partitionSpecs, string source_db, string source_table_name, string dest_db, string dest_table_name)')
-  print('  Partition get_partition_with_auth(string db_name, string tbl_name,  part_vals, string user_name,  group_names)')
-  print('  Partition get_partition_by_name(string db_name, string tbl_name, string part_name)')
-  print('   get_partitions(string db_name, string tbl_name, i16 max_parts)')
-  print('   get_partitions_with_auth(string db_name, string tbl_name, i16 max_parts, string user_name,  group_names)')
-  print('   get_partitions_pspec(string db_name, string tbl_name, i32 max_parts)')
-  print('   get_partition_names(string db_name, string tbl_name, i16 max_parts)')
-  print('   get_partitions_ps(string db_name, string tbl_name,  part_vals, i16 max_parts)')
-  print('   get_partitions_ps_with_auth(string db_name, string tbl_name,  part_vals, i16 max_parts, string user_name,  group_names)')
-  print('   get_partition_names_ps(string db_name, string tbl_name,  part_vals, i16 max_parts)')
-  print('   get_partitions_by_filter(string db_name, string tbl_name, string filter, i16 max_parts)')
-  print('   get_part_specs_by_filter(string db_name, string tbl_name, string filter, i32 max_parts)')
-  print('  PartitionsByExprResult get_partitions_by_expr(PartitionsByExprRequest req)')
-  print('  i32 get_num_partitions_by_filter(string db_name, string tbl_name, string filter)')
-  print('   get_partitions_by_names(string db_name, string tbl_name,  names)')
-  print('  void alter_partition(string db_name, string tbl_name, Partition new_part)')
-  print('  void alter_partitions(string db_name, string tbl_name,  new_parts)')
-  print('  void alter_partitions_with_environment_context(string db_name, string tbl_name,  new_parts, EnvironmentContext environment_context)')
-  print('  void alter_partition_with_environment_context(string db_name, string tbl_name, Partition new_part, EnvironmentContext environment_context)')
-  print('  void rename_partition(string db_name, string tbl_name,  part_vals, Partition new_part)')
-  print('  bool partition_name_has_valid_characters( part_vals, bool throw_exception)')
-  print('  string get_config_value(string name, string defaultValue)')
-  print('   partition_name_to_vals(string part_name)')
-  print('   partition_name_to_spec(string part_name)')
-  print('  void markPartitionForEvent(string db_name, string tbl_name,  part_vals, PartitionEventType eventType)')
-  print('  bool isPartitionMarkedForEvent(string db_name, string tbl_name,  part_vals, PartitionEventType eventType)')
-  print('  Index add_index(Index new_index, Table index_table)')
-  print('  void alter_index(string dbname, string base_tbl_name, string idx_name, Index new_idx)')
-  print('  bool drop_index_by_name(string db_name, string tbl_name, string index_name, bool deleteData)')
-  print('  Index get_index_by_name(string db_name, string tbl_name, string index_name)')
-  print('   get_indexes(string db_name, string tbl_name, i16 max_indexes)')
-  print('   get_index_names(string db_name, string tbl_name, i16 max_indexes)')
-  print('  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)')
-  print('  ColumnStatistics get_partition_column_statistics(string db_name, string tbl_name, string part_name, string col_name)')
-  print('  TableStatsResult get_table_statistics_req(TableStatsRequest request)')
-  print('  PartitionsStatsResult get_partitions_statistics_req(PartitionsStatsRequest request)')
-  print('  AggrStats get_aggr_stats_for(PartitionsStatsRequest request)')
-  print('  bool set_aggr_stats_for(SetPartitionsStatsRequest request)')
-  print('  bool delete_partition_column_statistics(string db_name, string tbl_name, string part_name, string col_name)')
-  print('  bool delete_table_column_statistics(string db_name, string tbl_name, string col_name)')
-  print('  void create_function(Function func)')
-  print('  void drop_function(string dbName, string funcName)')
-  print('  void alter_function(string dbName, string funcName, Function newFunc)')
-  print('   get_functions(string dbName, string pattern)')
-  print('  Function get_function(string dbName, string funcName)')
-  print('  GetAllFunctionsResponse get_all_functions()')
-  print('  bool create_role(Role role)')
-  print('  bool drop_role(string role_name)')
-  print('   get_role_names()')
-  print('  bool grant_role(string role_name, string principal_name, PrincipalType principal_type, string grantor, PrincipalType grantorType, bool grant_option)')
-  print('  bool revoke_role(string role_name, string principal_name, PrincipalType principal_type)')
-  print('   list_roles(string principal_name, PrincipalType principal_type)')
-  print('  GrantRevokeRoleResponse grant_revoke_role(GrantRevokeRoleRequest request)')
-  print('  GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest request)')
-  print('  GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(GetRoleGrantsForPrincipalRequest request)')
-  print('  PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject, string user_name,  group_names)')
-  print('   list_privileges(string principal_name, PrincipalType principal_type, HiveObjectRef hiveObject)')
-  print('  bool grant_privileges(PrivilegeBag privileges)')
-  print('  bool revoke_privileges(PrivilegeBag privileges)')
-  print('  GrantRevokePrivilegeResponse grant_revoke_privileges(GrantRevokePrivilegeRequest request)')
-  print('   set_ugi(string user_name,  group_names)')
-  print('  string get_delegation_token(string token_owner, string renewer_kerberos_principal_name)')
-  print('  i64 renew_delegation_token(string token_str_form)')
-  print('  void cancel_delegation_token(string token_str_form)')
-  print('  bool add_token(string token_identifier, string delegation_token)')
-  print('  bool remove_token(string token_identifier)')
-  print('  string get_token(string token_identifier)')
-  print('   get_all_token_identifiers()')
-  print('  i32 add_master_key(string key)')
-  print('  void update_master_key(i32 seq_number, string key)')
-  print('  bool remove_master_key(i32 key_seq)')
-  print('   get_master_keys()')
-  print('  GetOpenTxnsResponse get_open_txns()')
-  print('  GetOpenTxnsInfoResponse get_open_txns_info()')
-  print('  OpenTxnsResponse open_txns(OpenTxnRequest rqst)')
-  print('  void abort_txn(AbortTxnRequest rqst)')
-  print('  void commit_txn(CommitTxnRequest rqst)')
-  print('  LockResponse lock(LockRequest rqst)')
-  print('  LockResponse check_lock(CheckLockRequest rqst)')
-  print('  void unlock(UnlockRequest rqst)')
-  print('  ShowLocksResponse show_locks(ShowLocksRequest rqst)')
-  print('  void heartbeat(HeartbeatRequest ids)')
-  print('  HeartbeatTxnRangeResponse heartbeat_txn_range(HeartbeatTxnRangeRequest txns)')
-  print('  void compact(CompactionRequest rqst)')
-  print('  ShowCompactResponse show_compact(ShowCompactRequest rqst)')
-  print('  void add_dynamic_partitions(AddDynamicPartitions rqst)')
-  print('  NotificationEventResponse get_next_notification(NotificationEventRequest rqst)')
-  print('  CurrentNotificationEventId get_current_notificationEventId()')
-  print('  FireEventResponse fire_listener_event(FireEventRequest rqst)')
-  print('  void flushCache()')
-  print('  GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req)')
-  print('  GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req)')
-  print('  PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req)')
-  print('  ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req)')
-  print('  CacheFileMetadataResult cache_file_metadata(CacheFileMetadataRequest req)')
-  print('  GetChangeVersionResult get_change_version(GetChangeVersionRequest req)')
-  print('  string getName()')
-  print('  string getVersion()')
-  print('  fb_status getStatus()')
-  print('  string getStatusDetails()')
-  print('   getCounters()')
-  print('  i64 getCounter(string key)')
-  print('  void setOption(string key, string value)')
-  print('  string getOption(string key)')
-  print('   getOptions()')
-  print('  string getCpuProfile(i32 profileDurationInSec)')
-  print('  i64 aliveSince()')
-  print('  void reinitialize()')
-  print('  void shutdown()')
-  print('')
-  sys.exit(0)
-
-pp = pprint.PrettyPrinter(indent = 2)
-host = 'localhost'
-port = 9090
-uri = ''
-framed = False
-ssl = False
-http = False
-argi = 1
-
-if sys.argv[argi] == '-h':
-  parts = sys.argv[argi+1].split(':')
-  host = parts[0]
-  if len(parts) > 1:
-    port = int(parts[1])
-  argi += 2
-
-if sys.argv[argi] == '-u':
-  url = urlparse(sys.argv[argi+1])
-  parts = url[1].split(':')
-  host = parts[0]
-  if len(parts) > 1:
-    port = int(parts[1])
-  else:
-    port = 80
-  uri = url[2]
-  if url[4]:
-    uri += '?%s' % url[4]
-  http = True
-  argi += 2
-
-if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed':
-  framed = True
-  argi += 1
-
-if sys.argv[argi] == '-s' or sys.argv[argi] == '-ssl':
-  ssl = True
-  argi += 1
-
-cmd = sys.argv[argi]
-args = sys.argv[argi+1:]
-
-if http:
-  transport = THttpClient.THttpClient(host, port, uri)
-else:
-  socket = TSSLSocket.TSSLSocket(host, port, validate=False) if ssl else TSocket.TSocket(host, port)
-  if framed:
-    transport = TTransport.TFramedTransport(socket)
-  else:
-    transport = TTransport.TBufferedTransport(socket)
-protocol = TBinaryProtocol.TBinaryProtocol(transport)
-client = ThriftHive.Client(protocol)
-transport.open()
-
-if cmd == 'execute':
-  if len(args) != 1:
-    print('execute requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.execute(args[0],))
-
-elif cmd == 'fetchOne':
-  if len(args) != 0:
-    print('fetchOne requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.fetchOne())
-
-elif cmd == 'fetchN':
-  if len(args) != 1:
-    print('fetchN requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.fetchN(eval(args[0]),))
-
-elif cmd == 'fetchAll':
-  if len(args) != 0:
-    print('fetchAll requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.fetchAll())
-
-elif cmd == 'getSchema':
-  if len(args) != 0:
-    print('getSchema requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getSchema())
-
-elif cmd == 'getThriftSchema':
-  if len(args) != 0:
-    print('getThriftSchema requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getThriftSchema())
-
-elif cmd == 'getClusterStatus':
-  if len(args) != 0:
-    print('getClusterStatus requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getClusterStatus())
-
-elif cmd == 'getQueryPlan':
-  if len(args) != 0:
-    print('getQueryPlan requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getQueryPlan())
-
-elif cmd == 'clean':
-  if len(args) != 0:
-    print('clean requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.clean())
-
-elif cmd == 'getMetaConf':
-  if len(args) != 1:
-    print('getMetaConf requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.getMetaConf(args[0],))
-
-elif cmd == 'setMetaConf':
-  if len(args) != 2:
-    print('setMetaConf requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.setMetaConf(args[0],args[1],))
-
-elif cmd == 'create_database':
-  if len(args) != 1:
-    print('create_database requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.create_database(eval(args[0]),))
-
-elif cmd == 'get_database':
-  if len(args) != 1:
-    print('get_database requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_database(args[0],))
-
-elif cmd == 'drop_database':
-  if len(args) != 3:
-    print('drop_database requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.drop_database(args[0],eval(args[1]),eval(args[2]),))
-
-elif cmd == 'get_databases':
-  if len(args) != 1:
-    print('get_databases requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_databases(args[0],))
-
-elif cmd == 'get_all_databases':
-  if len(args) != 0:
-    print('get_all_databases requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.get_all_databases())
-
-elif cmd == 'alter_database':
-  if len(args) != 2:
-    print('alter_database requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.alter_database(args[0],eval(args[1]),))
-
-elif cmd == 'get_type':
-  if len(args) != 1:
-    print('get_type requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_type(args[0],))
-
-elif cmd == 'create_type':
-  if len(args) != 1:
-    print('create_type requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.create_type(eval(args[0]),))
-
-elif cmd == 'drop_type':
-  if len(args) != 1:
-    print('drop_type requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.drop_type(args[0],))
-
-elif cmd == 'get_type_all':
-  if len(args) != 1:
-    print('get_type_all requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_type_all(args[0],))
-
-elif cmd == 'get_fields':
-  if len(args) != 2:
-    print('get_fields requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_fields(args[0],args[1],))
-
-elif cmd == 'get_fields_with_environment_context':
-  if len(args) != 3:
-    print('get_fields_with_environment_context requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_fields_with_environment_context(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'get_schema':
-  if len(args) != 2:
-    print('get_schema requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_schema(args[0],args[1],))
-
-elif cmd == 'get_schema_with_environment_context':
-  if len(args) != 3:
-    print('get_schema_with_environment_context requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_schema_with_environment_context(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'create_table':
-  if len(args) != 1:
-    print('create_table requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.create_table(eval(args[0]),))
-
-elif cmd == 'create_table_with_environment_context':
-  if len(args) != 2:
-    print('create_table_with_environment_context requires 2 args')
-    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')
-    sys.exit(1)
-  pp.pprint(client.drop_table(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'drop_table_with_environment_context':
-  if len(args) != 4:
-    print('drop_table_with_environment_context requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.drop_table_with_environment_context(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'get_tables':
-  if len(args) != 2:
-    print('get_tables requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_tables(args[0],args[1],))
-
-elif cmd == 'get_table_meta':
-  if len(args) != 3:
-    print('get_table_meta requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_table_meta(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'get_all_tables':
-  if len(args) != 1:
-    print('get_all_tables requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_all_tables(args[0],))
-
-elif cmd == 'get_table':
-  if len(args) != 2:
-    print('get_table requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_table(args[0],args[1],))
-
-elif cmd == 'get_table_objects_by_name':
-  if len(args) != 2:
-    print('get_table_objects_by_name requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_table_objects_by_name(args[0],eval(args[1]),))
-
-elif cmd == 'get_table_names_by_filter':
-  if len(args) != 3:
-    print('get_table_names_by_filter requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_table_names_by_filter(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'alter_table':
-  if len(args) != 3:
-    print('alter_table requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.alter_table(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'alter_table_with_environment_context':
-  if len(args) != 4:
-    print('alter_table_with_environment_context requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.alter_table_with_environment_context(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'alter_table_with_cascade':
-  if len(args) != 4:
-    print('alter_table_with_cascade requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.alter_table_with_cascade(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'add_partition':
-  if len(args) != 1:
-    print('add_partition requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.add_partition(eval(args[0]),))
-
-elif cmd == 'add_partition_with_environment_context':
-  if len(args) != 2:
-    print('add_partition_with_environment_context requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.add_partition_with_environment_context(eval(args[0]),eval(args[1]),))
-
-elif cmd == 'add_partitions':
-  if len(args) != 1:
-    print('add_partitions requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.add_partitions(eval(args[0]),))
-
-elif cmd == 'add_partitions_pspec':
-  if len(args) != 1:
-    print('add_partitions_pspec requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.add_partitions_pspec(eval(args[0]),))
-
-elif cmd == 'append_partition':
-  if len(args) != 3:
-    print('append_partition requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.append_partition(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'add_partitions_req':
-  if len(args) != 1:
-    print('add_partitions_req requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.add_partitions_req(eval(args[0]),))
-
-elif cmd == 'append_partition_with_environment_context':
-  if len(args) != 4:
-    print('append_partition_with_environment_context requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.append_partition_with_environment_context(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'append_partition_by_name':
-  if len(args) != 3:
-    print('append_partition_by_name requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.append_partition_by_name(args[0],args[1],args[2],))
-
-elif cmd == 'append_partition_by_name_with_environment_context':
-  if len(args) != 4:
-    print('append_partition_by_name_with_environment_context requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.append_partition_by_name_with_environment_context(args[0],args[1],args[2],eval(args[3]),))
-
-elif cmd == 'drop_partition':
-  if len(args) != 4:
-    print('drop_partition requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.drop_partition(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'drop_partition_with_environment_context':
-  if len(args) != 5:
-    print('drop_partition_with_environment_context requires 5 args')
-    sys.exit(1)
-  pp.pprint(client.drop_partition_with_environment_context(args[0],args[1],eval(args[2]),eval(args[3]),eval(args[4]),))
-
-elif cmd == 'drop_partition_by_name':
-  if len(args) != 4:
-    print('drop_partition_by_name requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.drop_partition_by_name(args[0],args[1],args[2],eval(args[3]),))
-
-elif cmd == 'drop_partition_by_name_with_environment_context':
-  if len(args) != 5:
-    print('drop_partition_by_name_with_environment_context requires 5 args')
-    sys.exit(1)
-  pp.pprint(client.drop_partition_by_name_with_environment_context(args[0],args[1],args[2],eval(args[3]),eval(args[4]),))
-
-elif cmd == 'drop_partitions_req':
-  if len(args) != 1:
-    print('drop_partitions_req requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.drop_partitions_req(eval(args[0]),))
-
-elif cmd == 'get_partition':
-  if len(args) != 3:
-    print('get_partition requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_partition(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'exchange_partition':
-  if len(args) != 5:
-    print('exchange_partition requires 5 args')
-    sys.exit(1)
-  pp.pprint(client.exchange_partition(eval(args[0]),args[1],args[2],args[3],args[4],))
-
-elif cmd == 'exchange_partitions':
-  if len(args) != 5:
-    print('exchange_partitions requires 5 args')
-    sys.exit(1)
-  pp.pprint(client.exchange_partitions(eval(args[0]),args[1],args[2],args[3],args[4],))
-
-elif cmd == 'get_partition_with_auth':
-  if len(args) != 5:
-    print('get_partition_with_auth requires 5 args')
-    sys.exit(1)
-  pp.pprint(client.get_partition_with_auth(args[0],args[1],eval(args[2]),args[3],eval(args[4]),))
-
-elif cmd == 'get_partition_by_name':
-  if len(args) != 3:
-    print('get_partition_by_name requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_partition_by_name(args[0],args[1],args[2],))
-
-elif cmd == 'get_partitions':
-  if len(args) != 3:
-    print('get_partitions requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'get_partitions_with_auth':
-  if len(args) != 5:
-    print('get_partitions_with_auth requires 5 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions_with_auth(args[0],args[1],eval(args[2]),args[3],eval(args[4]),))
-
-elif cmd == 'get_partitions_pspec':
-  if len(args) != 3:
-    print('get_partitions_pspec requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions_pspec(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'get_partition_names':
-  if len(args) != 3:
-    print('get_partition_names requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_partition_names(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'get_partitions_ps':
-  if len(args) != 4:
-    print('get_partitions_ps requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions_ps(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'get_partitions_ps_with_auth':
-  if len(args) != 6:
-    print('get_partitions_ps_with_auth requires 6 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions_ps_with_auth(args[0],args[1],eval(args[2]),eval(args[3]),args[4],eval(args[5]),))
-
-elif cmd == 'get_partition_names_ps':
-  if len(args) != 4:
-    print('get_partition_names_ps requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.get_partition_names_ps(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'get_partitions_by_filter':
-  if len(args) != 4:
-    print('get_partitions_by_filter requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions_by_filter(args[0],args[1],args[2],eval(args[3]),))
-
-elif cmd == 'get_part_specs_by_filter':
-  if len(args) != 4:
-    print('get_part_specs_by_filter requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.get_part_specs_by_filter(args[0],args[1],args[2],eval(args[3]),))
-
-elif cmd == 'get_partitions_by_expr':
-  if len(args) != 1:
-    print('get_partitions_by_expr requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions_by_expr(eval(args[0]),))
-
-elif cmd == 'get_num_partitions_by_filter':
-  if len(args) != 3:
-    print('get_num_partitions_by_filter requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_num_partitions_by_filter(args[0],args[1],args[2],))
-
-elif cmd == 'get_partitions_by_names':
-  if len(args) != 3:
-    print('get_partitions_by_names requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions_by_names(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'alter_partition':
-  if len(args) != 3:
-    print('alter_partition requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.alter_partition(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'alter_partitions':
-  if len(args) != 3:
-    print('alter_partitions requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.alter_partitions(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'alter_partitions_with_environment_context':
-  if len(args) != 4:
-    print('alter_partitions_with_environment_context requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.alter_partitions_with_environment_context(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'alter_partition_with_environment_context':
-  if len(args) != 4:
-    print('alter_partition_with_environment_context requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.alter_partition_with_environment_context(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'rename_partition':
-  if len(args) != 4:
-    print('rename_partition requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.rename_partition(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'partition_name_has_valid_characters':
-  if len(args) != 2:
-    print('partition_name_has_valid_characters requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.partition_name_has_valid_characters(eval(args[0]),eval(args[1]),))
-
-elif cmd == 'get_config_value':
-  if len(args) != 2:
-    print('get_config_value requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_config_value(args[0],args[1],))
-
-elif cmd == 'partition_name_to_vals':
-  if len(args) != 1:
-    print('partition_name_to_vals requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.partition_name_to_vals(args[0],))
-
-elif cmd == 'partition_name_to_spec':
-  if len(args) != 1:
-    print('partition_name_to_spec requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.partition_name_to_spec(args[0],))
-
-elif cmd == 'markPartitionForEvent':
-  if len(args) != 4:
-    print('markPartitionForEvent requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.markPartitionForEvent(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'isPartitionMarkedForEvent':
-  if len(args) != 4:
-    print('isPartitionMarkedForEvent requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.isPartitionMarkedForEvent(args[0],args[1],eval(args[2]),eval(args[3]),))
-
-elif cmd == 'add_index':
-  if len(args) != 2:
-    print('add_index requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.add_index(eval(args[0]),eval(args[1]),))
-
-elif cmd == 'alter_index':
-  if len(args) != 4:
-    print('alter_index requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.alter_index(args[0],args[1],args[2],eval(args[3]),))
-
-elif cmd == 'drop_index_by_name':
-  if len(args) != 4:
-    print('drop_index_by_name requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.drop_index_by_name(args[0],args[1],args[2],eval(args[3]),))
-
-elif cmd == 'get_index_by_name':
-  if len(args) != 3:
-    print('get_index_by_name requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_index_by_name(args[0],args[1],args[2],))
-
-elif cmd == 'get_indexes':
-  if len(args) != 3:
-    print('get_indexes requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_indexes(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'get_index_names':
-  if len(args) != 3:
-    print('get_index_names requires 3 args')
-    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')
-    sys.exit(1)
-  pp.pprint(client.update_table_column_statistics(eval(args[0]),))
-
-elif cmd == 'update_partition_column_statistics':
-  if len(args) != 1:
-    print('update_partition_column_statistics requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.update_partition_column_statistics(eval(args[0]),))
-
-elif cmd == 'get_table_column_statistics':
-  if len(args) != 3:
-    print('get_table_column_statistics requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_table_column_statistics(args[0],args[1],args[2],))
-
-elif cmd == 'get_partition_column_statistics':
-  if len(args) != 4:
-    print('get_partition_column_statistics requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.get_partition_column_statistics(args[0],args[1],args[2],args[3],))
-
-elif cmd == 'get_table_statistics_req':
-  if len(args) != 1:
-    print('get_table_statistics_req requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_table_statistics_req(eval(args[0]),))
-
-elif cmd == 'get_partitions_statistics_req':
-  if len(args) != 1:
-    print('get_partitions_statistics_req requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_partitions_statistics_req(eval(args[0]),))
-
-elif cmd == 'get_aggr_stats_for':
-  if len(args) != 1:
-    print('get_aggr_stats_for requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_aggr_stats_for(eval(args[0]),))
-
-elif cmd == 'set_aggr_stats_for':
-  if len(args) != 1:
-    print('set_aggr_stats_for requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.set_aggr_stats_for(eval(args[0]),))
-
-elif cmd == 'delete_partition_column_statistics':
-  if len(args) != 4:
-    print('delete_partition_column_statistics requires 4 args')
-    sys.exit(1)
-  pp.pprint(client.delete_partition_column_statistics(args[0],args[1],args[2],args[3],))
-
-elif cmd == 'delete_table_column_statistics':
-  if len(args) != 3:
-    print('delete_table_column_statistics requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.delete_table_column_statistics(args[0],args[1],args[2],))
-
-elif cmd == 'create_function':
-  if len(args) != 1:
-    print('create_function requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.create_function(eval(args[0]),))
-
-elif cmd == 'drop_function':
-  if len(args) != 2:
-    print('drop_function requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.drop_function(args[0],args[1],))
-
-elif cmd == 'alter_function':
-  if len(args) != 3:
-    print('alter_function requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.alter_function(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'get_functions':
-  if len(args) != 2:
-    print('get_functions requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_functions(args[0],args[1],))
-
-elif cmd == 'get_function':
-  if len(args) != 2:
-    print('get_function requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_function(args[0],args[1],))
-
-elif cmd == 'get_all_functions':
-  if len(args) != 0:
-    print('get_all_functions requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.get_all_functions())
-
-elif cmd == 'create_role':
-  if len(args) != 1:
-    print('create_role requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.create_role(eval(args[0]),))
-
-elif cmd == 'drop_role':
-  if len(args) != 1:
-    print('drop_role requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.drop_role(args[0],))
-
-elif cmd == 'get_role_names':
-  if len(args) != 0:
-    print('get_role_names requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.get_role_names())
-
-elif cmd == 'grant_role':
-  if len(args) != 6:
-    print('grant_role requires 6 args')
-    sys.exit(1)
-  pp.pprint(client.grant_role(args[0],args[1],eval(args[2]),args[3],eval(args[4]),eval(args[5]),))
-
-elif cmd == 'revoke_role':
-  if len(args) != 3:
-    print('revoke_role requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.revoke_role(args[0],args[1],eval(args[2]),))
-
-elif cmd == 'list_roles':
-  if len(args) != 2:
-    print('list_roles requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.list_roles(args[0],eval(args[1]),))
-
-elif cmd == 'grant_revoke_role':
-  if len(args) != 1:
-    print('grant_revoke_role requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.grant_revoke_role(eval(args[0]),))
-
-elif cmd == 'get_principals_in_role':
-  if len(args) != 1:
-    print('get_principals_in_role requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_principals_in_role(eval(args[0]),))
-
-elif cmd == 'get_role_grants_for_principal':
-  if len(args) != 1:
-    print('get_role_grants_for_principal requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_role_grants_for_principal(eval(args[0]),))
-
-elif cmd == 'get_privilege_set':
-  if len(args) != 3:
-    print('get_privilege_set requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_privilege_set(eval(args[0]),args[1],eval(args[2]),))
-
-elif cmd == 'list_privileges':
-  if len(args) != 3:
-    print('list_privileges requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.list_privileges(args[0],eval(args[1]),eval(args[2]),))
-
-elif cmd == 'grant_privileges':
-  if len(args) != 1:
-    print('grant_privileges requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.grant_privileges(eval(args[0]),))
-
-elif cmd == 'revoke_privileges':
-  if len(args) != 1:
-    print('revoke_privileges requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.revoke_privileges(eval(args[0]),))
-
-elif cmd == 'grant_revoke_privileges':
-  if len(args) != 1:
-    print('grant_revoke_privileges requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.grant_revoke_privileges(eval(args[0]),))
-
-elif cmd == 'set_ugi':
-  if len(args) != 2:
-    print('set_ugi requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.set_ugi(args[0],eval(args[1]),))
-
-elif cmd == 'get_delegation_token':
-  if len(args) != 2:
-    print('get_delegation_token requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.get_delegation_token(args[0],args[1],))
-
-elif cmd == 'renew_delegation_token':
-  if len(args) != 1:
-    print('renew_delegation_token requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.renew_delegation_token(args[0],))
-
-elif cmd == 'cancel_delegation_token':
-  if len(args) != 1:
-    print('cancel_delegation_token requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.cancel_delegation_token(args[0],))
-
-elif cmd == 'add_token':
-  if len(args) != 2:
-    print('add_token requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.add_token(args[0],args[1],))
-
-elif cmd == 'remove_token':
-  if len(args) != 1:
-    print('remove_token requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.remove_token(args[0],))
-
-elif cmd == 'get_token':
-  if len(args) != 1:
-    print('get_token requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_token(args[0],))
-
-elif cmd == 'get_all_token_identifiers':
-  if len(args) != 0:
-    print('get_all_token_identifiers requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.get_all_token_identifiers())
-
-elif cmd == 'add_master_key':
-  if len(args) != 1:
-    print('add_master_key requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.add_master_key(args[0],))
-
-elif cmd == 'update_master_key':
-  if len(args) != 2:
-    print('update_master_key requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.update_master_key(eval(args[0]),args[1],))
-
-elif cmd == 'remove_master_key':
-  if len(args) != 1:
-    print('remove_master_key requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.remove_master_key(eval(args[0]),))
-
-elif cmd == 'get_master_keys':
-  if len(args) != 0:
-    print('get_master_keys requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.get_master_keys())
-
-elif cmd == 'get_open_txns':
-  if len(args) != 0:
-    print('get_open_txns requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.get_open_txns())
-
-elif cmd == 'get_open_txns_info':
-  if len(args) != 0:
-    print('get_open_txns_info requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.get_open_txns_info())
-
-elif cmd == 'open_txns':
-  if len(args) != 1:
-    print('open_txns requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.open_txns(eval(args[0]),))
-
-elif cmd == 'abort_txn':
-  if len(args) != 1:
-    print('abort_txn requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.abort_txn(eval(args[0]),))
-
-elif cmd == 'commit_txn':
-  if len(args) != 1:
-    print('commit_txn requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.commit_txn(eval(args[0]),))
-
-elif cmd == 'lock':
-  if len(args) != 1:
-    print('lock requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.lock(eval(args[0]),))
-
-elif cmd == 'check_lock':
-  if len(args) != 1:
-    print('check_lock requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.check_lock(eval(args[0]),))
-
-elif cmd == 'unlock':
-  if len(args) != 1:
-    print('unlock requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.unlock(eval(args[0]),))
-
-elif cmd == 'show_locks':
-  if len(args) != 1:
-    print('show_locks requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.show_locks(eval(args[0]),))
-
-elif cmd == 'heartbeat':
-  if len(args) != 1:
-    print('heartbeat requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.heartbeat(eval(args[0]),))
-
-elif cmd == 'heartbeat_txn_range':
-  if len(args) != 1:
-    print('heartbeat_txn_range requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.heartbeat_txn_range(eval(args[0]),))
-
-elif cmd == 'compact':
-  if len(args) != 1:
-    print('compact requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.compact(eval(args[0]),))
-
-elif cmd == 'show_compact':
-  if len(args) != 1:
-    print('show_compact requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.show_compact(eval(args[0]),))
-
-elif cmd == 'add_dynamic_partitions':
-  if len(args) != 1:
-    print('add_dynamic_partitions requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.add_dynamic_partitions(eval(args[0]),))
-
-elif cmd == 'get_next_notification':
-  if len(args) != 1:
-    print('get_next_notification requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_next_notification(eval(args[0]),))
-
-elif cmd == 'get_current_notificationEventId':
-  if len(args) != 0:
-    print('get_current_notificationEventId requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.get_current_notificationEventId())
-
-elif cmd == 'fire_listener_event':
-  if len(args) != 1:
-    print('fire_listener_event requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.fire_listener_event(eval(args[0]),))
-
-elif cmd == 'flushCache':
-  if len(args) != 0:
-    print('flushCache requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.flushCache())
-
-elif cmd == 'get_file_metadata_by_expr':
-  if len(args) != 1:
-    print('get_file_metadata_by_expr requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_file_metadata_by_expr(eval(args[0]),))
-
-elif cmd == 'get_file_metadata':
-  if len(args) != 1:
-    print('get_file_metadata requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_file_metadata(eval(args[0]),))
-
-elif cmd == 'put_file_metadata':
-  if len(args) != 1:
-    print('put_file_metadata requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.put_file_metadata(eval(args[0]),))
-
-elif cmd == 'clear_file_metadata':
-  if len(args) != 1:
-    print('clear_file_metadata requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.clear_file_metadata(eval(args[0]),))
-
-elif cmd == 'cache_file_metadata':
-  if len(args) != 1:
-    print('cache_file_metadata requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.cache_file_metadata(eval(args[0]),))
-
-elif cmd == 'get_change_version':
-  if len(args) != 1:
-    print('get_change_version requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_change_version(eval(args[0]),))
-
-elif cmd == 'getName':
-  if len(args) != 0:
-    print('getName requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getName())
-
-elif cmd == 'getVersion':
-  if len(args) != 0:
-    print('getVersion requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getVersion())
-
-elif cmd == 'getStatus':
-  if len(args) != 0:
-    print('getStatus requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getStatus())
-
-elif cmd == 'getStatusDetails':
-  if len(args) != 0:
-    print('getStatusDetails requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getStatusDetails())
-
-elif cmd == 'getCounters':
-  if len(args) != 0:
-    print('getCounters requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getCounters())
-
-elif cmd == 'getCounter':
-  if len(args) != 1:
-    print('getCounter requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.getCounter(args[0],))
-
-elif cmd == 'setOption':
-  if len(args) != 2:
-    print('setOption requires 2 args')
-    sys.exit(1)
-  pp.pprint(client.setOption(args[0],args[1],))
-
-elif cmd == 'getOption':
-  if len(args) != 1:
-    print('getOption requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.getOption(args[0],))
-
-elif cmd == 'getOptions':
-  if len(args) != 0:
-    print('getOptions requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.getOptions())
-
-elif cmd == 'getCpuProfile':
-  if len(args) != 1:
-    print('getCpuProfile requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.getCpuProfile(eval(args[0]),))
-
-elif cmd == 'aliveSince':
-  if len(args) != 0:
-    print('aliveSince requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.aliveSince())
-
-elif cmd == 'reinitialize':
-  if len(args) != 0:
-    print('reinitialize requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.reinitialize())
-
-elif cmd == 'shutdown':
-  if len(args) != 0:
-    print('shutdown requires 0 args')
-    sys.exit(1)
-  pp.pprint(client.shutdown())
-
-else:
-  print('Unrecognized method %s' % cmd)
-  sys.exit(1)
-
-transport.close()


[20/20] 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/390cb8cd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/390cb8cd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/390cb8cd

Branch: refs/heads/llap
Commit: 390cb8cd0259c1830f5d42c3ca63829bcd7022ec
Parents: ffed821 0ebcd93
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Apr 28 15:09:59 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Apr 28 15:09:59 2016 -0700

----------------------------------------------------------------------
 bin/ext/hiveserver.cmd                          |   48 -
 bin/ext/hiveserver.sh                           |   35 -
 .../common/metrics/common/MetricsConstant.java  |    2 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   23 +-
 data/conf/hive-site.xml                         |    2 +-
 .../hive/hcatalog/streaming/HiveEndPoint.java   |    3 +-
 .../hive/hcatalog/streaming/TestStreaming.java  |    2 +
 .../TestHiveAuthorizerCheckInvocation.java      |   52 +
 .../hive/llap/registry/ServiceInstanceSet.java  |    6 +
 .../registry/impl/LlapFixedRegistryImpl.java    |    5 +
 .../impl/LlapZookeeperRegistryImpl.java         |    5 +
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |    5 +-
 .../llap/cache/LowLevelLrfuCachePolicy.java     |    9 +-
 .../llap/daemon/impl/ContainerRunnerImpl.java   |    7 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |   20 +-
 .../llap/daemon/impl/TaskExecutorService.java   |   53 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |   28 +-
 .../hive/llap/io/api/impl/LlapIoImpl.java       |    1 -
 .../llap/io/decode/EncodedDataConsumer.java     |    5 -
 .../llap/io/encoded/OrcEncodedDataReader.java   |    3 -
 .../hive/llap/io/metadata/OrcMetadataCache.java |   42 +-
 .../hive/llap/metrics/LlapDaemonCacheInfo.java  |    6 +-
 .../llap/metrics/LlapDaemonCacheMetrics.java    |   35 +-
 .../llap/metrics/LlapDaemonExecutorInfo.java    |   23 +-
 .../llap/metrics/LlapDaemonExecutorMetrics.java |  152 +-
 .../hive/llap/metrics/LlapDaemonIOInfo.java     |    4 -
 .../hive/llap/metrics/LlapDaemonIOMetrics.java  |   37 +-
 .../hive/llap/cache/TestOrcMetadataCache.java   |   17 +-
 .../daemon/impl/TestTaskExecutorService.java    |    2 +-
 .../tezplugins/LlapTaskSchedulerService.java    |  234 +-
 .../metrics/LlapTaskSchedulerInfo.java          |   59 +
 .../metrics/LlapTaskSchedulerMetrics.java       |  197 +
 .../TestLlapTaskSchedulerService.java           |    2 +-
 metastore/pom.xml                               |    8 +
 .../hadoop/hive/metastore/ObjectStore.java      |   19 +-
 .../metastore/txn/CompactionTxnHandler.java     |    3 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    2 +
 .../hadoop/hive/metastore/TestObjectStore.java  |   50 +
 pom.xml                                         |    2 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |    2 +-
 .../hadoop/hive/ql/exec/FetchOperator.java      |   16 +-
 .../apache/hadoop/hive/ql/exec/FetchTask.java   |    4 +
 .../ql/optimizer/ColumnPrunerProcFactory.java   |    2 +-
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |    2 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      |    2 +-
 .../hive/ql/parse/ColumnAccessAnalyzer.java     |   31 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   25 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   12 +-
 .../hadoop/hive/ql/parse/TestColumnAccess.java  |   10 +-
 .../clientpositive/lateral_view_noalias.q.out   |  120 +-
 .../results/clientpositive/nonmr_fetch.q.out    |   78 +-
 .../clientpositive/select_dummy_source.q.out    |   38 +-
 .../clientpositive/tez/explainuser_1.q.out      |   14 +-
 .../clientpositive/tez/explainuser_3.q.out      |   38 +-
 .../tez/select_dummy_source.q.out               |   76 +-
 .../results/clientpositive/udf_explode.q.out    |  208 +-
 .../results/clientpositive/udf_inline.q.out     |   42 +-
 .../results/clientpositive/udtf_explode.q.out   |  150 +-
 service/if/hive_service.thrift                  |   86 -
 service/src/gen/thrift/gen-cpp/ThriftHive.cpp   | 3544 --------
 service/src/gen/thrift/gen-cpp/ThriftHive.h     | 1224 ---
 .../gen-cpp/ThriftHive_server.skeleton.cpp      |   84 -
 .../thrift/gen-cpp/hive_service_constants.cpp   |   17 -
 .../gen/thrift/gen-cpp/hive_service_constants.h |   24 -
 .../gen/thrift/gen-cpp/hive_service_types.cpp   |  351 -
 .../src/gen/thrift/gen-cpp/hive_service_types.h |  176 -
 .../hadoop/hive/service/HiveClusterStatus.java  |  901 --
 .../hive/service/HiveServerException.java       |  601 --
 .../hadoop/hive/service/JobTrackerState.java    |   45 -
 .../apache/hadoop/hive/service/ThriftHive.java  | 7784 ------------------
 service/src/gen/thrift/gen-php/ThriftHive.php   | 1943 -----
 service/src/gen/thrift/gen-php/Types.php        |  338 -
 .../gen-py/hive_service/ThriftHive-remote       | 1263 ---
 .../thrift/gen-py/hive_service/ThriftHive.py    | 1674 ----
 .../gen/thrift/gen-py/hive_service/__init__.py  |    1 -
 .../gen/thrift/gen-py/hive_service/constants.py |   11 -
 .../gen/thrift/gen-py/hive_service/ttypes.py    |  260 -
 .../gen/thrift/gen-rb/hive_service_constants.rb |    9 -
 .../src/gen/thrift/gen-rb/hive_service_types.rb |   68 -
 service/src/gen/thrift/gen-rb/thrift_hive.rb    |  555 --
 .../TestScripts.testPrepGit.approved.txt        |    2 -
 .../TestScripts.testPrepHadoop1.approved.txt    |    2 -
 .../TestScripts.testPrepNone.approved.txt       |    2 -
 .../TestScripts.testPrepSvn.approved.txt        |    2 -
 84 files changed, 1191 insertions(+), 21854 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------

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


[16/20] hive git commit: HIVE-13537. Update slf4j version to 1.7.10. (Siddharth Seth, reviewed by Prasanth Jayachandran)

Posted by jd...@apache.org.
HIVE-13537. Update slf4j version to 1.7.10. (Siddharth Seth, 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/b3eb86c3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b3eb86c3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b3eb86c3

Branch: refs/heads/llap
Commit: b3eb86c3c6defe025cb8927fdef443f6af6e4a10
Parents: 58450d1
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 28 11:59:36 2016 +0530
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Apr 28 11:59:36 2016 +0530

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


http://git-wip-us.apache.org/repos/asf/hive/blob/b3eb86c3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a108c96..dff2a72 100644
--- a/pom.xml
+++ b/pom.xml
@@ -168,7 +168,7 @@
     <pig.version>0.12.0</pig.version>
     <protobuf.version>2.5.0</protobuf.version>
     <stax.version>1.0.1</stax.version>
-    <slf4j.version>1.7.5</slf4j.version>
+    <slf4j.version>1.7.10</slf4j.version>
     <ST4.version>4.0.4</ST4.version>
     <tez.version>0.8.3</tez.version>
     <slider.version>0.90.2-incubating</slider.version>


[12/20] hive git commit: HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)

Posted by jd...@apache.org.
HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)


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

Branch: refs/heads/llap
Commit: 6a4e0806a81bbdae56f63d355a18e434767cf9b9
Parents: 815499a
Author: Balint Molnar <bm...@hortonworks.com>
Authored: Wed Apr 27 15:10:24 2016 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Wed Apr 27 15:15:23 2016 -0700

----------------------------------------------------------------------
 bin/ext/hiveserver.cmd                          |   48 -
 bin/ext/hiveserver.sh                           |   35 -
 service/if/hive_service.thrift                  |   86 -
 service/src/gen/thrift/gen-cpp/ThriftHive.cpp   | 3544 --------
 service/src/gen/thrift/gen-cpp/ThriftHive.h     | 1224 ---
 .../gen-cpp/ThriftHive_server.skeleton.cpp      |   84 -
 .../thrift/gen-cpp/hive_service_constants.cpp   |   17 -
 .../gen/thrift/gen-cpp/hive_service_constants.h |   24 -
 .../gen/thrift/gen-cpp/hive_service_types.cpp   |  351 -
 .../src/gen/thrift/gen-cpp/hive_service_types.h |  176 -
 .../hadoop/hive/service/HiveClusterStatus.java  |  901 --
 .../hive/service/HiveServerException.java       |  601 --
 .../hadoop/hive/service/JobTrackerState.java    |   45 -
 .../apache/hadoop/hive/service/ThriftHive.java  | 7784 ------------------
 service/src/gen/thrift/gen-php/ThriftHive.php   | 1943 -----
 service/src/gen/thrift/gen-php/Types.php        |  338 -
 .../gen-py/hive_service/ThriftHive-remote       | 1263 ---
 .../thrift/gen-py/hive_service/ThriftHive.py    | 1674 ----
 .../gen/thrift/gen-py/hive_service/__init__.py  |    1 -
 .../gen/thrift/gen-py/hive_service/constants.py |   11 -
 .../gen/thrift/gen-py/hive_service/ttypes.py    |  260 -
 .../gen/thrift/gen-rb/hive_service_constants.rb |    9 -
 .../src/gen/thrift/gen-rb/hive_service_types.rb |   68 -
 service/src/gen/thrift/gen-rb/thrift_hive.rb    |  555 --
 24 files changed, 21042 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/bin/ext/hiveserver.cmd
----------------------------------------------------------------------
diff --git a/bin/ext/hiveserver.cmd b/bin/ext/hiveserver.cmd
deleted file mode 100644
index 1f0e717..0000000
--- a/bin/ext/hiveserver.cmd
+++ /dev/null
@@ -1,48 +0,0 @@
-@echo off
-@rem Licensed to the Apache Software Foundation (ASF) under one or more
-@rem contributor license agreements.  See the NOTICE file distributed with
-@rem this work for additional information regarding copyright ownership.
-@rem The ASF licenses this file to You under the Apache License, Version 2.0
-@rem (the "License"); you may not use this file except in compliance with
-@rem the License.  You may obtain a copy of the License at
-@rem
-@rem     http://www.apache.org/licenses/LICENSE-2.0
-@rem
-@rem Unless required by applicable law or agreed to in writing, software
-@rem distributed under the License is distributed on an "AS IS" BASIS,
-@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-@rem See the License for the specific language governing permissions and
-@rem limitations under the License.
-
-set CLASS=org.apache.hadoop.hive.service.HiveServer
-pushd %HIVE_LIB%
-for /f %%a IN ('dir /b hive-service-*.jar') do (
-	set JAR=%HIVE_LIB%\%%a
-)
-popd
-
-if [%1]==[hiveserver_help] goto :hiveserver_help
-
-if [%1]==[hiveserver_catservice] goto :hiveserver_catservice
-
-:hiveserver
-  echo "Starting Hive Thrift Server"
-
-  @rem hadoop 20 or newer - skip the aux_jars option and hiveconf
-  call %HIVE_BIN_PATH%\ext\util\execHiveCmd.cmd %CLASS%
-goto :EOF
-
-:hiveserver_help
-	set HIVEARGS=-h
-  goto :hiveserver
-goto :EOF
-
-:hiveserver_catservice
-@echo ^<service^>
-@echo   ^<id^>HiveServer^</id^>
-@echo   ^<name^>HiveServer^</name^>
-@echo   ^<description^>Hadoop HiveServer Service^</description^>
-@echo   ^<executable^>%JAVA_HOME%\bin\java^</executable^>
-@echo   ^<arguments^>%JAVA_HEAP_MAX% %HADOOP_OPTS% %AUX_PARAM% -classpath %CLASSPATH% %CLASS% -hiveconf hive.hadoop.classpath=%HIVE_LIB%\* %HIVE_OPTS%^</arguments^>
-@echo ^</service^>
-goto :EOF

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/bin/ext/hiveserver.sh
----------------------------------------------------------------------
diff --git a/bin/ext/hiveserver.sh b/bin/ext/hiveserver.sh
deleted file mode 100644
index 7335b83..0000000
--- a/bin/ext/hiveserver.sh
+++ /dev/null
@@ -1,35 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-THISSERVICE=hiveserver
-export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} "
-
-hiveserver() {
-  echo "Starting Hive Thrift Server"
-  CLASS=org.apache.hadoop.hive.service.HiveServer
-  if $cygwin; then
-    HIVE_LIB=`cygpath -w "$HIVE_LIB"`
-  fi
-  JAR=${HIVE_LIB}/hive-service-*.jar
-
-  # hadoop 20 or newer - skip the aux_jars option and hiveconf
-
-  exec $HADOOP jar $JAR $CLASS $HIVE_OPTS "$@"
-}
-
-hiveserver_help() {
-  hiveserver -h
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/if/hive_service.thrift
----------------------------------------------------------------------
diff --git a/service/if/hive_service.thrift b/service/if/hive_service.thrift
deleted file mode 100644
index 8f007bc..0000000
--- a/service/if/hive_service.thrift
+++ /dev/null
@@ -1,86 +0,0 @@
-#!/usr/local/bin/thrift -java
-
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-#
-# Thrift Service that the hive service is built on
-#
-
-#
-# TODO: include/thrift is shared among different components. It
-# should not be under metastore.
-
-include "share/fb303/if/fb303.thrift"
-include "metastore/if/hive_metastore.thrift"
-include "ql/if/queryplan.thrift"
-
-namespace java org.apache.hadoop.hive.service
-namespace cpp Apache.Hadoop.Hive
-
-// Enumeration of JobTracker.State                                                                      
-enum JobTrackerState {                                                                   
-  INITIALIZING   = 1,           
-  RUNNING        = 2,                                                      
-}  
-
-// Map-Reduce cluster status information
-struct HiveClusterStatus {
-  1: i32              taskTrackers,
-  2: i32              mapTasks,
-  3: i32              reduceTasks,
-  4: i32              maxMapTasks,
-  5: i32              maxReduceTasks,
-  6: JobTrackerState  state,
-}
-
-exception HiveServerException {
-  1: string message
-  2: i32 errorCode
-  3: string SQLState
-}
-
-# Interface for Thrift Hive Server
-service ThriftHive extends hive_metastore.ThriftHiveMetastore {
-  # Execute a query. Takes a HiveQL string
-  void execute(1:string query) throws(1:HiveServerException ex)
-
-  # Fetch one row. This row is the serialized form
-  # of the result of the query
-  string fetchOne() throws(1:HiveServerException ex)
-
-  # Fetch a given number of rows or remaining number of
-  # rows whichever is smaller.
-  list<string> fetchN(1:i32 numRows) throws(1:HiveServerException ex)
-
-  # Fetch all rows of the query result
-  list<string> fetchAll() throws(1:HiveServerException ex)
-
-  # Get a schema object with fields represented with native Hive types
-  hive_metastore.Schema getSchema() throws(1:HiveServerException ex)
-
-  # Get a schema object with fields represented with Thrift DDL types
-  hive_metastore.Schema getThriftSchema() throws(1:HiveServerException ex)
-  
-  # Get the status information about the Map-Reduce cluster
-  HiveClusterStatus getClusterStatus() throws(1:HiveServerException ex)
-  
-  # Get the queryplan annotated with counter information
-  queryplan.QueryPlan getQueryPlan() throws(1:HiveServerException ex)
-
-  # clean up last Hive query (releasing locks etc.)
-  void clean()
-}


[15/20] hive git commit: HIVE-13346 : LLAP doesn't update metadata priority when reusing from cache; some tweaks in LRFU policy (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by jd...@apache.org.
HIVE-13346 : LLAP doesn't update metadata priority when reusing from cache; some tweaks in LRFU policy (Sergey Shelukhin, 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/58450d12
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/58450d12
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/58450d12

Branch: refs/heads/llap
Commit: 58450d121437b614427bf8dac8c4eca2f7e29365
Parents: c3dd00b
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Apr 27 16:54:05 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Apr 27 17:40:56 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  4 +-
 data/conf/hive-site.xml                         |  2 +-
 .../llap/cache/LowLevelLrfuCachePolicy.java     |  9 +++--
 .../hive/llap/io/metadata/OrcMetadataCache.java | 42 ++++++++++++--------
 .../hive/llap/cache/TestOrcMetadataCache.java   | 17 +++++++-
 5 files changed, 49 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/58450d12/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 49d748c..eeb9b84 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2575,7 +2575,7 @@ public class HiveConf extends Configuration {
         "LLAP IO memory usage; 'cache' (the default) uses data and metadata cache with a\n" +
         "custom off-heap allocator, 'allocator' uses the custom allocator without the caches,\n" +
         "'none' doesn't use either (this mode may result in significant performance degradation)"),
-    LLAP_ALLOCATOR_MIN_ALLOC("hive.llap.io.allocator.alloc.min", "128Kb", new SizeValidator(),
+    LLAP_ALLOCATOR_MIN_ALLOC("hive.llap.io.allocator.alloc.min", "16Kb", new SizeValidator(),
         "Minimum allocation possible from LLAP buddy allocator. Allocations below that are\n" +
         "padded to minimum allocation. For ORC, should generally be the same as the expected\n" +
         "compression buffer size, or next lowest power of 2. Must be a power of 2."),
@@ -2590,7 +2590,7 @@ public class HiveConf extends Configuration {
         "Maximum size for IO allocator or ORC low-level cache.", "hive.llap.io.cache.orc.size"),
     LLAP_ALLOCATOR_DIRECT("hive.llap.io.allocator.direct", true,
         "Whether ORC low-level cache should use direct allocation."),
-    LLAP_USE_LRFU("hive.llap.io.use.lrfu", false,
+    LLAP_USE_LRFU("hive.llap.io.use.lrfu", true,
         "Whether ORC low-level cache should use LRFU cache policy instead of default (FIFO)."),
     LLAP_LRFU_LAMBDA("hive.llap.io.lrfu.lambda", 0.01f,
         "Lambda for ORC low-level cache LRFU cache policy. Must be in [0, 1]. 0 makes LRFU\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/58450d12/data/conf/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/hive-site.xml b/data/conf/hive-site.xml
index cbb5546..041b67d 100644
--- a/data/conf/hive-site.xml
+++ b/data/conf/hive-site.xml
@@ -288,7 +288,7 @@
 
 <property>
   <name>hive.llap.io.use.lrfu</name>
-  <value>false</value>
+  <value>true</value>
 </property>
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/58450d12/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
index bbff3cc..5a0b27f 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
@@ -100,7 +100,11 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
     buffer.priority = F0;
     buffer.lastUpdate = time;
     if (priority == Priority.HIGH) {
-      buffer.priority *= 8; // this is arbitrary
+      // This is arbitrary. Note that metadata may come from a big scan and nuke all the data
+      // from some small frequently accessed tables, because it gets such a large priority boost
+      // to start with. Think of the multiplier as the number of accesses after which the data
+      // becomes more important than some random read-once metadata, in a pure-LFU scheme.
+      buffer.priority *= 3;
     } else {
       assert priority == Priority.NORMAL;
     }
@@ -256,7 +260,7 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
     heap[ix] = buffer;
   }
 
-  // Note: almost never called (unless buffers are very large or we evict a lot).
+  // Note: almost never called (unless buffers are very large or we evict a lot, or LFU).
   private LlapCacheableBuffer evictFromHeapUnderLock(long time) {
     while (true) {
       if (heapSize == 0) return null;
@@ -287,7 +291,6 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
     // down; therefore, we can update priorities of other blocks as we go for part of the heap -
     // we correct any discrepancy w/the parent after expiring priority, and any block we expire
     // the priority for already has lower priority than that of its children.
-    // TODO: avoid expiring priorities if times are close? might be needlessly expensive.
     int ix = buffer.indexInHeap;
     double priority = buffer.priority;
     while (true) {

http://git-wip-us.apache.org/repos/asf/hive/blob/58450d12/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java
index 66713d3..3f4f43b 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.llap.io.metadata;
 
+import org.apache.hadoop.hive.llap.cache.LlapCacheableBuffer;
+
 import java.io.IOException;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -51,15 +53,7 @@ public class OrcMetadataCache {
     memoryManager.reserveMemory(memUsage, false);
     OrcFileMetadata val = metadata.putIfAbsent(metaData.getFileKey(), metaData);
     // See OrcFileMetadata; it is always unlocked, so we just "touch" it here to simulate use.
-    if (val == null) {
-      val = metaData;
-      policy.cache(val, Priority.HIGH);
-    } else {
-      memoryManager.releaseMemory(memUsage);
-      policy.notifyLock(val);
-    }
-    policy.notifyUnlock(val);
-    return val;
+    return touchOnPut(metaData, val, memUsage);
   }
 
   public OrcStripeMetadata putStripeMetadata(OrcStripeMetadata metaData) {
@@ -67,17 +61,22 @@ public class OrcMetadataCache {
     memoryManager.reserveMemory(memUsage, false);
     OrcStripeMetadata val = stripeMetadata.putIfAbsent(metaData.getKey(), metaData);
     // See OrcStripeMetadata; it is always unlocked, so we just "touch" it here to simulate use.
-    if (val == null) {
-      val = metaData;
-      policy.cache(val, Priority.HIGH);
+    return touchOnPut(metaData, val, memUsage);
+  }
+
+  private <T extends LlapCacheableBuffer> T touchOnPut(T newVal, T oldVal, long memUsage) {
+    if (oldVal == null) {
+      oldVal = newVal;
+      policy.cache(oldVal, Priority.HIGH);
     } else {
       memoryManager.releaseMemory(memUsage);
-      policy.notifyLock(val);
+      policy.notifyLock(oldVal);
     }
-    policy.notifyUnlock(val);
-    return val;
+    policy.notifyUnlock(oldVal);
+    return oldVal;
   }
 
+
   public void putIncompleteCbs(Object fileKey, DiskRange[] ranges, long baseOffset) {
     if (estimateErrors == null) return;
     OrcFileEstimateErrors errorData = estimateErrors.get(fileKey);
@@ -110,11 +109,20 @@ public class OrcMetadataCache {
   }
 
   public OrcStripeMetadata getStripeMetadata(OrcBatchKey stripeKey) throws IOException {
-    return stripeMetadata.get(stripeKey);
+    return touchOnGet(stripeMetadata.get(stripeKey));
   }
 
   public OrcFileMetadata getFileMetadata(Object fileKey) throws IOException {
-    return metadata.get(fileKey);
+    return touchOnGet(metadata.get(fileKey));
+  }
+
+
+  private <T extends LlapCacheableBuffer> T touchOnGet(T result) {
+    if (result != null) {
+      policy.notifyLock(result);
+      policy.notifyUnlock(result); // Never locked for eviction; Java object.
+    }
+    return result;
   }
 
   public DiskRangeList getIncompleteCbs(Object fileKey, DiskRangeList ranges, long baseOffset,

http://git-wip-us.apache.org/repos/asf/hive/blob/58450d12/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java
index 3f2e750..40edb28 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java
@@ -29,16 +29,18 @@ import org.junit.Test;
 
 public class TestOrcMetadataCache {
   private static class DummyCachePolicy implements LowLevelCachePolicy {
-    public DummyCachePolicy() {
-    }
+    int lockCount = 0, unlockCount = 0;
 
     public void cache(LlapCacheableBuffer buffer, Priority pri) {
+      ++lockCount;
     }
 
     public void notifyLock(LlapCacheableBuffer buffer) {
+      ++lockCount;
     }
 
     public void notifyUnlock(LlapCacheableBuffer buffer) {
+      ++unlockCount;
     }
 
     public long evictSomeBlocks(long memoryToReserve) {
@@ -54,6 +56,11 @@ public class TestOrcMetadataCache {
 
     public void setParentDebugDumper(LlapOomDebugDump dumper) {
     }
+
+    public void verifyEquals(int i) {
+      assertEquals(i, lockCount);
+      assertEquals(i, unlockCount);
+    }
   }
 
   private static class DummyMemoryManager implements MemoryManager {
@@ -92,14 +99,19 @@ public class TestOrcMetadataCache {
     OrcFileMetadata ofm1 = OrcFileMetadata.createDummy(1), ofm2 = OrcFileMetadata.createDummy(2);
     assertSame(ofm1, cache.putFileMetadata(ofm1));
     assertEquals(1, mm.allocs);
+    cp.verifyEquals(1);
     assertSame(ofm2, cache.putFileMetadata(ofm2));
     assertEquals(2, mm.allocs);
+    cp.verifyEquals(2);
     assertSame(ofm1, cache.getFileMetadata(1));
     assertSame(ofm2, cache.getFileMetadata(2));
+    cp.verifyEquals(4);
     OrcFileMetadata ofm3 = OrcFileMetadata.createDummy(1);
     assertSame(ofm1, cache.putFileMetadata(ofm3));
     assertEquals(2, mm.allocs);
+    cp.verifyEquals(5);
     assertSame(ofm1, cache.getFileMetadata(1));
+    cp.verifyEquals(6);
 
     OrcStripeMetadata osm1 = OrcStripeMetadata.createDummy(1), osm2 = OrcStripeMetadata.createDummy(2);
     assertSame(osm1, cache.putStripeMetadata(osm1));
@@ -112,5 +124,6 @@ public class TestOrcMetadataCache {
     assertSame(osm1, cache.putStripeMetadata(osm3));
     assertEquals(4, mm.allocs);
     assertSame(osm1, cache.getStripeMetadata(osm3.getKey()));
+    cp.verifyEquals(12);
   }
 }


[14/20] hive git commit: HIVE-12959: LLAP: Add task scheduler timeout when no nodes are alive (Prasanth Jayachandran reviewed by Siddharth SetH)

Posted by jd...@apache.org.
HIVE-12959: LLAP: Add task scheduler timeout when no nodes are alive (Prasanth Jayachandran reviewed by Siddharth SetH)


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

Branch: refs/heads/llap
Commit: c3dd00b27afbdcc2ef7b3c16ee1d446da16ff0a5
Parents: ce457a4
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Wed Apr 27 17:55:04 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Wed Apr 27 17:55:04 2016 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   5 +
 .../hive/llap/registry/ServiceInstanceSet.java  |   6 +
 .../registry/impl/LlapFixedRegistryImpl.java    |   5 +
 .../impl/LlapZookeeperRegistryImpl.java         |   5 +
 .../tezplugins/LlapTaskSchedulerService.java    | 112 ++++++++++++++++---
 5 files changed, 117 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c3dd00b2/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 5360ed4..49d748c 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2700,6 +2700,11 @@ public class HiveConf extends Configuration {
       "Sleep duration while waiting to retry connection failures to the AM from the daemon for\n" +
       "the general keep-alive thread (milliseconds).",
       "llap.am.liveness.connection.sleep-between-retries-millis"),
+    LLAP_DAEMON_TASK_SCHEDULER_TIMEOUT_SECONDS(
+        "hive.llap.task.scheduler.timeout.seconds", "60s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Amount of time to wait before failing the query when there are no llap daemons running\n" +
+            "(alive) in the cluster.", "llap.daemon.scheduler.timeout.seconds"),
     LLAP_DAEMON_NUM_EXECUTORS("hive.llap.daemon.num.executors", 4,
       "Number of executors to use in LLAP daemon; essentially, the number of tasks that can be\n" +
       "executed in parallel.", "llap.daemon.num.executors"),

http://git-wip-us.apache.org/repos/asf/hive/blob/c3dd00b2/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java
index 73f94f3..99ead9b 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java
@@ -55,4 +55,10 @@ public interface ServiceInstanceSet {
    */
   public Set<ServiceInstance> getByHost(String host);
 
+  /**
+   * Get number of instances in the currently availabe.
+   *
+   * @return - number of instances
+   */
+  public int size();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c3dd00b2/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
index bd814b9..67443a7 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
@@ -246,6 +246,11 @@ public class LlapFixedRegistryImpl implements ServiceRegistry {
       }
       return byHost;
     }
+
+    @Override
+    public int size() {
+      return instances.size();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/c3dd00b2/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index 6af30d4..d51249a 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@ -493,6 +493,11 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
       }
       return byHost;
     }
+
+    @Override
+    public int size() {
+      return instancesCache.getCurrentData().size();
+    }
   }
 
   private class InstanceStateChangeListener implements PathChildrenCacheListener {

http://git-wip-us.apache.org/repos/asf/hive/blob/c3dd00b2/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
index b57ae1a..5ecbf79 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
@@ -38,6 +38,8 @@ import java.util.concurrent.DelayQueue;
 import java.util.concurrent.Delayed;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -74,6 +76,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.serviceplugins.api.ServicePluginErrorDefaults;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.serviceplugins.api.TaskScheduler;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
@@ -129,7 +132,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
 
   private final Lock scheduleLock = new ReentrantLock();
   private final Condition scheduleCondition = scheduleLock.newCondition();
-  private final AtomicBoolean pendingScheduleInvodations = new AtomicBoolean(false);
+  private final AtomicBoolean pendingScheduleInvocations = new AtomicBoolean(false);
   private final ListeningExecutorService schedulerExecutor;
   private final SchedulerCallable schedulerCallable = new SchedulerCallable();
 
@@ -151,6 +154,13 @@ public class LlapTaskSchedulerService extends TaskScheduler {
   // Per Executor Thread
   private final Resource resourcePerExecutor;
 
+  // when there are no live nodes in the cluster and this timeout elapses the query is failed
+  private final long timeout;
+  private final Lock timeoutLock = new ReentrantLock();
+  private final ScheduledExecutorService timeoutExecutor;
+  private final SchedulerTimeoutMonitor timeoutMonitor;
+  private ScheduledFuture<?> timeoutFuture;
+
   private final LlapRegistryService registry = new LlapRegistryService(false);
 
   private volatile ListenableFuture<Void> nodeEnablerFuture;
@@ -200,6 +210,14 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       this.forceLocation = false;
     }
 
+    this.timeoutMonitor = new SchedulerTimeoutMonitor();
+    this.timeout = HiveConf.getTimeVar(conf,
+        ConfVars.LLAP_DAEMON_TASK_SCHEDULER_TIMEOUT_SECONDS, TimeUnit.MILLISECONDS);
+    this.timeoutExecutor = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LlapTaskSchedulerTimeoutMonitor")
+            .build());
+    this.timeoutFuture = null;
+
     int memoryPerExecutor = (int) (memoryPerInstance / (float) executorsPerInstance);
     int coresPerExecutor = (int) (coresPerInstance / (float) executorsPerInstance);
     this.resourcePerExecutor = Resource.newInstance(memoryPerExecutor, coresPerExecutor);
@@ -210,11 +228,11 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         + " must be defined");
 
     ExecutorService executorServiceRaw =
-        Executors.newFixedThreadPool(1,
+        Executors.newSingleThreadExecutor(
             new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LlapSchedulerNodeEnabler").build());
     nodeEnabledExecutor = MoreExecutors.listeningDecorator(executorServiceRaw);
 
-    ExecutorService schedulerExecutorServiceRaw = Executors.newFixedThreadPool(1,
+    ExecutorService schedulerExecutorServiceRaw = Executors.newSingleThreadExecutor(
         new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LlapScheduler").build());
     schedulerExecutor = MoreExecutors.listeningDecorator(schedulerExecutorServiceRaw);
 
@@ -291,6 +309,45 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       // FIXME: disabling this for now
       // instanceToNodeMap.remove(serviceInstance.getWorkerIdentity());
       LOG.info("Removed node with identity: {}", serviceInstance.getWorkerIdentity());
+      // if there are no more nodes. Signal timeout monitor to start timer
+      if (activeInstances.size() == 0) {
+        LOG.info("No node found. Signalling scheduler timeout monitor thread to start timer.");
+        startTimeoutMonitor();
+      }
+    }
+  }
+
+  private void startTimeoutMonitor() {
+    timeoutLock.lock();
+    try {
+      // If timer is null, start a new one.
+      // If timer has completed during previous invocation, start a new one.
+      // If timer already started and is not completed, leaving it running without resetting it.
+      if ((timeoutFuture == null || (timeoutFuture != null && timeoutFuture.isDone()))
+          && activeInstances.size() == 0) {
+        timeoutFuture = timeoutExecutor.schedule(timeoutMonitor, timeout, TimeUnit.MILLISECONDS);
+        LOG.info("Scheduled timeout monitor task to run after {} ms", timeout);
+      } else {
+        LOG.info("Timeout monitor task not started. Timeout future state: {}, #instances: {}",
+            timeoutFuture == null ? "null" : timeoutFuture.isDone(), activeInstances.size());
+      }
+    } finally {
+      timeoutLock.unlock();
+    }
+  }
+
+  private void stopTimeoutMonitor() {
+    timeoutLock.lock();
+    try {
+      if (timeoutFuture != null && activeInstances.size() != 0 && timeoutFuture.cancel(false)) {
+        LOG.info("Stopped timeout monitor task");
+      } else {
+        LOG.info("Timeout monitor task not stopped. Timeout future state: {}, #instances: {}",
+            timeoutFuture == null ? "null" : timeoutFuture.isDone(), activeInstances.size());
+      }
+      timeoutFuture = null;
+    } finally {
+      timeoutLock.unlock();
     }
   }
 
@@ -305,6 +362,13 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         }
         nodeEnabledExecutor.shutdownNow();
 
+        timeoutExecutor.shutdown();
+        if (timeoutFuture != null) {
+          timeoutFuture.cancel(true);
+          timeoutFuture = null;
+        }
+        timeoutExecutor.shutdownNow();
+
         schedulerCallable.shutdown();
         if (schedulerFuture != null) {
           schedulerFuture.cancel(true);
@@ -567,13 +631,6 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     String[] requestedHosts = request.requestedHosts;
     readLock.lock(); // Read-lock. Not updating any stats at the moment.
     try {
-      // Check if any hosts are active.
-      if (getAvailableResources().getMemory() <= 0) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Refreshing instances since total memory is 0");
-        }
-      }
-
       // If there's no memory available, fail
       if (getTotalResources().getMemory() <= 0) {
         return SELECT_HOST_RESULT_INADEQUATE_TOTAL_CAPACITY;
@@ -657,6 +714,11 @@ public class LlapTaskSchedulerService extends TaskScheduler {
 
   private void addNode(ServiceInstance inst, NodeInfo node) {
     LOG.info("Adding node: " + inst);
+    // we have just added a new node. Signal timeout monitor to reset timer
+    if (activeInstances.size() == 1) {
+      LOG.info("New node added. Signalling scheduler timeout monitor thread to stop timer.");
+      stopTimeoutMonitor();
+    }
     instanceToNodeMap.put(inst.getWorkerIdentity(), node);
     // Trigger scheduling since a new node became available.
     trySchedulingPendingTasks();
@@ -794,7 +856,6 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         Iterator<TaskInfo> taskIter = taskListAtPriority.iterator();
         boolean scheduledAllAtPriority = true;
         while (taskIter.hasNext()) {
-
           // TODO Optimization: Add a check to see if there's any capacity available. No point in
           // walking through all active nodes, if they don't have potential capacity.
 
@@ -807,8 +868,11 @@ public class LlapTaskSchedulerService extends TaskScheduler {
           if (scheduleResult == ScheduleResult.SCHEDULED) {
             taskIter.remove();
           } else {
-            // TODO Handle INADEQUATE_TOTAL_RESOURCES eventually - either by throwin an error immediately,
-            // or waiting for some timeout for new executors and then throwing an error
+            if (scheduleResult == ScheduleResult.INADEQUATE_TOTAL_RESOURCES) {
+              LOG.info("Inadequate total resources before scheduling pending tasks." +
+                  " Signalling scheduler timeout monitor thread to start timer.");
+              startTimeoutMonitor();
+            }
 
             // Try pre-empting a task so that a higher priority task can take it's place.
             // Preempt only if there's no pending preemptions to avoid preempting twice for a task.
@@ -1052,13 +1116,29 @@ public class LlapTaskSchedulerService extends TaskScheduler {
   private void trySchedulingPendingTasks() {
     scheduleLock.lock();
     try {
-      pendingScheduleInvodations.set(true);
+      pendingScheduleInvocations.set(true);
       scheduleCondition.signal();
     } finally {
       scheduleLock.unlock();
     }
   }
 
+  private class SchedulerTimeoutMonitor implements Runnable {
+    private final Logger LOG = LoggerFactory.getLogger(SchedulerTimeoutMonitor.class);
+
+    @Override
+    public void run() {
+      LOG.info("Reporting SERVICE_UNAVAILABLE error as no instances are running");
+      try {
+        getContext().reportError(ServicePluginErrorDefaults.SERVICE_UNAVAILABLE,
+            "No LLAP Daemons are running", getContext().getCurrentDagInfo());
+      } catch (Exception e) {
+        LOG.error("Exception when reporting SERVICE_UNAVAILABLE error for dag: {}",
+            getContext().getCurrentDagInfo().getName(), e);
+      }
+    }
+  }
+
   private class SchedulerCallable implements Callable<Void> {
     private AtomicBoolean isShutdown = new AtomicBoolean(false);
 
@@ -1067,7 +1147,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       while (!isShutdown.get() && !Thread.currentThread().isInterrupted()) {
         scheduleLock.lock();
         try {
-          while (!pendingScheduleInvodations.get()) {
+          while (!pendingScheduleInvocations.get()) {
             scheduleCondition.await();
           }
         } catch (InterruptedException e) {
@@ -1086,7 +1166,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         // will be handled in the next run.
         // A new request may come in right after this is set to false, but before the actual scheduling.
         // This will be handled in this run, but will cause an immediate run after, which is harmless.
-        pendingScheduleInvodations.set(false);
+        pendingScheduleInvocations.set(false);
         // Schedule outside of the scheduleLock - which should only be used to wait on the condition.
         schedulePendingTasks();
       }


[10/20] hive git commit: HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-cpp/ThriftHive.h
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-cpp/ThriftHive.h b/service/src/gen/thrift/gen-cpp/ThriftHive.h
deleted file mode 100644
index 902bd4b..0000000
--- a/service/src/gen/thrift/gen-cpp/ThriftHive.h
+++ /dev/null
@@ -1,1224 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-#ifndef ThriftHive_H
-#define ThriftHive_H
-
-#include <thrift/TDispatchProcessor.h>
-#include <thrift/async/TConcurrentClientSyncInfo.h>
-#include "hive_service_types.h"
-#include "ThriftHiveMetastore.h"
-
-namespace Apache { namespace Hadoop { namespace Hive {
-
-#ifdef _WIN32
-  #pragma warning( push )
-  #pragma warning (disable : 4250 ) //inheriting methods via dominance 
-#endif
-
-class ThriftHiveIf : virtual public  ::Apache::Hadoop::Hive::ThriftHiveMetastoreIf {
- public:
-  virtual ~ThriftHiveIf() {}
-  virtual void execute(const std::string& query) = 0;
-  virtual void fetchOne(std::string& _return) = 0;
-  virtual void fetchN(std::vector<std::string> & _return, const int32_t numRows) = 0;
-  virtual void fetchAll(std::vector<std::string> & _return) = 0;
-  virtual void getSchema( ::Apache::Hadoop::Hive::Schema& _return) = 0;
-  virtual void getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return) = 0;
-  virtual void getClusterStatus(HiveClusterStatus& _return) = 0;
-  virtual void getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return) = 0;
-  virtual void clean() = 0;
-};
-
-class ThriftHiveIfFactory : virtual public  ::Apache::Hadoop::Hive::ThriftHiveMetastoreIfFactory {
- public:
-  typedef ThriftHiveIf Handler;
-
-  virtual ~ThriftHiveIfFactory() {}
-
-  virtual ThriftHiveIf* getHandler(const ::apache::thrift::TConnectionInfo& connInfo) = 0;
-  virtual void releaseHandler( ::facebook::fb303::FacebookServiceIf* /* handler */) = 0;
-};
-
-class ThriftHiveIfSingletonFactory : virtual public ThriftHiveIfFactory {
- public:
-  ThriftHiveIfSingletonFactory(const boost::shared_ptr<ThriftHiveIf>& iface) : iface_(iface) {}
-  virtual ~ThriftHiveIfSingletonFactory() {}
-
-  virtual ThriftHiveIf* getHandler(const ::apache::thrift::TConnectionInfo&) {
-    return iface_.get();
-  }
-  virtual void releaseHandler( ::facebook::fb303::FacebookServiceIf* /* handler */) {}
-
- protected:
-  boost::shared_ptr<ThriftHiveIf> iface_;
-};
-
-class ThriftHiveNull : virtual public ThriftHiveIf , virtual public  ::Apache::Hadoop::Hive::ThriftHiveMetastoreNull {
- public:
-  virtual ~ThriftHiveNull() {}
-  void execute(const std::string& /* query */) {
-    return;
-  }
-  void fetchOne(std::string& /* _return */) {
-    return;
-  }
-  void fetchN(std::vector<std::string> & /* _return */, const int32_t /* numRows */) {
-    return;
-  }
-  void fetchAll(std::vector<std::string> & /* _return */) {
-    return;
-  }
-  void getSchema( ::Apache::Hadoop::Hive::Schema& /* _return */) {
-    return;
-  }
-  void getThriftSchema( ::Apache::Hadoop::Hive::Schema& /* _return */) {
-    return;
-  }
-  void getClusterStatus(HiveClusterStatus& /* _return */) {
-    return;
-  }
-  void getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& /* _return */) {
-    return;
-  }
-  void clean() {
-    return;
-  }
-};
-
-typedef struct _ThriftHive_execute_args__isset {
-  _ThriftHive_execute_args__isset() : query(false) {}
-  bool query :1;
-} _ThriftHive_execute_args__isset;
-
-class ThriftHive_execute_args {
- public:
-
-  ThriftHive_execute_args(const ThriftHive_execute_args&);
-  ThriftHive_execute_args& operator=(const ThriftHive_execute_args&);
-  ThriftHive_execute_args() : query() {
-  }
-
-  virtual ~ThriftHive_execute_args() throw();
-  std::string query;
-
-  _ThriftHive_execute_args__isset __isset;
-
-  void __set_query(const std::string& val);
-
-  bool operator == (const ThriftHive_execute_args & rhs) const
-  {
-    if (!(query == rhs.query))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_execute_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_execute_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_execute_pargs {
- public:
-
-
-  virtual ~ThriftHive_execute_pargs() throw();
-  const std::string* query;
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_execute_result__isset {
-  _ThriftHive_execute_result__isset() : ex(false) {}
-  bool ex :1;
-} _ThriftHive_execute_result__isset;
-
-class ThriftHive_execute_result {
- public:
-
-  ThriftHive_execute_result(const ThriftHive_execute_result&);
-  ThriftHive_execute_result& operator=(const ThriftHive_execute_result&);
-  ThriftHive_execute_result() {
-  }
-
-  virtual ~ThriftHive_execute_result() throw();
-  HiveServerException ex;
-
-  _ThriftHive_execute_result__isset __isset;
-
-  void __set_ex(const HiveServerException& val);
-
-  bool operator == (const ThriftHive_execute_result & rhs) const
-  {
-    if (!(ex == rhs.ex))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_execute_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_execute_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_execute_presult__isset {
-  _ThriftHive_execute_presult__isset() : ex(false) {}
-  bool ex :1;
-} _ThriftHive_execute_presult__isset;
-
-class ThriftHive_execute_presult {
- public:
-
-
-  virtual ~ThriftHive_execute_presult() throw();
-  HiveServerException ex;
-
-  _ThriftHive_execute_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-
-class ThriftHive_fetchOne_args {
- public:
-
-  ThriftHive_fetchOne_args(const ThriftHive_fetchOne_args&);
-  ThriftHive_fetchOne_args& operator=(const ThriftHive_fetchOne_args&);
-  ThriftHive_fetchOne_args() {
-  }
-
-  virtual ~ThriftHive_fetchOne_args() throw();
-
-  bool operator == (const ThriftHive_fetchOne_args & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const ThriftHive_fetchOne_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_fetchOne_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_fetchOne_pargs {
- public:
-
-
-  virtual ~ThriftHive_fetchOne_pargs() throw();
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_fetchOne_result__isset {
-  _ThriftHive_fetchOne_result__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_fetchOne_result__isset;
-
-class ThriftHive_fetchOne_result {
- public:
-
-  ThriftHive_fetchOne_result(const ThriftHive_fetchOne_result&);
-  ThriftHive_fetchOne_result& operator=(const ThriftHive_fetchOne_result&);
-  ThriftHive_fetchOne_result() : success() {
-  }
-
-  virtual ~ThriftHive_fetchOne_result() throw();
-  std::string success;
-  HiveServerException ex;
-
-  _ThriftHive_fetchOne_result__isset __isset;
-
-  void __set_success(const std::string& val);
-
-  void __set_ex(const HiveServerException& val);
-
-  bool operator == (const ThriftHive_fetchOne_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    if (!(ex == rhs.ex))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_fetchOne_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_fetchOne_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_fetchOne_presult__isset {
-  _ThriftHive_fetchOne_presult__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_fetchOne_presult__isset;
-
-class ThriftHive_fetchOne_presult {
- public:
-
-
-  virtual ~ThriftHive_fetchOne_presult() throw();
-  std::string* success;
-  HiveServerException ex;
-
-  _ThriftHive_fetchOne_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-typedef struct _ThriftHive_fetchN_args__isset {
-  _ThriftHive_fetchN_args__isset() : numRows(false) {}
-  bool numRows :1;
-} _ThriftHive_fetchN_args__isset;
-
-class ThriftHive_fetchN_args {
- public:
-
-  ThriftHive_fetchN_args(const ThriftHive_fetchN_args&);
-  ThriftHive_fetchN_args& operator=(const ThriftHive_fetchN_args&);
-  ThriftHive_fetchN_args() : numRows(0) {
-  }
-
-  virtual ~ThriftHive_fetchN_args() throw();
-  int32_t numRows;
-
-  _ThriftHive_fetchN_args__isset __isset;
-
-  void __set_numRows(const int32_t val);
-
-  bool operator == (const ThriftHive_fetchN_args & rhs) const
-  {
-    if (!(numRows == rhs.numRows))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_fetchN_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_fetchN_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_fetchN_pargs {
- public:
-
-
-  virtual ~ThriftHive_fetchN_pargs() throw();
-  const int32_t* numRows;
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_fetchN_result__isset {
-  _ThriftHive_fetchN_result__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_fetchN_result__isset;
-
-class ThriftHive_fetchN_result {
- public:
-
-  ThriftHive_fetchN_result(const ThriftHive_fetchN_result&);
-  ThriftHive_fetchN_result& operator=(const ThriftHive_fetchN_result&);
-  ThriftHive_fetchN_result() {
-  }
-
-  virtual ~ThriftHive_fetchN_result() throw();
-  std::vector<std::string>  success;
-  HiveServerException ex;
-
-  _ThriftHive_fetchN_result__isset __isset;
-
-  void __set_success(const std::vector<std::string> & val);
-
-  void __set_ex(const HiveServerException& val);
-
-  bool operator == (const ThriftHive_fetchN_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    if (!(ex == rhs.ex))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_fetchN_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_fetchN_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_fetchN_presult__isset {
-  _ThriftHive_fetchN_presult__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_fetchN_presult__isset;
-
-class ThriftHive_fetchN_presult {
- public:
-
-
-  virtual ~ThriftHive_fetchN_presult() throw();
-  std::vector<std::string> * success;
-  HiveServerException ex;
-
-  _ThriftHive_fetchN_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-
-class ThriftHive_fetchAll_args {
- public:
-
-  ThriftHive_fetchAll_args(const ThriftHive_fetchAll_args&);
-  ThriftHive_fetchAll_args& operator=(const ThriftHive_fetchAll_args&);
-  ThriftHive_fetchAll_args() {
-  }
-
-  virtual ~ThriftHive_fetchAll_args() throw();
-
-  bool operator == (const ThriftHive_fetchAll_args & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const ThriftHive_fetchAll_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_fetchAll_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_fetchAll_pargs {
- public:
-
-
-  virtual ~ThriftHive_fetchAll_pargs() throw();
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_fetchAll_result__isset {
-  _ThriftHive_fetchAll_result__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_fetchAll_result__isset;
-
-class ThriftHive_fetchAll_result {
- public:
-
-  ThriftHive_fetchAll_result(const ThriftHive_fetchAll_result&);
-  ThriftHive_fetchAll_result& operator=(const ThriftHive_fetchAll_result&);
-  ThriftHive_fetchAll_result() {
-  }
-
-  virtual ~ThriftHive_fetchAll_result() throw();
-  std::vector<std::string>  success;
-  HiveServerException ex;
-
-  _ThriftHive_fetchAll_result__isset __isset;
-
-  void __set_success(const std::vector<std::string> & val);
-
-  void __set_ex(const HiveServerException& val);
-
-  bool operator == (const ThriftHive_fetchAll_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    if (!(ex == rhs.ex))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_fetchAll_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_fetchAll_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_fetchAll_presult__isset {
-  _ThriftHive_fetchAll_presult__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_fetchAll_presult__isset;
-
-class ThriftHive_fetchAll_presult {
- public:
-
-
-  virtual ~ThriftHive_fetchAll_presult() throw();
-  std::vector<std::string> * success;
-  HiveServerException ex;
-
-  _ThriftHive_fetchAll_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-
-class ThriftHive_getSchema_args {
- public:
-
-  ThriftHive_getSchema_args(const ThriftHive_getSchema_args&);
-  ThriftHive_getSchema_args& operator=(const ThriftHive_getSchema_args&);
-  ThriftHive_getSchema_args() {
-  }
-
-  virtual ~ThriftHive_getSchema_args() throw();
-
-  bool operator == (const ThriftHive_getSchema_args & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const ThriftHive_getSchema_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_getSchema_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_getSchema_pargs {
- public:
-
-
-  virtual ~ThriftHive_getSchema_pargs() throw();
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_getSchema_result__isset {
-  _ThriftHive_getSchema_result__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_getSchema_result__isset;
-
-class ThriftHive_getSchema_result {
- public:
-
-  ThriftHive_getSchema_result(const ThriftHive_getSchema_result&);
-  ThriftHive_getSchema_result& operator=(const ThriftHive_getSchema_result&);
-  ThriftHive_getSchema_result() {
-  }
-
-  virtual ~ThriftHive_getSchema_result() throw();
-   ::Apache::Hadoop::Hive::Schema success;
-  HiveServerException ex;
-
-  _ThriftHive_getSchema_result__isset __isset;
-
-  void __set_success(const  ::Apache::Hadoop::Hive::Schema& val);
-
-  void __set_ex(const HiveServerException& val);
-
-  bool operator == (const ThriftHive_getSchema_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    if (!(ex == rhs.ex))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_getSchema_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_getSchema_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_getSchema_presult__isset {
-  _ThriftHive_getSchema_presult__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_getSchema_presult__isset;
-
-class ThriftHive_getSchema_presult {
- public:
-
-
-  virtual ~ThriftHive_getSchema_presult() throw();
-   ::Apache::Hadoop::Hive::Schema* success;
-  HiveServerException ex;
-
-  _ThriftHive_getSchema_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-
-class ThriftHive_getThriftSchema_args {
- public:
-
-  ThriftHive_getThriftSchema_args(const ThriftHive_getThriftSchema_args&);
-  ThriftHive_getThriftSchema_args& operator=(const ThriftHive_getThriftSchema_args&);
-  ThriftHive_getThriftSchema_args() {
-  }
-
-  virtual ~ThriftHive_getThriftSchema_args() throw();
-
-  bool operator == (const ThriftHive_getThriftSchema_args & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const ThriftHive_getThriftSchema_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_getThriftSchema_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_getThriftSchema_pargs {
- public:
-
-
-  virtual ~ThriftHive_getThriftSchema_pargs() throw();
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_getThriftSchema_result__isset {
-  _ThriftHive_getThriftSchema_result__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_getThriftSchema_result__isset;
-
-class ThriftHive_getThriftSchema_result {
- public:
-
-  ThriftHive_getThriftSchema_result(const ThriftHive_getThriftSchema_result&);
-  ThriftHive_getThriftSchema_result& operator=(const ThriftHive_getThriftSchema_result&);
-  ThriftHive_getThriftSchema_result() {
-  }
-
-  virtual ~ThriftHive_getThriftSchema_result() throw();
-   ::Apache::Hadoop::Hive::Schema success;
-  HiveServerException ex;
-
-  _ThriftHive_getThriftSchema_result__isset __isset;
-
-  void __set_success(const  ::Apache::Hadoop::Hive::Schema& val);
-
-  void __set_ex(const HiveServerException& val);
-
-  bool operator == (const ThriftHive_getThriftSchema_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    if (!(ex == rhs.ex))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_getThriftSchema_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_getThriftSchema_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_getThriftSchema_presult__isset {
-  _ThriftHive_getThriftSchema_presult__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_getThriftSchema_presult__isset;
-
-class ThriftHive_getThriftSchema_presult {
- public:
-
-
-  virtual ~ThriftHive_getThriftSchema_presult() throw();
-   ::Apache::Hadoop::Hive::Schema* success;
-  HiveServerException ex;
-
-  _ThriftHive_getThriftSchema_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-
-class ThriftHive_getClusterStatus_args {
- public:
-
-  ThriftHive_getClusterStatus_args(const ThriftHive_getClusterStatus_args&);
-  ThriftHive_getClusterStatus_args& operator=(const ThriftHive_getClusterStatus_args&);
-  ThriftHive_getClusterStatus_args() {
-  }
-
-  virtual ~ThriftHive_getClusterStatus_args() throw();
-
-  bool operator == (const ThriftHive_getClusterStatus_args & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const ThriftHive_getClusterStatus_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_getClusterStatus_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_getClusterStatus_pargs {
- public:
-
-
-  virtual ~ThriftHive_getClusterStatus_pargs() throw();
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_getClusterStatus_result__isset {
-  _ThriftHive_getClusterStatus_result__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_getClusterStatus_result__isset;
-
-class ThriftHive_getClusterStatus_result {
- public:
-
-  ThriftHive_getClusterStatus_result(const ThriftHive_getClusterStatus_result&);
-  ThriftHive_getClusterStatus_result& operator=(const ThriftHive_getClusterStatus_result&);
-  ThriftHive_getClusterStatus_result() {
-  }
-
-  virtual ~ThriftHive_getClusterStatus_result() throw();
-  HiveClusterStatus success;
-  HiveServerException ex;
-
-  _ThriftHive_getClusterStatus_result__isset __isset;
-
-  void __set_success(const HiveClusterStatus& val);
-
-  void __set_ex(const HiveServerException& val);
-
-  bool operator == (const ThriftHive_getClusterStatus_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    if (!(ex == rhs.ex))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_getClusterStatus_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_getClusterStatus_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_getClusterStatus_presult__isset {
-  _ThriftHive_getClusterStatus_presult__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_getClusterStatus_presult__isset;
-
-class ThriftHive_getClusterStatus_presult {
- public:
-
-
-  virtual ~ThriftHive_getClusterStatus_presult() throw();
-  HiveClusterStatus* success;
-  HiveServerException ex;
-
-  _ThriftHive_getClusterStatus_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-
-class ThriftHive_getQueryPlan_args {
- public:
-
-  ThriftHive_getQueryPlan_args(const ThriftHive_getQueryPlan_args&);
-  ThriftHive_getQueryPlan_args& operator=(const ThriftHive_getQueryPlan_args&);
-  ThriftHive_getQueryPlan_args() {
-  }
-
-  virtual ~ThriftHive_getQueryPlan_args() throw();
-
-  bool operator == (const ThriftHive_getQueryPlan_args & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const ThriftHive_getQueryPlan_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_getQueryPlan_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_getQueryPlan_pargs {
- public:
-
-
-  virtual ~ThriftHive_getQueryPlan_pargs() throw();
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_getQueryPlan_result__isset {
-  _ThriftHive_getQueryPlan_result__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_getQueryPlan_result__isset;
-
-class ThriftHive_getQueryPlan_result {
- public:
-
-  ThriftHive_getQueryPlan_result(const ThriftHive_getQueryPlan_result&);
-  ThriftHive_getQueryPlan_result& operator=(const ThriftHive_getQueryPlan_result&);
-  ThriftHive_getQueryPlan_result() {
-  }
-
-  virtual ~ThriftHive_getQueryPlan_result() throw();
-   ::Apache::Hadoop::Hive::QueryPlan success;
-  HiveServerException ex;
-
-  _ThriftHive_getQueryPlan_result__isset __isset;
-
-  void __set_success(const  ::Apache::Hadoop::Hive::QueryPlan& val);
-
-  void __set_ex(const HiveServerException& val);
-
-  bool operator == (const ThriftHive_getQueryPlan_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    if (!(ex == rhs.ex))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHive_getQueryPlan_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_getQueryPlan_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHive_getQueryPlan_presult__isset {
-  _ThriftHive_getQueryPlan_presult__isset() : success(false), ex(false) {}
-  bool success :1;
-  bool ex :1;
-} _ThriftHive_getQueryPlan_presult__isset;
-
-class ThriftHive_getQueryPlan_presult {
- public:
-
-
-  virtual ~ThriftHive_getQueryPlan_presult() throw();
-   ::Apache::Hadoop::Hive::QueryPlan* success;
-  HiveServerException ex;
-
-  _ThriftHive_getQueryPlan_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-
-class ThriftHive_clean_args {
- public:
-
-  ThriftHive_clean_args(const ThriftHive_clean_args&);
-  ThriftHive_clean_args& operator=(const ThriftHive_clean_args&);
-  ThriftHive_clean_args() {
-  }
-
-  virtual ~ThriftHive_clean_args() throw();
-
-  bool operator == (const ThriftHive_clean_args & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const ThriftHive_clean_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_clean_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_clean_pargs {
- public:
-
-
-  virtual ~ThriftHive_clean_pargs() throw();
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_clean_result {
- public:
-
-  ThriftHive_clean_result(const ThriftHive_clean_result&);
-  ThriftHive_clean_result& operator=(const ThriftHive_clean_result&);
-  ThriftHive_clean_result() {
-  }
-
-  virtual ~ThriftHive_clean_result() throw();
-
-  bool operator == (const ThriftHive_clean_result & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const ThriftHive_clean_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHive_clean_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHive_clean_presult {
- public:
-
-
-  virtual ~ThriftHive_clean_presult() throw();
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-class ThriftHiveClient : virtual public ThriftHiveIf, public  ::Apache::Hadoop::Hive::ThriftHiveMetastoreClient {
- public:
-  ThriftHiveClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :
-     ::Apache::Hadoop::Hive::ThriftHiveMetastoreClient(prot, prot) {}
-  ThriftHiveClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, boost::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) :     ::Apache::Hadoop::Hive::ThriftHiveMetastoreClient(iprot, oprot) {}
-  boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() {
-    return piprot_;
-  }
-  boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() {
-    return poprot_;
-  }
-  void execute(const std::string& query);
-  void send_execute(const std::string& query);
-  void recv_execute();
-  void fetchOne(std::string& _return);
-  void send_fetchOne();
-  void recv_fetchOne(std::string& _return);
-  void fetchN(std::vector<std::string> & _return, const int32_t numRows);
-  void send_fetchN(const int32_t numRows);
-  void recv_fetchN(std::vector<std::string> & _return);
-  void fetchAll(std::vector<std::string> & _return);
-  void send_fetchAll();
-  void recv_fetchAll(std::vector<std::string> & _return);
-  void getSchema( ::Apache::Hadoop::Hive::Schema& _return);
-  void send_getSchema();
-  void recv_getSchema( ::Apache::Hadoop::Hive::Schema& _return);
-  void getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return);
-  void send_getThriftSchema();
-  void recv_getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return);
-  void getClusterStatus(HiveClusterStatus& _return);
-  void send_getClusterStatus();
-  void recv_getClusterStatus(HiveClusterStatus& _return);
-  void getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return);
-  void send_getQueryPlan();
-  void recv_getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return);
-  void clean();
-  void send_clean();
-  void recv_clean();
-};
-
-class ThriftHiveProcessor : public  ::Apache::Hadoop::Hive::ThriftHiveMetastoreProcessor {
- protected:
-  boost::shared_ptr<ThriftHiveIf> iface_;
-  virtual bool dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext);
- private:
-  typedef  void (ThriftHiveProcessor::*ProcessFunction)(int32_t, ::apache::thrift::protocol::TProtocol*, ::apache::thrift::protocol::TProtocol*, void*);
-  typedef std::map<std::string, ProcessFunction> ProcessMap;
-  ProcessMap processMap_;
-  void process_execute(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_fetchOne(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_fetchN(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_fetchAll(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_getSchema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_getThriftSchema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_getClusterStatus(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_getQueryPlan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_clean(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
- public:
-  ThriftHiveProcessor(boost::shared_ptr<ThriftHiveIf> iface) :
-     ::Apache::Hadoop::Hive::ThriftHiveMetastoreProcessor(iface),
-    iface_(iface) {
-    processMap_["execute"] = &ThriftHiveProcessor::process_execute;
-    processMap_["fetchOne"] = &ThriftHiveProcessor::process_fetchOne;
-    processMap_["fetchN"] = &ThriftHiveProcessor::process_fetchN;
-    processMap_["fetchAll"] = &ThriftHiveProcessor::process_fetchAll;
-    processMap_["getSchema"] = &ThriftHiveProcessor::process_getSchema;
-    processMap_["getThriftSchema"] = &ThriftHiveProcessor::process_getThriftSchema;
-    processMap_["getClusterStatus"] = &ThriftHiveProcessor::process_getClusterStatus;
-    processMap_["getQueryPlan"] = &ThriftHiveProcessor::process_getQueryPlan;
-    processMap_["clean"] = &ThriftHiveProcessor::process_clean;
-  }
-
-  virtual ~ThriftHiveProcessor() {}
-};
-
-class ThriftHiveProcessorFactory : public ::apache::thrift::TProcessorFactory {
- public:
-  ThriftHiveProcessorFactory(const ::boost::shared_ptr< ThriftHiveIfFactory >& handlerFactory) :
-      handlerFactory_(handlerFactory) {}
-
-  ::boost::shared_ptr< ::apache::thrift::TProcessor > getProcessor(const ::apache::thrift::TConnectionInfo& connInfo);
-
- protected:
-  ::boost::shared_ptr< ThriftHiveIfFactory > handlerFactory_;
-};
-
-class ThriftHiveMultiface : virtual public ThriftHiveIf, public  ::Apache::Hadoop::Hive::ThriftHiveMetastoreMultiface {
- public:
-  ThriftHiveMultiface(std::vector<boost::shared_ptr<ThriftHiveIf> >& ifaces) : ifaces_(ifaces) {
-    std::vector<boost::shared_ptr<ThriftHiveIf> >::iterator iter;
-    for (iter = ifaces.begin(); iter != ifaces.end(); ++iter) {
-       ::Apache::Hadoop::Hive::ThriftHiveMetastoreMultiface::add(*iter);
-    }
-  }
-  virtual ~ThriftHiveMultiface() {}
- protected:
-  std::vector<boost::shared_ptr<ThriftHiveIf> > ifaces_;
-  ThriftHiveMultiface() {}
-  void add(boost::shared_ptr<ThriftHiveIf> iface) {
-     ::Apache::Hadoop::Hive::ThriftHiveMetastoreMultiface::add(iface);
-    ifaces_.push_back(iface);
-  }
- public:
-  void execute(const std::string& query) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->execute(query);
-    }
-    ifaces_[i]->execute(query);
-  }
-
-  void fetchOne(std::string& _return) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->fetchOne(_return);
-    }
-    ifaces_[i]->fetchOne(_return);
-    return;
-  }
-
-  void fetchN(std::vector<std::string> & _return, const int32_t numRows) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->fetchN(_return, numRows);
-    }
-    ifaces_[i]->fetchN(_return, numRows);
-    return;
-  }
-
-  void fetchAll(std::vector<std::string> & _return) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->fetchAll(_return);
-    }
-    ifaces_[i]->fetchAll(_return);
-    return;
-  }
-
-  void getSchema( ::Apache::Hadoop::Hive::Schema& _return) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->getSchema(_return);
-    }
-    ifaces_[i]->getSchema(_return);
-    return;
-  }
-
-  void getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->getThriftSchema(_return);
-    }
-    ifaces_[i]->getThriftSchema(_return);
-    return;
-  }
-
-  void getClusterStatus(HiveClusterStatus& _return) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->getClusterStatus(_return);
-    }
-    ifaces_[i]->getClusterStatus(_return);
-    return;
-  }
-
-  void getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->getQueryPlan(_return);
-    }
-    ifaces_[i]->getQueryPlan(_return);
-    return;
-  }
-
-  void clean() {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->clean();
-    }
-    ifaces_[i]->clean();
-  }
-
-};
-
-// The 'concurrent' client is a thread safe client that correctly handles
-// out of order responses.  It is slower than the regular client, so should
-// only be used when you need to share a connection among multiple threads
-class ThriftHiveConcurrentClient : virtual public ThriftHiveIf, public  ::Apache::Hadoop::Hive::ThriftHiveMetastoreConcurrentClient {
- public:
-  ThriftHiveConcurrentClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :
-     ::Apache::Hadoop::Hive::ThriftHiveMetastoreConcurrentClient(prot, prot) {}
-  ThriftHiveConcurrentClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, boost::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) :     ::Apache::Hadoop::Hive::ThriftHiveMetastoreConcurrentClient(iprot, oprot) {}
-  boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() {
-    return piprot_;
-  }
-  boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() {
-    return poprot_;
-  }
-  void execute(const std::string& query);
-  int32_t send_execute(const std::string& query);
-  void recv_execute(const int32_t seqid);
-  void fetchOne(std::string& _return);
-  int32_t send_fetchOne();
-  void recv_fetchOne(std::string& _return, const int32_t seqid);
-  void fetchN(std::vector<std::string> & _return, const int32_t numRows);
-  int32_t send_fetchN(const int32_t numRows);
-  void recv_fetchN(std::vector<std::string> & _return, const int32_t seqid);
-  void fetchAll(std::vector<std::string> & _return);
-  int32_t send_fetchAll();
-  void recv_fetchAll(std::vector<std::string> & _return, const int32_t seqid);
-  void getSchema( ::Apache::Hadoop::Hive::Schema& _return);
-  int32_t send_getSchema();
-  void recv_getSchema( ::Apache::Hadoop::Hive::Schema& _return, const int32_t seqid);
-  void getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return);
-  int32_t send_getThriftSchema();
-  void recv_getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return, const int32_t seqid);
-  void getClusterStatus(HiveClusterStatus& _return);
-  int32_t send_getClusterStatus();
-  void recv_getClusterStatus(HiveClusterStatus& _return, const int32_t seqid);
-  void getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return);
-  int32_t send_getQueryPlan();
-  void recv_getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return, const int32_t seqid);
-  void clean();
-  int32_t send_clean();
-  void recv_clean(const int32_t seqid);
-};
-
-#ifdef _WIN32
-  #pragma warning( pop )
-#endif
-
-}}} // namespace
-
-#endif

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-cpp/ThriftHive_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-cpp/ThriftHive_server.skeleton.cpp b/service/src/gen/thrift/gen-cpp/ThriftHive_server.skeleton.cpp
deleted file mode 100644
index 3b61b76..0000000
--- a/service/src/gen/thrift/gen-cpp/ThriftHive_server.skeleton.cpp
+++ /dev/null
@@ -1,84 +0,0 @@
-// This autogenerated skeleton file illustrates how to build a server.
-// You should copy it to another filename to avoid overwriting it.
-
-#include "ThriftHive.h"
-#include <thrift/protocol/TBinaryProtocol.h>
-#include <thrift/server/TSimpleServer.h>
-#include <thrift/transport/TServerSocket.h>
-#include <thrift/transport/TBufferTransports.h>
-
-using namespace ::apache::thrift;
-using namespace ::apache::thrift::protocol;
-using namespace ::apache::thrift::transport;
-using namespace ::apache::thrift::server;
-
-using boost::shared_ptr;
-
-using namespace  ::Apache::Hadoop::Hive;
-
-class ThriftHiveHandler : virtual public ThriftHiveIf {
- public:
-  ThriftHiveHandler() {
-    // Your initialization goes here
-  }
-
-  void execute(const std::string& query) {
-    // Your implementation goes here
-    printf("execute\n");
-  }
-
-  void fetchOne(std::string& _return) {
-    // Your implementation goes here
-    printf("fetchOne\n");
-  }
-
-  void fetchN(std::vector<std::string> & _return, const int32_t numRows) {
-    // Your implementation goes here
-    printf("fetchN\n");
-  }
-
-  void fetchAll(std::vector<std::string> & _return) {
-    // Your implementation goes here
-    printf("fetchAll\n");
-  }
-
-  void getSchema( ::Apache::Hadoop::Hive::Schema& _return) {
-    // Your implementation goes here
-    printf("getSchema\n");
-  }
-
-  void getThriftSchema( ::Apache::Hadoop::Hive::Schema& _return) {
-    // Your implementation goes here
-    printf("getThriftSchema\n");
-  }
-
-  void getClusterStatus(HiveClusterStatus& _return) {
-    // Your implementation goes here
-    printf("getClusterStatus\n");
-  }
-
-  void getQueryPlan( ::Apache::Hadoop::Hive::QueryPlan& _return) {
-    // Your implementation goes here
-    printf("getQueryPlan\n");
-  }
-
-  void clean() {
-    // Your implementation goes here
-    printf("clean\n");
-  }
-
-};
-
-int main(int argc, char **argv) {
-  int port = 9090;
-  shared_ptr<ThriftHiveHandler> handler(new ThriftHiveHandler());
-  shared_ptr<TProcessor> processor(new ThriftHiveProcessor(handler));
-  shared_ptr<TServerTransport> serverTransport(new TServerSocket(port));
-  shared_ptr<TTransportFactory> transportFactory(new TBufferedTransportFactory());
-  shared_ptr<TProtocolFactory> protocolFactory(new TBinaryProtocolFactory());
-
-  TSimpleServer server(processor, serverTransport, transportFactory, protocolFactory);
-  server.serve();
-  return 0;
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-cpp/hive_service_constants.cpp
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-cpp/hive_service_constants.cpp b/service/src/gen/thrift/gen-cpp/hive_service_constants.cpp
deleted file mode 100644
index e2bbe71..0000000
--- a/service/src/gen/thrift/gen-cpp/hive_service_constants.cpp
+++ /dev/null
@@ -1,17 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-#include "hive_service_constants.h"
-
-namespace Apache { namespace Hadoop { namespace Hive {
-
-const hive_serviceConstants g_hive_service_constants;
-
-hive_serviceConstants::hive_serviceConstants() {
-}
-
-}}} // namespace
-

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-cpp/hive_service_constants.h
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-cpp/hive_service_constants.h b/service/src/gen/thrift/gen-cpp/hive_service_constants.h
deleted file mode 100644
index e0887f4..0000000
--- a/service/src/gen/thrift/gen-cpp/hive_service_constants.h
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-#ifndef hive_service_CONSTANTS_H
-#define hive_service_CONSTANTS_H
-
-#include "hive_service_types.h"
-
-namespace Apache { namespace Hadoop { namespace Hive {
-
-class hive_serviceConstants {
- public:
-  hive_serviceConstants();
-
-};
-
-extern const hive_serviceConstants g_hive_service_constants;
-
-}}} // namespace
-
-#endif

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-cpp/hive_service_types.cpp
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-cpp/hive_service_types.cpp b/service/src/gen/thrift/gen-cpp/hive_service_types.cpp
deleted file mode 100644
index 9ddf7c1..0000000
--- a/service/src/gen/thrift/gen-cpp/hive_service_types.cpp
+++ /dev/null
@@ -1,351 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-#include "hive_service_types.h"
-
-#include <algorithm>
-#include <ostream>
-
-#include <thrift/TToString.h>
-
-namespace Apache { namespace Hadoop { namespace Hive {
-
-int _kJobTrackerStateValues[] = {
-  JobTrackerState::INITIALIZING,
-  JobTrackerState::RUNNING
-};
-const char* _kJobTrackerStateNames[] = {
-  "INITIALIZING",
-  "RUNNING"
-};
-const std::map<int, const char*> _JobTrackerState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kJobTrackerStateValues, _kJobTrackerStateNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
-
-
-HiveClusterStatus::~HiveClusterStatus() throw() {
-}
-
-
-void HiveClusterStatus::__set_taskTrackers(const int32_t val) {
-  this->taskTrackers = val;
-}
-
-void HiveClusterStatus::__set_mapTasks(const int32_t val) {
-  this->mapTasks = val;
-}
-
-void HiveClusterStatus::__set_reduceTasks(const int32_t val) {
-  this->reduceTasks = val;
-}
-
-void HiveClusterStatus::__set_maxMapTasks(const int32_t val) {
-  this->maxMapTasks = val;
-}
-
-void HiveClusterStatus::__set_maxReduceTasks(const int32_t val) {
-  this->maxReduceTasks = val;
-}
-
-void HiveClusterStatus::__set_state(const JobTrackerState::type val) {
-  this->state = val;
-}
-
-uint32_t HiveClusterStatus::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_I32) {
-          xfer += iprot->readI32(this->taskTrackers);
-          this->__isset.taskTrackers = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->mapTasks);
-          this->__isset.mapTasks = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->reduceTasks);
-          this->__isset.reduceTasks = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 4:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->maxMapTasks);
-          this->__isset.maxMapTasks = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 5:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->maxReduceTasks);
-          this->__isset.maxReduceTasks = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 6:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast0;
-          xfer += iprot->readI32(ecast0);
-          this->state = (JobTrackerState::type)ecast0;
-          this->__isset.state = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t HiveClusterStatus::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("HiveClusterStatus");
-
-  xfer += oprot->writeFieldBegin("taskTrackers", ::apache::thrift::protocol::T_I32, 1);
-  xfer += oprot->writeI32(this->taskTrackers);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("mapTasks", ::apache::thrift::protocol::T_I32, 2);
-  xfer += oprot->writeI32(this->mapTasks);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("reduceTasks", ::apache::thrift::protocol::T_I32, 3);
-  xfer += oprot->writeI32(this->reduceTasks);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("maxMapTasks", ::apache::thrift::protocol::T_I32, 4);
-  xfer += oprot->writeI32(this->maxMapTasks);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("maxReduceTasks", ::apache::thrift::protocol::T_I32, 5);
-  xfer += oprot->writeI32(this->maxReduceTasks);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_I32, 6);
-  xfer += oprot->writeI32((int32_t)this->state);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-void swap(HiveClusterStatus &a, HiveClusterStatus &b) {
-  using ::std::swap;
-  swap(a.taskTrackers, b.taskTrackers);
-  swap(a.mapTasks, b.mapTasks);
-  swap(a.reduceTasks, b.reduceTasks);
-  swap(a.maxMapTasks, b.maxMapTasks);
-  swap(a.maxReduceTasks, b.maxReduceTasks);
-  swap(a.state, b.state);
-  swap(a.__isset, b.__isset);
-}
-
-HiveClusterStatus::HiveClusterStatus(const HiveClusterStatus& other1) {
-  taskTrackers = other1.taskTrackers;
-  mapTasks = other1.mapTasks;
-  reduceTasks = other1.reduceTasks;
-  maxMapTasks = other1.maxMapTasks;
-  maxReduceTasks = other1.maxReduceTasks;
-  state = other1.state;
-  __isset = other1.__isset;
-}
-HiveClusterStatus& HiveClusterStatus::operator=(const HiveClusterStatus& other2) {
-  taskTrackers = other2.taskTrackers;
-  mapTasks = other2.mapTasks;
-  reduceTasks = other2.reduceTasks;
-  maxMapTasks = other2.maxMapTasks;
-  maxReduceTasks = other2.maxReduceTasks;
-  state = other2.state;
-  __isset = other2.__isset;
-  return *this;
-}
-void HiveClusterStatus::printTo(std::ostream& out) const {
-  using ::apache::thrift::to_string;
-  out << "HiveClusterStatus(";
-  out << "taskTrackers=" << to_string(taskTrackers);
-  out << ", " << "mapTasks=" << to_string(mapTasks);
-  out << ", " << "reduceTasks=" << to_string(reduceTasks);
-  out << ", " << "maxMapTasks=" << to_string(maxMapTasks);
-  out << ", " << "maxReduceTasks=" << to_string(maxReduceTasks);
-  out << ", " << "state=" << to_string(state);
-  out << ")";
-}
-
-
-HiveServerException::~HiveServerException() throw() {
-}
-
-
-void HiveServerException::__set_message(const std::string& val) {
-  this->message = val;
-}
-
-void HiveServerException::__set_errorCode(const int32_t val) {
-  this->errorCode = val;
-}
-
-void HiveServerException::__set_SQLState(const std::string& val) {
-  this->SQLState = val;
-}
-
-uint32_t HiveServerException::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->message);
-          this->__isset.message = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->errorCode);
-          this->__isset.errorCode = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->SQLState);
-          this->__isset.SQLState = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t HiveServerException::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("HiveServerException");
-
-  xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->message);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("errorCode", ::apache::thrift::protocol::T_I32, 2);
-  xfer += oprot->writeI32(this->errorCode);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("SQLState", ::apache::thrift::protocol::T_STRING, 3);
-  xfer += oprot->writeString(this->SQLState);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-void swap(HiveServerException &a, HiveServerException &b) {
-  using ::std::swap;
-  swap(a.message, b.message);
-  swap(a.errorCode, b.errorCode);
-  swap(a.SQLState, b.SQLState);
-  swap(a.__isset, b.__isset);
-}
-
-HiveServerException::HiveServerException(const HiveServerException& other3) : TException() {
-  message = other3.message;
-  errorCode = other3.errorCode;
-  SQLState = other3.SQLState;
-  __isset = other3.__isset;
-}
-HiveServerException& HiveServerException::operator=(const HiveServerException& other4) {
-  message = other4.message;
-  errorCode = other4.errorCode;
-  SQLState = other4.SQLState;
-  __isset = other4.__isset;
-  return *this;
-}
-void HiveServerException::printTo(std::ostream& out) const {
-  using ::apache::thrift::to_string;
-  out << "HiveServerException(";
-  out << "message=" << to_string(message);
-  out << ", " << "errorCode=" << to_string(errorCode);
-  out << ", " << "SQLState=" << to_string(SQLState);
-  out << ")";
-}
-
-const char* HiveServerException::what() const throw() {
-  try {
-    std::stringstream ss;
-    ss << "TException - service has thrown: " << *this;
-    this->thriftTExceptionMessageHolder_ = ss.str();
-    return this->thriftTExceptionMessageHolder_.c_str();
-  } catch (const std::exception&) {
-    return "TException - service has thrown: HiveServerException";
-  }
-}
-
-}}} // namespace

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-cpp/hive_service_types.h
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-cpp/hive_service_types.h b/service/src/gen/thrift/gen-cpp/hive_service_types.h
deleted file mode 100644
index 266f8ea..0000000
--- a/service/src/gen/thrift/gen-cpp/hive_service_types.h
+++ /dev/null
@@ -1,176 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-#ifndef hive_service_TYPES_H
-#define hive_service_TYPES_H
-
-#include <iosfwd>
-
-#include <thrift/Thrift.h>
-#include <thrift/TApplicationException.h>
-#include <thrift/protocol/TProtocol.h>
-#include <thrift/transport/TTransport.h>
-
-#include <thrift/cxxfunctional.h>
-#include "fb303_types.h"
-#include "hive_metastore_types.h"
-#include "queryplan_types.h"
-
-
-namespace Apache { namespace Hadoop { namespace Hive {
-
-struct JobTrackerState {
-  enum type {
-    INITIALIZING = 1,
-    RUNNING = 2
-  };
-};
-
-extern const std::map<int, const char*> _JobTrackerState_VALUES_TO_NAMES;
-
-class HiveClusterStatus;
-
-class HiveServerException;
-
-typedef struct _HiveClusterStatus__isset {
-  _HiveClusterStatus__isset() : taskTrackers(false), mapTasks(false), reduceTasks(false), maxMapTasks(false), maxReduceTasks(false), state(false) {}
-  bool taskTrackers :1;
-  bool mapTasks :1;
-  bool reduceTasks :1;
-  bool maxMapTasks :1;
-  bool maxReduceTasks :1;
-  bool state :1;
-} _HiveClusterStatus__isset;
-
-class HiveClusterStatus {
- public:
-
-  HiveClusterStatus(const HiveClusterStatus&);
-  HiveClusterStatus& operator=(const HiveClusterStatus&);
-  HiveClusterStatus() : taskTrackers(0), mapTasks(0), reduceTasks(0), maxMapTasks(0), maxReduceTasks(0), state((JobTrackerState::type)0) {
-  }
-
-  virtual ~HiveClusterStatus() throw();
-  int32_t taskTrackers;
-  int32_t mapTasks;
-  int32_t reduceTasks;
-  int32_t maxMapTasks;
-  int32_t maxReduceTasks;
-  JobTrackerState::type state;
-
-  _HiveClusterStatus__isset __isset;
-
-  void __set_taskTrackers(const int32_t val);
-
-  void __set_mapTasks(const int32_t val);
-
-  void __set_reduceTasks(const int32_t val);
-
-  void __set_maxMapTasks(const int32_t val);
-
-  void __set_maxReduceTasks(const int32_t val);
-
-  void __set_state(const JobTrackerState::type val);
-
-  bool operator == (const HiveClusterStatus & rhs) const
-  {
-    if (!(taskTrackers == rhs.taskTrackers))
-      return false;
-    if (!(mapTasks == rhs.mapTasks))
-      return false;
-    if (!(reduceTasks == rhs.reduceTasks))
-      return false;
-    if (!(maxMapTasks == rhs.maxMapTasks))
-      return false;
-    if (!(maxReduceTasks == rhs.maxReduceTasks))
-      return false;
-    if (!(state == rhs.state))
-      return false;
-    return true;
-  }
-  bool operator != (const HiveClusterStatus &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const HiveClusterStatus & ) 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(HiveClusterStatus &a, HiveClusterStatus &b);
-
-inline std::ostream& operator<<(std::ostream& out, const HiveClusterStatus& obj)
-{
-  obj.printTo(out);
-  return out;
-}
-
-typedef struct _HiveServerException__isset {
-  _HiveServerException__isset() : message(false), errorCode(false), SQLState(false) {}
-  bool message :1;
-  bool errorCode :1;
-  bool SQLState :1;
-} _HiveServerException__isset;
-
-class HiveServerException : public ::apache::thrift::TException {
- public:
-
-  HiveServerException(const HiveServerException&);
-  HiveServerException& operator=(const HiveServerException&);
-  HiveServerException() : message(), errorCode(0), SQLState() {
-  }
-
-  virtual ~HiveServerException() throw();
-  std::string message;
-  int32_t errorCode;
-  std::string SQLState;
-
-  _HiveServerException__isset __isset;
-
-  void __set_message(const std::string& val);
-
-  void __set_errorCode(const int32_t val);
-
-  void __set_SQLState(const std::string& val);
-
-  bool operator == (const HiveServerException & rhs) const
-  {
-    if (!(message == rhs.message))
-      return false;
-    if (!(errorCode == rhs.errorCode))
-      return false;
-    if (!(SQLState == rhs.SQLState))
-      return false;
-    return true;
-  }
-  bool operator != (const HiveServerException &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const HiveServerException & ) 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;
-  mutable std::string thriftTExceptionMessageHolder_;
-  const char* what() const throw();
-};
-
-void swap(HiveServerException &a, HiveServerException &b);
-
-inline std::ostream& operator<<(std::ostream& out, const HiveServerException& obj)
-{
-  obj.printTo(out);
-  return out;
-}
-
-}}} // namespace
-
-#endif

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java
deleted file mode 100644
index ad89867..0000000
--- a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java
+++ /dev/null
@@ -1,901 +0,0 @@
-/**
- * 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.service;
-
-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 HiveClusterStatus implements org.apache.thrift.TBase<HiveClusterStatus, HiveClusterStatus._Fields>, java.io.Serializable, Cloneable, Comparable<HiveClusterStatus> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HiveClusterStatus");
-
-  private static final org.apache.thrift.protocol.TField TASK_TRACKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("taskTrackers", org.apache.thrift.protocol.TType.I32, (short)1);
-  private static final org.apache.thrift.protocol.TField MAP_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("mapTasks", org.apache.thrift.protocol.TType.I32, (short)2);
-  private static final org.apache.thrift.protocol.TField REDUCE_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("reduceTasks", org.apache.thrift.protocol.TType.I32, (short)3);
-  private static final org.apache.thrift.protocol.TField MAX_MAP_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxMapTasks", org.apache.thrift.protocol.TType.I32, (short)4);
-  private static final org.apache.thrift.protocol.TField MAX_REDUCE_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxReduceTasks", org.apache.thrift.protocol.TType.I32, (short)5);
-  private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.I32, (short)6);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new HiveClusterStatusStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new HiveClusterStatusTupleSchemeFactory());
-  }
-
-  private int taskTrackers; // required
-  private int mapTasks; // required
-  private int reduceTasks; // required
-  private int maxMapTasks; // required
-  private int maxReduceTasks; // required
-  private JobTrackerState state; // 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 {
-    TASK_TRACKERS((short)1, "taskTrackers"),
-    MAP_TASKS((short)2, "mapTasks"),
-    REDUCE_TASKS((short)3, "reduceTasks"),
-    MAX_MAP_TASKS((short)4, "maxMapTasks"),
-    MAX_REDUCE_TASKS((short)5, "maxReduceTasks"),
-    /**
-     * 
-     * @see JobTrackerState
-     */
-    STATE((short)6, "state");
-
-    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: // TASK_TRACKERS
-          return TASK_TRACKERS;
-        case 2: // MAP_TASKS
-          return MAP_TASKS;
-        case 3: // REDUCE_TASKS
-          return REDUCE_TASKS;
-        case 4: // MAX_MAP_TASKS
-          return MAX_MAP_TASKS;
-        case 5: // MAX_REDUCE_TASKS
-          return MAX_REDUCE_TASKS;
-        case 6: // STATE
-          return STATE;
-        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 __TASKTRACKERS_ISSET_ID = 0;
-  private static final int __MAPTASKS_ISSET_ID = 1;
-  private static final int __REDUCETASKS_ISSET_ID = 2;
-  private static final int __MAXMAPTASKS_ISSET_ID = 3;
-  private static final int __MAXREDUCETASKS_ISSET_ID = 4;
-  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.TASK_TRACKERS, new org.apache.thrift.meta_data.FieldMetaData("taskTrackers", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.MAP_TASKS, new org.apache.thrift.meta_data.FieldMetaData("mapTasks", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.REDUCE_TASKS, new org.apache.thrift.meta_data.FieldMetaData("reduceTasks", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.MAX_MAP_TASKS, new org.apache.thrift.meta_data.FieldMetaData("maxMapTasks", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.MAX_REDUCE_TASKS, new org.apache.thrift.meta_data.FieldMetaData("maxReduceTasks", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, JobTrackerState.class)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HiveClusterStatus.class, metaDataMap);
-  }
-
-  public HiveClusterStatus() {
-  }
-
-  public HiveClusterStatus(
-    int taskTrackers,
-    int mapTasks,
-    int reduceTasks,
-    int maxMapTasks,
-    int maxReduceTasks,
-    JobTrackerState state)
-  {
-    this();
-    this.taskTrackers = taskTrackers;
-    setTaskTrackersIsSet(true);
-    this.mapTasks = mapTasks;
-    setMapTasksIsSet(true);
-    this.reduceTasks = reduceTasks;
-    setReduceTasksIsSet(true);
-    this.maxMapTasks = maxMapTasks;
-    setMaxMapTasksIsSet(true);
-    this.maxReduceTasks = maxReduceTasks;
-    setMaxReduceTasksIsSet(true);
-    this.state = state;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public HiveClusterStatus(HiveClusterStatus other) {
-    __isset_bitfield = other.__isset_bitfield;
-    this.taskTrackers = other.taskTrackers;
-    this.mapTasks = other.mapTasks;
-    this.reduceTasks = other.reduceTasks;
-    this.maxMapTasks = other.maxMapTasks;
-    this.maxReduceTasks = other.maxReduceTasks;
-    if (other.isSetState()) {
-      this.state = other.state;
-    }
-  }
-
-  public HiveClusterStatus deepCopy() {
-    return new HiveClusterStatus(this);
-  }
-
-  @Override
-  public void clear() {
-    setTaskTrackersIsSet(false);
-    this.taskTrackers = 0;
-    setMapTasksIsSet(false);
-    this.mapTasks = 0;
-    setReduceTasksIsSet(false);
-    this.reduceTasks = 0;
-    setMaxMapTasksIsSet(false);
-    this.maxMapTasks = 0;
-    setMaxReduceTasksIsSet(false);
-    this.maxReduceTasks = 0;
-    this.state = null;
-  }
-
-  public int getTaskTrackers() {
-    return this.taskTrackers;
-  }
-
-  public void setTaskTrackers(int taskTrackers) {
-    this.taskTrackers = taskTrackers;
-    setTaskTrackersIsSet(true);
-  }
-
-  public void unsetTaskTrackers() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TASKTRACKERS_ISSET_ID);
-  }
-
-  /** Returns true if field taskTrackers is set (has been assigned a value) and false otherwise */
-  public boolean isSetTaskTrackers() {
-    return EncodingUtils.testBit(__isset_bitfield, __TASKTRACKERS_ISSET_ID);
-  }
-
-  public void setTaskTrackersIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TASKTRACKERS_ISSET_ID, value);
-  }
-
-  public int getMapTasks() {
-    return this.mapTasks;
-  }
-
-  public void setMapTasks(int mapTasks) {
-    this.mapTasks = mapTasks;
-    setMapTasksIsSet(true);
-  }
-
-  public void unsetMapTasks() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAPTASKS_ISSET_ID);
-  }
-
-  /** Returns true if field mapTasks is set (has been assigned a value) and false otherwise */
-  public boolean isSetMapTasks() {
-    return EncodingUtils.testBit(__isset_bitfield, __MAPTASKS_ISSET_ID);
-  }
-
-  public void setMapTasksIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAPTASKS_ISSET_ID, value);
-  }
-
-  public int getReduceTasks() {
-    return this.reduceTasks;
-  }
-
-  public void setReduceTasks(int reduceTasks) {
-    this.reduceTasks = reduceTasks;
-    setReduceTasksIsSet(true);
-  }
-
-  public void unsetReduceTasks() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REDUCETASKS_ISSET_ID);
-  }
-
-  /** Returns true if field reduceTasks is set (has been assigned a value) and false otherwise */
-  public boolean isSetReduceTasks() {
-    return EncodingUtils.testBit(__isset_bitfield, __REDUCETASKS_ISSET_ID);
-  }
-
-  public void setReduceTasksIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REDUCETASKS_ISSET_ID, value);
-  }
-
-  public int getMaxMapTasks() {
-    return this.maxMapTasks;
-  }
-
-  public void setMaxMapTasks(int maxMapTasks) {
-    this.maxMapTasks = maxMapTasks;
-    setMaxMapTasksIsSet(true);
-  }
-
-  public void unsetMaxMapTasks() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXMAPTASKS_ISSET_ID);
-  }
-
-  /** Returns true if field maxMapTasks is set (has been assigned a value) and false otherwise */
-  public boolean isSetMaxMapTasks() {
-    return EncodingUtils.testBit(__isset_bitfield, __MAXMAPTASKS_ISSET_ID);
-  }
-
-  public void setMaxMapTasksIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXMAPTASKS_ISSET_ID, value);
-  }
-
-  public int getMaxReduceTasks() {
-    return this.maxReduceTasks;
-  }
-
-  public void setMaxReduceTasks(int maxReduceTasks) {
-    this.maxReduceTasks = maxReduceTasks;
-    setMaxReduceTasksIsSet(true);
-  }
-
-  public void unsetMaxReduceTasks() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXREDUCETASKS_ISSET_ID);
-  }
-
-  /** Returns true if field maxReduceTasks is set (has been assigned a value) and false otherwise */
-  public boolean isSetMaxReduceTasks() {
-    return EncodingUtils.testBit(__isset_bitfield, __MAXREDUCETASKS_ISSET_ID);
-  }
-
-  public void setMaxReduceTasksIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXREDUCETASKS_ISSET_ID, value);
-  }
-
-  /**
-   * 
-   * @see JobTrackerState
-   */
-  public JobTrackerState getState() {
-    return this.state;
-  }
-
-  /**
-   * 
-   * @see JobTrackerState
-   */
-  public void setState(JobTrackerState state) {
-    this.state = state;
-  }
-
-  public void unsetState() {
-    this.state = null;
-  }
-
-  /** Returns true if field state is set (has been assigned a value) and false otherwise */
-  public boolean isSetState() {
-    return this.state != null;
-  }
-
-  public void setStateIsSet(boolean value) {
-    if (!value) {
-      this.state = null;
-    }
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case TASK_TRACKERS:
-      if (value == null) {
-        unsetTaskTrackers();
-      } else {
-        setTaskTrackers((Integer)value);
-      }
-      break;
-
-    case MAP_TASKS:
-      if (value == null) {
-        unsetMapTasks();
-      } else {
-        setMapTasks((Integer)value);
-      }
-      break;
-
-    case REDUCE_TASKS:
-      if (value == null) {
-        unsetReduceTasks();
-      } else {
-        setReduceTasks((Integer)value);
-      }
-      break;
-
-    case MAX_MAP_TASKS:
-      if (value == null) {
-        unsetMaxMapTasks();
-      } else {
-        setMaxMapTasks((Integer)value);
-      }
-      break;
-
-    case MAX_REDUCE_TASKS:
-      if (value == null) {
-        unsetMaxReduceTasks();
-      } else {
-        setMaxReduceTasks((Integer)value);
-      }
-      break;
-
-    case STATE:
-      if (value == null) {
-        unsetState();
-      } else {
-        setState((JobTrackerState)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case TASK_TRACKERS:
-      return getTaskTrackers();
-
-    case MAP_TASKS:
-      return getMapTasks();
-
-    case REDUCE_TASKS:
-      return getReduceTasks();
-
-    case MAX_MAP_TASKS:
-      return getMaxMapTasks();
-
-    case MAX_REDUCE_TASKS:
-      return getMaxReduceTasks();
-
-    case STATE:
-      return getState();
-
-    }
-    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 TASK_TRACKERS:
-      return isSetTaskTrackers();
-    case MAP_TASKS:
-      return isSetMapTasks();
-    case REDUCE_TASKS:
-      return isSetReduceTasks();
-    case MAX_MAP_TASKS:
-      return isSetMaxMapTasks();
-    case MAX_REDUCE_TASKS:
-      return isSetMaxReduceTasks();
-    case STATE:
-      return isSetState();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof HiveClusterStatus)
-      return this.equals((HiveClusterStatus)that);
-    return false;
-  }
-
-  public boolean equals(HiveClusterStatus that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_taskTrackers = true;
-    boolean that_present_taskTrackers = true;
-    if (this_present_taskTrackers || that_present_taskTrackers) {
-      if (!(this_present_taskTrackers && that_present_taskTrackers))
-        return false;
-      if (this.taskTrackers != that.taskTrackers)
-        return false;
-    }
-
-    boolean this_present_mapTasks = true;
-    boolean that_present_mapTasks = true;
-    if (this_present_mapTasks || that_present_mapTasks) {
-      if (!(this_present_mapTasks && that_present_mapTasks))
-        return false;
-      if (this.mapTasks != that.mapTasks)
-        return false;
-    }
-
-    boolean this_present_reduceTasks = true;
-    boolean that_present_reduceTasks = true;
-    if (this_present_reduceTasks || that_present_reduceTasks) {
-      if (!(this_present_reduceTasks && that_present_reduceTasks))
-        return false;
-      if (this.reduceTasks != that.reduceTasks)
-        return false;
-    }
-
-    boolean this_present_maxMapTasks = true;
-    boolean that_present_maxMapTasks = true;
-    if (this_present_maxMapTasks || that_present_maxMapTasks) {
-      if (!(this_present_maxMapTasks && that_present_maxMapTasks))
-        return false;
-      if (this.maxMapTasks != that.maxMapTasks)
-        return false;
-    }
-
-    boolean this_present_maxReduceTasks = true;
-    boolean that_present_maxReduceTasks = true;
-    if (this_present_maxReduceTasks || that_present_maxReduceTasks) {
-      if (!(this_present_maxReduceTasks && that_present_maxReduceTasks))
-        return false;
-      if (this.maxReduceTasks != that.maxReduceTasks)
-        return false;
-    }
-
-    boolean this_present_state = true && this.isSetState();
-    boolean that_present_state = true && that.isSetState();
-    if (this_present_state || that_present_state) {
-      if (!(this_present_state && that_present_state))
-        return false;
-      if (!this.state.equals(that.state))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    List<Object> list = new ArrayList<Object>();
-
-    boolean present_taskTrackers = true;
-    list.add(present_taskTrackers);
-    if (present_taskTrackers)
-      list.add(taskTrackers);
-
-    boolean present_mapTasks = true;
-    list.add(present_mapTasks);
-    if (present_mapTasks)
-      list.add(mapTasks);
-
-    boolean present_reduceTasks = true;
-    list.add(present_reduceTasks);
-    if (present_reduceTasks)
-      list.add(reduceTasks);
-
-    boolean present_maxMapTasks = true;
-    list.add(present_maxMapTasks);
-    if (present_maxMapTasks)
-      list.add(maxMapTasks);
-
-    boolean present_maxReduceTasks = true;
-    list.add(present_maxReduceTasks);
-    if (present_maxReduceTasks)
-      list.add(maxReduceTasks);
-
-    boolean present_state = true && (isSetState());
-    list.add(present_state);
-    if (present_state)
-      list.add(state.getValue());
-
-    return list.hashCode();
-  }
-
-  @Override
-  public int compareTo(HiveClusterStatus other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = Boolean.valueOf(isSetTaskTrackers()).compareTo(other.isSetTaskTrackers());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetTaskTrackers()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskTrackers, other.taskTrackers);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetMapTasks()).compareTo(other.isSetMapTasks());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetMapTasks()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mapTasks, other.mapTasks);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetReduceTasks()).compareTo(other.isSetReduceTasks());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetReduceTasks()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reduceTasks, other.reduceTasks);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetMaxMapTasks()).compareTo(other.isSetMaxMapTasks());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetMaxMapTasks()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxMapTasks, other.maxMapTasks);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetMaxReduceTasks()).compareTo(other.isSetMaxReduceTasks());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetMaxReduceTasks()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxReduceTasks, other.maxReduceTasks);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetState()).compareTo(other.isSetState());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetState()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, other.state);
-      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("HiveClusterStatus(");
-    boolean first = true;
-
-    sb.append("taskTrackers:");
-    sb.append(this.taskTrackers);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("mapTasks:");
-    sb.append(this.mapTasks);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("reduceTasks:");
-    sb.append(this.reduceTasks);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("maxMapTasks:");
-    sb.append(this.maxMapTasks);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("maxReduceTasks:");
-    sb.append(this.maxReduceTasks);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("state:");
-    if (this.state == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.state);
-    }
-    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 HiveClusterStatusStandardSchemeFactory implements SchemeFactory {
-    public HiveClusterStatusStandardScheme getScheme() {
-      return new HiveClusterStatusStandardScheme();
-    }
-  }
-
-  private static class HiveClusterStatusStandardScheme extends StandardScheme<HiveClusterStatus> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, HiveClusterStatus 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: // TASK_TRACKERS
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.taskTrackers = iprot.readI32();
-              struct.setTaskTrackersIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // MAP_TASKS
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.mapTasks = iprot.readI32();
-              struct.setMapTasksIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // REDUCE_TASKS
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.reduceTasks = iprot.readI32();
-              struct.setReduceTasksIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 4: // MAX_MAP_TASKS
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.maxMapTasks = iprot.readI32();
-              struct.setMaxMapTasksIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 5: // MAX_REDUCE_TASKS
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.maxReduceTasks = iprot.readI32();
-              struct.setMaxReduceTasksIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 6: // STATE
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.state = org.apache.hadoop.hive.service.JobTrackerState.findByValue(iprot.readI32());
-              struct.setStateIsSet(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, HiveClusterStatus struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      oprot.writeFieldBegin(TASK_TRACKERS_FIELD_DESC);
-      oprot.writeI32(struct.taskTrackers);
-      oprot.writeFieldEnd();
-      oprot.writeFieldBegin(MAP_TASKS_FIELD_DESC);
-      oprot.writeI32(struct.mapTasks);
-      oprot.writeFieldEnd();
-      oprot.writeFieldBegin(REDUCE_TASKS_FIELD_DESC);
-      oprot.writeI32(struct.reduceTasks);
-      oprot.writeFieldEnd();
-      oprot.writeFieldBegin(MAX_MAP_TASKS_FIELD_DESC);
-      oprot.writeI32(struct.maxMapTasks);
-      oprot.writeFieldEnd();
-      oprot.writeFieldBegin(MAX_REDUCE_TASKS_FIELD_DESC);
-      oprot.writeI32(struct.maxReduceTasks);
-      oprot.writeFieldEnd();
-      if (struct.state != null) {
-        oprot.writeFieldBegin(STATE_FIELD_DESC);
-        oprot.writeI32(struct.state.getValue());
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class HiveClusterStatusTupleSchemeFactory implements SchemeFactory {
-    public HiveClusterStatusTupleScheme getScheme() {
-      return new HiveClusterStatusTupleScheme();
-    }
-  }
-
-  private static class HiveClusterStatusTupleScheme extends TupleScheme<HiveClusterStatus> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, HiveClusterStatus struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      BitSet optionals = new BitSet();
-      if (struct.isSetTaskTrackers()) {
-        optionals.set(0);
-      }
-      if (struct.isSetMapTasks()) {
-        optionals.set(1);
-      }
-      if (struct.isSetReduceTasks()) {
-        optionals.set(2);
-      }
-      if (struct.isSetMaxMapTasks()) {
-        optionals.set(3);
-      }
-      if (struct.isSetMaxReduceTasks()) {
-        optionals.set(4);
-      }
-      if (struct.isSetState()) {
-        optionals.set(5);
-      }
-      oprot.writeBitSet(optionals, 6);
-      if (struct.isSetTaskTrackers()) {
-        oprot.writeI32(struct.taskTrackers);
-      }
-      if (struct.isSetMapTasks()) {
-        oprot.writeI32(struct.mapTasks);
-      }
-      if (struct.isSetReduceTasks()) {
-        oprot.writeI32(struct.reduceTasks);
-      }
-      if (struct.isSetMaxMapTasks()) {
-        oprot.writeI32(struct.maxMapTasks);
-      }
-      if (struct.isSetMaxReduceTasks()) {
-        oprot.writeI32(struct.maxReduceTasks);
-      }
-      if (struct.isSetState()) {
-        oprot.writeI32(struct.state.getValue());
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, HiveClusterStatus struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(6);
-      if (incoming.get(0)) {
-        struct.taskTrackers = iprot.readI32();
-        struct.setTaskTrackersIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.mapTasks = iprot.readI32();
-        struct.setMapTasksIsSet(true);
-      }
-      if (incoming.get(2)) {
-        struct.reduceTasks = iprot.readI32();
-        struct.setReduceTasksIsSet(true);
-      }
-      if (incoming.get(3)) {
-        struct.maxMapTasks = iprot.readI32();
-        struct.setMaxMapTasksIsSet(true);
-      }
-      if (incoming.get(4)) {
-        struct.maxReduceTasks = iprot.readI32();
-        struct.setMaxReduceTasksIsSet(true);
-      }
-      if (incoming.get(5)) {
-        struct.state = org.apache.hadoop.hive.service.JobTrackerState.findByValue(iprot.readI32());
-        struct.setStateIsSet(true);
-      }
-    }
-  }
-
-}
-


[09/20] hive git commit: HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java
deleted file mode 100644
index 97b1219..0000000
--- a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java
+++ /dev/null
@@ -1,601 +0,0 @@
-/**
- * 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.service;
-
-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 HiveServerException extends TException implements org.apache.thrift.TBase<HiveServerException, HiveServerException._Fields>, java.io.Serializable, Cloneable, Comparable<HiveServerException> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HiveServerException");
-
-  private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)2);
-  private static final org.apache.thrift.protocol.TField SQLSTATE_FIELD_DESC = new org.apache.thrift.protocol.TField("SQLState", org.apache.thrift.protocol.TType.STRING, (short)3);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new HiveServerExceptionStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new HiveServerExceptionTupleSchemeFactory());
-  }
-
-  private String message; // required
-  private int errorCode; // required
-  private String SQLState; // 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 {
-    MESSAGE((short)1, "message"),
-    ERROR_CODE((short)2, "errorCode"),
-    SQLSTATE((short)3, "SQLState");
-
-    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: // MESSAGE
-          return MESSAGE;
-        case 2: // ERROR_CODE
-          return ERROR_CODE;
-        case 3: // SQLSTATE
-          return SQLSTATE;
-        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 __ERRORCODE_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.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.SQLSTATE, new org.apache.thrift.meta_data.FieldMetaData("SQLState", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HiveServerException.class, metaDataMap);
-  }
-
-  public HiveServerException() {
-  }
-
-  public HiveServerException(
-    String message,
-    int errorCode,
-    String SQLState)
-  {
-    this();
-    this.message = message;
-    this.errorCode = errorCode;
-    setErrorCodeIsSet(true);
-    this.SQLState = SQLState;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public HiveServerException(HiveServerException other) {
-    __isset_bitfield = other.__isset_bitfield;
-    if (other.isSetMessage()) {
-      this.message = other.message;
-    }
-    this.errorCode = other.errorCode;
-    if (other.isSetSQLState()) {
-      this.SQLState = other.SQLState;
-    }
-  }
-
-  public HiveServerException deepCopy() {
-    return new HiveServerException(this);
-  }
-
-  @Override
-  public void clear() {
-    this.message = null;
-    setErrorCodeIsSet(false);
-    this.errorCode = 0;
-    this.SQLState = null;
-  }
-
-  public String getMessage() {
-    return this.message;
-  }
-
-  public void setMessage(String message) {
-    this.message = message;
-  }
-
-  public void unsetMessage() {
-    this.message = null;
-  }
-
-  /** Returns true if field message is set (has been assigned a value) and false otherwise */
-  public boolean isSetMessage() {
-    return this.message != null;
-  }
-
-  public void setMessageIsSet(boolean value) {
-    if (!value) {
-      this.message = null;
-    }
-  }
-
-  public int getErrorCode() {
-    return this.errorCode;
-  }
-
-  public void setErrorCode(int errorCode) {
-    this.errorCode = errorCode;
-    setErrorCodeIsSet(true);
-  }
-
-  public void unsetErrorCode() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID);
-  }
-
-  /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */
-  public boolean isSetErrorCode() {
-    return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID);
-  }
-
-  public void setErrorCodeIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value);
-  }
-
-  public String getSQLState() {
-    return this.SQLState;
-  }
-
-  public void setSQLState(String SQLState) {
-    this.SQLState = SQLState;
-  }
-
-  public void unsetSQLState() {
-    this.SQLState = null;
-  }
-
-  /** Returns true if field SQLState is set (has been assigned a value) and false otherwise */
-  public boolean isSetSQLState() {
-    return this.SQLState != null;
-  }
-
-  public void setSQLStateIsSet(boolean value) {
-    if (!value) {
-      this.SQLState = null;
-    }
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case MESSAGE:
-      if (value == null) {
-        unsetMessage();
-      } else {
-        setMessage((String)value);
-      }
-      break;
-
-    case ERROR_CODE:
-      if (value == null) {
-        unsetErrorCode();
-      } else {
-        setErrorCode((Integer)value);
-      }
-      break;
-
-    case SQLSTATE:
-      if (value == null) {
-        unsetSQLState();
-      } else {
-        setSQLState((String)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case MESSAGE:
-      return getMessage();
-
-    case ERROR_CODE:
-      return getErrorCode();
-
-    case SQLSTATE:
-      return getSQLState();
-
-    }
-    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 MESSAGE:
-      return isSetMessage();
-    case ERROR_CODE:
-      return isSetErrorCode();
-    case SQLSTATE:
-      return isSetSQLState();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof HiveServerException)
-      return this.equals((HiveServerException)that);
-    return false;
-  }
-
-  public boolean equals(HiveServerException that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_message = true && this.isSetMessage();
-    boolean that_present_message = true && that.isSetMessage();
-    if (this_present_message || that_present_message) {
-      if (!(this_present_message && that_present_message))
-        return false;
-      if (!this.message.equals(that.message))
-        return false;
-    }
-
-    boolean this_present_errorCode = true;
-    boolean that_present_errorCode = true;
-    if (this_present_errorCode || that_present_errorCode) {
-      if (!(this_present_errorCode && that_present_errorCode))
-        return false;
-      if (this.errorCode != that.errorCode)
-        return false;
-    }
-
-    boolean this_present_SQLState = true && this.isSetSQLState();
-    boolean that_present_SQLState = true && that.isSetSQLState();
-    if (this_present_SQLState || that_present_SQLState) {
-      if (!(this_present_SQLState && that_present_SQLState))
-        return false;
-      if (!this.SQLState.equals(that.SQLState))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    List<Object> list = new ArrayList<Object>();
-
-    boolean present_message = true && (isSetMessage());
-    list.add(present_message);
-    if (present_message)
-      list.add(message);
-
-    boolean present_errorCode = true;
-    list.add(present_errorCode);
-    if (present_errorCode)
-      list.add(errorCode);
-
-    boolean present_SQLState = true && (isSetSQLState());
-    list.add(present_SQLState);
-    if (present_SQLState)
-      list.add(SQLState);
-
-    return list.hashCode();
-  }
-
-  @Override
-  public int compareTo(HiveServerException other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = Boolean.valueOf(isSetMessage()).compareTo(other.isSetMessage());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetMessage()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.message);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(other.isSetErrorCode());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetErrorCode()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, other.errorCode);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetSQLState()).compareTo(other.isSetSQLState());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetSQLState()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.SQLState, other.SQLState);
-      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("HiveServerException(");
-    boolean first = true;
-
-    sb.append("message:");
-    if (this.message == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.message);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("errorCode:");
-    sb.append(this.errorCode);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("SQLState:");
-    if (this.SQLState == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.SQLState);
-    }
-    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 HiveServerExceptionStandardSchemeFactory implements SchemeFactory {
-    public HiveServerExceptionStandardScheme getScheme() {
-      return new HiveServerExceptionStandardScheme();
-    }
-  }
-
-  private static class HiveServerExceptionStandardScheme extends StandardScheme<HiveServerException> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, HiveServerException 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: // MESSAGE
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.message = iprot.readString();
-              struct.setMessageIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // ERROR_CODE
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.errorCode = iprot.readI32();
-              struct.setErrorCodeIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // SQLSTATE
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.SQLState = iprot.readString();
-              struct.setSQLStateIsSet(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, HiveServerException struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.message != null) {
-        oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
-        oprot.writeString(struct.message);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC);
-      oprot.writeI32(struct.errorCode);
-      oprot.writeFieldEnd();
-      if (struct.SQLState != null) {
-        oprot.writeFieldBegin(SQLSTATE_FIELD_DESC);
-        oprot.writeString(struct.SQLState);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class HiveServerExceptionTupleSchemeFactory implements SchemeFactory {
-    public HiveServerExceptionTupleScheme getScheme() {
-      return new HiveServerExceptionTupleScheme();
-    }
-  }
-
-  private static class HiveServerExceptionTupleScheme extends TupleScheme<HiveServerException> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, HiveServerException struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      BitSet optionals = new BitSet();
-      if (struct.isSetMessage()) {
-        optionals.set(0);
-      }
-      if (struct.isSetErrorCode()) {
-        optionals.set(1);
-      }
-      if (struct.isSetSQLState()) {
-        optionals.set(2);
-      }
-      oprot.writeBitSet(optionals, 3);
-      if (struct.isSetMessage()) {
-        oprot.writeString(struct.message);
-      }
-      if (struct.isSetErrorCode()) {
-        oprot.writeI32(struct.errorCode);
-      }
-      if (struct.isSetSQLState()) {
-        oprot.writeString(struct.SQLState);
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, HiveServerException struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(3);
-      if (incoming.get(0)) {
-        struct.message = iprot.readString();
-        struct.setMessageIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.errorCode = iprot.readI32();
-        struct.setErrorCodeIsSet(true);
-      }
-      if (incoming.get(2)) {
-        struct.SQLState = iprot.readString();
-        struct.setSQLStateIsSet(true);
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/JobTrackerState.java
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/JobTrackerState.java b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/JobTrackerState.java
deleted file mode 100644
index 46c71ee..0000000
--- a/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/JobTrackerState.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * 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.service;
-
-
-import java.util.Map;
-import java.util.HashMap;
-import org.apache.thrift.TEnum;
-
-public enum JobTrackerState implements org.apache.thrift.TEnum {
-  INITIALIZING(1),
-  RUNNING(2);
-
-  private final int value;
-
-  private JobTrackerState(int value) {
-    this.value = value;
-  }
-
-  /**
-   * Get the integer value of this enum value, as defined in the Thrift IDL.
-   */
-  public int getValue() {
-    return value;
-  }
-
-  /**
-   * Find a the enum type by its integer value, as defined in the Thrift IDL.
-   * @return null if the value is not found.
-   */
-  public static JobTrackerState findByValue(int value) { 
-    switch (value) {
-      case 1:
-        return INITIALIZING;
-      case 2:
-        return RUNNING;
-      default:
-        return null;
-    }
-  }
-}


[17/20] hive git commit: HIVE-13603. Fix ptest unit tests broken by HIVE13505. (Siddharth Seth, reviewed by Szehon Ho)

Posted by jd...@apache.org.
HIVE-13603. Fix ptest unit tests broken by HIVE13505. (Siddharth Seth, reviewed by Szehon Ho)


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

Branch: refs/heads/llap
Commit: 2fe47dc4a8398222b29853ac4e38c74e54ed14c5
Parents: b3eb86c
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 28 12:00:30 2016 +0530
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Apr 28 12:00:30 2016 +0530

----------------------------------------------------------------------
 .../hive/ptest/execution/TestScripts.testPrepGit.approved.txt      | 2 --
 .../hive/ptest/execution/TestScripts.testPrepHadoop1.approved.txt  | 2 --
 .../hive/ptest/execution/TestScripts.testPrepNone.approved.txt     | 2 --
 .../hive/ptest/execution/TestScripts.testPrepSvn.approved.txt      | 2 --
 4 files changed, 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2fe47dc4/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt
index 673614b..d58d910 100644
--- a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt
@@ -85,10 +85,8 @@ cd /some/working/dir/
   then
     rm -rf /some/working/dir/maven/org/apache/hive
         mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven -X -Phadoop-2
-    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy -X -Phadoop-1
     cd itests
     mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven -X -Phadoop-2
-    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy -X -Phadoop-1
   elif [[ "${buildTool}" == "ant" ]]
   then
     ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy \

http://git-wip-us.apache.org/repos/asf/hive/blob/2fe47dc4/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepHadoop1.approved.txt
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepHadoop1.approved.txt b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepHadoop1.approved.txt
index dbb6a6d..1b9ca94 100644
--- a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepHadoop1.approved.txt
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepHadoop1.approved.txt
@@ -94,10 +94,8 @@ cd /some/working/dir/
         cd ..
       done
         mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven $mavenArgs $mavenBuildArgs
-    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy $mavenArgs $mavenTestArgs
     cd itests
     mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven $mavenArgs $mavenBuildArgs
-    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy $mavenArgs $mavenTestArgs
   elif [[ "maven" == "ant" ]]
   then
     ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy \

http://git-wip-us.apache.org/repos/asf/hive/blob/2fe47dc4/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt
index c05dae9..dde8822 100644
--- a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt
@@ -85,10 +85,8 @@ cd /some/working/dir/
   then
     rm -rf /some/working/dir/maven/org/apache/hive
         mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven $mavenArgs $mavenBuildArgs
-    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy $mavenArgs $mavenTestArgs
     cd itests
     mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven $mavenArgs $mavenBuildArgs
-    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy $mavenArgs $mavenTestArgs
   elif [[ "ant" == "ant" ]]
   then
     ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy \

http://git-wip-us.apache.org/repos/asf/hive/blob/2fe47dc4/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt
index c8a4b6e..f3eec2d 100644
--- a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt
@@ -85,10 +85,8 @@ cd /some/working/dir/
   then
     rm -rf /some/working/dir/maven/org/apache/hive
         mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven $mavenArgs $mavenBuildArgs
-    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy $mavenArgs $mavenTestArgs
     cd itests
     mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven $mavenArgs $mavenBuildArgs
-    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy $mavenArgs $mavenTestArgs
   elif [[ "maven" == "ant" ]]
   then
     ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy \


[03/20] hive git commit: HIVE-13241 : LLAP: Incremental Caching marks some small chunks as "incomplete CB" (Sergey Shelukhin, reviewed by Prasanth Jayachandran) ADDENDUM

Posted by jd...@apache.org.
HIVE-13241 : LLAP: Incremental Caching marks some small chunks as "incomplete CB" (Sergey Shelukhin, reviewed by Prasanth Jayachandran) ADDENDUM


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

Branch: refs/heads/llap
Commit: 972bcba7a811bf7d464f59666b81bdfe6fd96d47
Parents: 85ffd22
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Apr 26 16:13:47 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Apr 26 16:13:47 2016 -0700

----------------------------------------------------------------------
 common/src/java/org/apache/hadoop/hive/conf/HiveConf.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/972bcba7/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index bae3999..5360ed4 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2604,9 +2604,9 @@ public class HiveConf extends Configuration {
         "use a FS without file IDs and rewrite files a lot (or are paranoid), you might want\n" +
         "to avoid this setting."),
     LLAP_CACHE_ENABLE_ORC_GAP_CACHE("hive.llap.orc.gap.cache", true,
-        "Whether LLAP cache for ORC should remember gaps in ORC RG read estimates, to avoid\n" +
-        "re-reading the data that was read once and discarded because it is unneeded. This is\n" +
-        "only necessary for ORC files written before HIVE-9660 (Hive 2.1?)."),
+        "Whether LLAP cache for ORC should remember gaps in ORC compression buffer read\n" +
+        "estimates, to avoid re-reading the data that was read once and discarded because it\n" +
+        "is unneeded. This is only necessary for ORC files written before HIVE-9660."),
     LLAP_IO_USE_FILEID_PATH("hive.llap.io.use.fileid.path", true,
         "Whether LLAP should use fileId (inode)-based path to ensure better consistency for the\n" +
         "cases of file overwrites. This is supported on HDFS."),


[19/20] hive git commit: HIVE-13536: LLAP: Add metrics for task scheduler

Posted by jd...@apache.org.
HIVE-13536: LLAP: Add metrics for task scheduler


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

Branch: refs/heads/llap
Commit: 0ebcd938cf0da7ef74ef3534a3aef4b7976a1163
Parents: 8bdf618
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Thu Apr 28 14:10:13 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Thu Apr 28 14:10:13 2016 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   8 +
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |   5 +-
 .../llap/daemon/impl/ContainerRunnerImpl.java   |   7 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |  20 +-
 .../llap/daemon/impl/TaskExecutorService.java   |  53 ++++-
 .../llap/daemon/impl/TaskRunnerCallable.java    |  28 ++-
 .../hive/llap/io/api/impl/LlapIoImpl.java       |   1 -
 .../llap/io/decode/EncodedDataConsumer.java     |   5 -
 .../llap/io/encoded/OrcEncodedDataReader.java   |   3 -
 .../hive/llap/metrics/LlapDaemonCacheInfo.java  |   6 +-
 .../llap/metrics/LlapDaemonCacheMetrics.java    |  35 +---
 .../llap/metrics/LlapDaemonExecutorInfo.java    |  23 ++-
 .../llap/metrics/LlapDaemonExecutorMetrics.java | 152 +++++++++++---
 .../hive/llap/metrics/LlapDaemonIOInfo.java     |   4 -
 .../hive/llap/metrics/LlapDaemonIOMetrics.java  |  37 +---
 .../daemon/impl/TestTaskExecutorService.java    |   2 +-
 .../tezplugins/LlapTaskSchedulerService.java    | 122 ++++++++++--
 .../metrics/LlapTaskSchedulerInfo.java          |  59 ++++++
 .../metrics/LlapTaskSchedulerMetrics.java       | 197 +++++++++++++++++++
 .../TestLlapTaskSchedulerService.java           |   2 +-
 20 files changed, 614 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index eeb9b84..566e9b6 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -330,6 +330,7 @@ public class HiveConf extends Configuration {
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_TASK_SCHEDULER_WAIT_QUEUE_SIZE.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_WAIT_QUEUE_COMPARATOR_CLASS_NAME.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_TASK_SCHEDULER_ENABLE_PREEMPTION.varname);
+    llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_TASK_PREEMPTION_METRICS_INTERVALS.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_WEB_PORT.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_WEB_SSL.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_CONTAINER_ID.varname);
@@ -2763,6 +2764,13 @@ public class HiveConf extends Configuration {
             " to a location other than the ones requested. Set to -1 for an infinite delay, 0" +
             "for a no delay. Currently these are the only two supported values"
     ),
+    LLAP_DAEMON_TASK_PREEMPTION_METRICS_INTERVALS(
+        "hive.llap.daemon.task.preemption.metrics.intervals", "30,60,300",
+        "Comma-delimited set of integers denoting the desired rollover intervals (in seconds)\n" +
+        " for percentile latency metrics. Used by LLAP daemon task scheduler metrics for\n" +
+        " time taken to kill task (due to pre-emption) and useful time wasted by the task that\n" +
+        " is about to be preempted."
+    ),
     LLAP_DAEMON_TASK_SCHEDULER_WAIT_QUEUE_SIZE("hive.llap.daemon.task.scheduler.wait.queue.size",
       10, "LLAP scheduler maximum queue size.", "llap.daemon.task.scheduler.wait.queue.size"),
     LLAP_DAEMON_WAIT_QUEUE_COMPARATOR_CLASS_NAME(

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
index 9f7e5c9..d78c1e0 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
@@ -40,7 +40,7 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
   private final long maxSize;
   private final boolean isDirect;
   private final LlapDaemonCacheMetrics metrics;
-  
+
   // We don't know the acceptable size for Java array, so we'll use 1Gb boundary.
   // That is guaranteed to fit any maximum allocation.
   private static final int MAX_ARENA_SIZE = 1024*1024*1024;
@@ -113,9 +113,6 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
 
     this.metrics = metrics;
     metrics.incrAllocatedArena();
-    metrics.setArenaSize(arenaSize);
-    metrics.setMinAllocationSize(minAllocation);
-    metrics.setMaxAllocationSize(maxAllocation);
   }
 
   // TODO: would it make sense to return buffers asynchronously?

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
index e80fb15..3d45c7a 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
@@ -106,7 +106,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
     String waitQueueSchedulerClassName = HiveConf.getVar(
         conf, ConfVars.LLAP_DAEMON_WAIT_QUEUE_COMPARATOR_CLASS_NAME);
     this.executorService = new TaskExecutorService(numExecutors, waitQueueSize,
-        waitQueueSchedulerClassName, enablePreemption, classLoader);
+        waitQueueSchedulerClassName, enablePreemption, classLoader, metrics);
 
     addIfService(executorService);
 
@@ -218,8 +218,9 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
             .setSubmissionState(SubmissionStateProto.valueOf(submissionState.name()))
             .build();
       }
-      metrics.incrExecutorTotalRequestsHandled();
-      metrics.incrExecutorNumQueuedRequests();
+      if (metrics != null) {
+        metrics.incrExecutorTotalRequestsHandled();
+      }
     } finally {
       NDC.pop();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index 33b41e8..63cb16b 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -20,7 +20,9 @@ import java.lang.management.MemoryPoolMXBean;
 import java.lang.management.MemoryType;
 import java.net.InetSocketAddress;
 import java.net.URL;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
@@ -68,6 +70,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
 
 public class LlapDaemon extends CompositeService implements ContainerRunner, LlapDaemonMXBean {
 
@@ -193,12 +196,25 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     String displayName = "LlapDaemonExecutorMetrics-" + MetricsUtils.getHostName();
     String sessionId = MetricsUtils.getUUID();
     daemonConf.set("llap.daemon.metrics.sessionid", sessionId);
-    this.metrics = LlapDaemonExecutorMetrics.create(displayName, sessionId, numExecutors);
+    String[] strIntervals = HiveConf.getTrimmedStringsVar(daemonConf,
+        HiveConf.ConfVars.LLAP_DAEMON_TASK_PREEMPTION_METRICS_INTERVALS);
+    List<Integer> intervalList = new ArrayList<>();
+    if (strIntervals != null) {
+      for (String strInterval : strIntervals) {
+        try {
+          intervalList.add(Integer.valueOf(strInterval));
+        } catch (NumberFormatException e) {
+          LOG.warn("Ignoring task pre-emption metrics interval {} from {} as it is invalid",
+              strInterval, Arrays.toString(strIntervals));
+        }
+      }
+    }
+    this.metrics = LlapDaemonExecutorMetrics.create(displayName, sessionId, numExecutors,
+        Ints.toArray(intervalList));
     this.metrics.setMemoryPerInstance(executorMemoryBytes);
     this.metrics.setCacheMemoryPerInstance(ioMemoryBytes);
     this.metrics.setJvmMaxMemory(maxJvmMemory);
     this.metrics.setWaitQueueSize(waitQueueSize);
-    this.metrics.setRpcNumHandlers(numHandlers);
     metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
     this.llapDaemonInfoBean = MBeans.register("LlapDaemon", "LlapDaemonInfo", this);
     LOG.info("Started LlapMetricsSystem with displayName: " + displayName +

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
index 57dd828..f621af2 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
@@ -44,6 +44,7 @@ import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.hadoop.hive.llap.daemon.FinishableStateUpdateHandler;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorMetrics;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.tez.runtime.task.EndReason;
 import org.apache.tez.runtime.task.TaskRunner2Result;
@@ -104,10 +105,11 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
   final ConcurrentMap<String, TaskWrapper> knownTasks = new ConcurrentHashMap<>();
 
   private final Object lock = new Object();
+  private final LlapDaemonExecutorMetrics metrics;
 
   public TaskExecutorService(int numExecutors, int waitQueueSize,
       String waitQueueComparatorClassName, boolean enablePreemption,
-      ClassLoader classLoader) {
+      ClassLoader classLoader, final LlapDaemonExecutorMetrics metrics) {
     super(TaskExecutorService.class.getSimpleName());
     LOG.info("TaskExecutorService is being setup with parameters: "
         + "numExecutors=" + numExecutors
@@ -127,6 +129,10 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
         new PreemptionQueueComparator());
     this.enablePreemption = enablePreemption;
     this.numSlotsAvailable = new AtomicInteger(numExecutors);
+    this.metrics = metrics;
+    if (metrics != null) {
+      metrics.setNumExecutorsAvailable(numSlotsAvailable.get());
+    }
 
     // single threaded scheduler for tasks from wait queue to executor threads
     ExecutorService wes = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder()
@@ -267,7 +273,11 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
               trySchedule(task);
               // wait queue could have been re-ordered in the mean time because of concurrent task
               // submission. So remove the specific task instead of the head task.
-              waitQueue.remove(task);
+              if (waitQueue.remove(task)) {
+                if (metrics != null) {
+                  metrics.setExecutorNumQueuedRequests(waitQueue.size());
+                }
+              }
             } catch (RejectedExecutionException e) {
               rejectedException = e;
             }
@@ -361,6 +371,9 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
         if (isDebugEnabled) {
           LOG.debug("{} is {} as wait queue is full", taskWrapper.getRequestId(), result);
         }
+        if (metrics != null) {
+          metrics.incrTotalRejectedRequests();
+        }
         return result;
       }
     }
@@ -392,11 +405,17 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
         LOG.info("{} evicted from wait queue in favor of {} because of lower priority",
             evictedTask.getRequestId(), task.getRequestId());
       }
+      if (metrics != null) {
+        metrics.incrTotalEvictedFromWaitQueue();
+      }
     }
     synchronized (lock) {
       lock.notify();
     }
 
+    if (metrics != null) {
+      metrics.setExecutorNumQueuedRequests(waitQueue.size());
+    }
     return result;
   }
 
@@ -411,7 +430,11 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
             LOG.debug("Removing {} from waitQueue", fragmentId);
           }
           taskWrapper.setIsInWaitQueue(false);
-          waitQueue.remove(taskWrapper);
+          if (waitQueue.remove(taskWrapper)) {
+            if (metrics != null) {
+              metrics.setExecutorNumQueuedRequests(waitQueue.size());
+            }
+          }
         }
         if (taskWrapper.isInPreemptionQueue()) {
           if (isDebugEnabled) {
@@ -419,6 +442,9 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
           }
           taskWrapper.setIsInPreemptableQueue(false);
           preemptionQueue.remove(taskWrapper);
+          if (metrics != null) {
+            metrics.setExecutorNumPreemptableRequests(preemptionQueue.size());
+          }
         }
         taskWrapper.getTaskRunnerCallable().killTask();
       } else {
@@ -460,6 +486,9 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
         }
       }
       numSlotsAvailable.decrementAndGet();
+      if (metrics != null) {
+        metrics.setNumExecutorsAvailable(numSlotsAvailable.get());
+      }
   }
 
   private void handleScheduleAttemptedRejection(TaskWrapper taskWrapper) {
@@ -511,11 +540,17 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
         LOG.debug("Removing {} from preemption queue because it's state changed to {}",
             taskWrapper.getRequestId(), newFinishableState);
         preemptionQueue.remove(taskWrapper.getTaskRunnerCallable());
+        if (metrics != null) {
+          metrics.setExecutorNumPreemptableRequests(preemptionQueue.size());
+        }
       } else if (newFinishableState == false && !taskWrapper.isInPreemptionQueue() &&
           !taskWrapper.isInWaitQueue()) {
         LOG.debug("Adding {} to preemption queue since finishable state changed to {}",
             taskWrapper.getRequestId(), newFinishableState);
         preemptionQueue.offer(taskWrapper);
+        if (metrics != null) {
+          metrics.setExecutorNumPreemptableRequests(preemptionQueue.size());
+        }
       }
       lock.notify();
     }
@@ -525,6 +560,9 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
     synchronized (lock) {
       preemptionQueue.add(taskWrapper);
       taskWrapper.setIsInPreemptableQueue(true);
+      if (metrics != null) {
+        metrics.setExecutorNumPreemptableRequests(preemptionQueue.size());
+      }
     }
   }
 
@@ -534,6 +572,9 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
        taskWrapper = preemptionQueue.remove();
       if (taskWrapper != null) {
         taskWrapper.setIsInPreemptableQueue(false);
+        if (metrics != null) {
+          metrics.setExecutorNumPreemptableRequests(preemptionQueue.size());
+        }
       }
     }
     return taskWrapper;
@@ -582,9 +623,15 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
               .getTaskIdentifierString(taskWrapper.getTaskRunnerCallable().getRequest())
               + " request " + state + "! Removed from preemption list.");
         }
+        if (metrics != null) {
+          metrics.setExecutorNumPreemptableRequests(preemptionQueue.size());
+        }
       }
 
       numSlotsAvailable.incrementAndGet();
+      if (metrics != null) {
+        metrics.setNumExecutorsAvailable(numSlotsAvailable.get());
+      }
       if (isDebugEnabled) {
         LOG.debug("Task {} complete. WaitQueueSize={}, numSlotsAvailable={}, preemptionQueueSize={}",
           taskWrapper.getRequestId(), waitQueue.size(), numSlotsAvailable.get(),

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
index 2a60123..fcfa940 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
@@ -413,10 +413,15 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
         // Only the KILLED case requires a message to be sent out to the AM.
         case SUCCESS:
           LOG.debug("Successfully finished {}", requestId);
-          metrics.incrExecutorTotalSuccess();
+          if (metrics != null) {
+            metrics.incrExecutorTotalSuccess();
+          }
           break;
         case CONTAINER_STOP_REQUESTED:
           LOG.info("Received container stop request (AM preemption) for {}", requestId);
+          if (metrics != null) {
+            metrics.incrExecutorTotalKilled();
+          }
           break;
         case KILL_REQUESTED:
           LOG.info("Killed task {}", requestId);
@@ -424,17 +429,26 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
             killtimerWatch.stop();
             long elapsed = killtimerWatch.elapsedMillis();
             LOG.info("Time to die for task {}", elapsed);
+            if (metrics != null) {
+              metrics.addMetricsPreemptionTimeToKill(elapsed);
+            }
+          }
+          if (metrics != null) {
+            metrics.addMetricsPreemptionTimeLost(runtimeWatch.elapsedMillis());
+            metrics.incrExecutorTotalKilled();
           }
-          metrics.incrPreemptionTimeLost(runtimeWatch.elapsedMillis());
-          metrics.incrExecutorTotalKilled();
           break;
         case COMMUNICATION_FAILURE:
           LOG.info("Failed to run {} due to communication failure", requestId);
-          metrics.incrExecutorTotalExecutionFailed();
+          if (metrics != null) {
+            metrics.incrExecutorTotalExecutionFailed();
+          }
           break;
         case TASK_ERROR:
           LOG.info("Failed to run {} due to task error", requestId);
-          metrics.incrExecutorTotalExecutionFailed();
+          if (metrics != null) {
+            metrics.incrExecutorTotalExecutionFailed();
+          }
           break;
       }
       fragmentCompletionHanler.fragmentComplete(fragmentInfo);
@@ -448,7 +462,6 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
               request.getFragmentSpec().getFragmentNumber(),
               request.getFragmentSpec().getAttemptNumber(), taskRunnerCallable.threadName,
               taskRunnerCallable.startTime, true);
-      metrics.decrExecutorNumQueuedRequests();
     }
 
     @Override
@@ -466,9 +479,6 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
               request.getFragmentSpec().getFragmentNumber(),
               request.getFragmentSpec().getAttemptNumber(), taskRunnerCallable.threadName,
               taskRunnerCallable.startTime, false);
-      if (metrics != null) {
-        metrics.decrExecutorNumQueuedRequests();
-      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
index 6a72b4c..fea3dc7 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
@@ -139,7 +139,6 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
     int numThreads = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_IO_THREADPOOL_SIZE);
     executor = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(numThreads,
         new ThreadFactoryBuilder().setNameFormat("IO-Elevator-Thread-%d").setDaemon(true).build()));
-    ioMetrics.setIoThreadPoolSize(numThreads);
     // TODO: this should depends on input format and be in a map, or something.
     this.cvp = new OrcColumnVectorProducer(
         metadataCache, orcCache, bufferManager, conf, cacheMetrics, ioMetrics);

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/EncodedDataConsumer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/EncodedDataConsumer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/EncodedDataConsumer.java
index 0ba7c09..6b54b30 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/EncodedDataConsumer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/EncodedDataConsumer.java
@@ -53,7 +53,6 @@ public abstract class EncodedDataConsumer<BatchKey, BatchType extends EncodedCol
             // Don't reset anything, we are reusing column vectors.
           }
         });
-    this.ioMetrics.setColumnVectorBatchPoolSize(cvbPool.size());
   }
 
   public void init(ConsumerFeedback<BatchType> upstreamFeedback,
@@ -62,10 +61,6 @@ public abstract class EncodedDataConsumer<BatchKey, BatchType extends EncodedCol
     this.readCallable = readCallable;
   }
 
-  public LlapDaemonIOMetrics getIOMetrics() {
-    return ioMetrics;
-  }
-
   @Override
   public Callable<Void> getReadCallable() {
     return readCallable;

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
index 83011fb..7effe69 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
@@ -182,9 +182,6 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    LlapDaemonIOMetrics ioMetrics = consumer.getIOMetrics();
-    ioMetrics.setColumnStreamDataPoolSize(CSD_POOL.size());
-    ioMetrics.setEncodedColumnBatchPoolSize(ECB_POOL.size());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheInfo.java
index 191345e..427a0b1 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheInfo.java
@@ -26,6 +26,7 @@ import com.google.common.base.Objects;
  */
 public enum LlapDaemonCacheInfo implements MetricsInfo {
   CacheMetrics("Llap daemon cache related metrics"),
+  CacheCapacityRemainingPercentage("Percentage of memory available in cache"),
   CacheCapacityRemaining("Amount of memory available in cache in bytes"),
   CacheCapacityTotal("Total amount of memory allocated for cache in bytes"),
   CacheCapacityUsed("Amount of memory used in cache in bytes"),
@@ -34,10 +35,7 @@ public enum LlapDaemonCacheInfo implements MetricsInfo {
   CacheHitRatio("Ratio of disk ranges cached vs requested"),
   CacheReadRequests("Number of disk range requests to cache"),
   CacheAllocatedArena("Number of arenas allocated"),
-  CacheNumLockedBuffers("Number of locked buffers in cache"),
-  CacheArenaSize("Size of arena used by allocator"),
-  CacheMinAllocationSize("Minimum allocation size used by allocator"),
-  CacheMaxAllocationSize("Maximum allocation size used by allocator");
+  CacheNumLockedBuffers("Number of locked buffers in cache");
 
   private final String desc;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java
index bb76da5..5f30b2d 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java
@@ -18,18 +18,16 @@
 package org.apache.hadoop.hive.llap.metrics;
 
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheAllocatedArena;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheArenaSize;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheCapacityRemaining;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheCapacityRemainingPercentage;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheCapacityTotal;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheCapacityUsed;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheHitBytes;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheHitRatio;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheMaxAllocationSize;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheMinAllocationSize;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheMetrics;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheNumLockedBuffers;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheReadRequests;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheRequestedBytes;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheInfo.CacheMetrics;
 import static org.apache.hadoop.metrics2.impl.MsInfo.ProcessName;
 import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
 
@@ -68,12 +66,6 @@ public class LlapDaemonCacheMetrics implements MetricsSource {
   MutableCounterLong cacheAllocatedArena;
   @Metric
   MutableCounterLong cacheNumLockedBuffers;
-  @Metric
-  MutableGaugeLong arenaSize;
-  @Metric
-  MutableGaugeLong minAllocationSize;
-  @Metric
-  MutableGaugeLong maxAllocationSize;
 
   private LlapDaemonCacheMetrics(String name, String sessionId) {
     this.name = name;
@@ -115,18 +107,6 @@ public class LlapDaemonCacheMetrics implements MetricsSource {
     cacheNumLockedBuffers.incr();
   }
 
-  public void setArenaSize(long value) {
-    arenaSize.set(value);
-  }
-
-  public void setMinAllocationSize(long value) {
-    minAllocationSize.set(value);
-  }
-
-  public void setMaxAllocationSize(long value) {
-    maxAllocationSize.set(value);
-  }
-
   public void decrCacheNumLockedBuffers() {
     cacheNumLockedBuffers.incr(-1);
   }
@@ -158,7 +138,11 @@ public class LlapDaemonCacheMetrics implements MetricsSource {
     float cacheHitRatio = cacheRequestedBytes.value() == 0 ? 0.0f :
         (float) cacheHitBytes.value() / (float) cacheRequestedBytes.value();
 
-    rb.addCounter(CacheCapacityRemaining, cacheCapacityTotal.value() - cacheCapacityUsed.value())
+    long cacheCapacityRemaining = cacheCapacityTotal.value() - cacheCapacityUsed.value();
+    float cacheRemainingPercent = cacheCapacityTotal.value() == 0 ? 0.0f :
+        (float) cacheCapacityRemaining / (float) cacheCapacityTotal.value();
+    rb.addCounter(CacheCapacityRemaining, cacheCapacityRemaining)
+        .addGauge(CacheCapacityRemainingPercentage, cacheRemainingPercent)
         .addCounter(CacheCapacityTotal, cacheCapacityTotal.value())
         .addCounter(CacheCapacityUsed, cacheCapacityUsed.value())
         .addCounter(CacheReadRequests, cacheReadRequests.value())
@@ -166,10 +150,7 @@ public class LlapDaemonCacheMetrics implements MetricsSource {
         .addCounter(CacheHitBytes, cacheHitBytes.value())
         .addCounter(CacheAllocatedArena, cacheAllocatedArena.value())
         .addCounter(CacheNumLockedBuffers, cacheNumLockedBuffers.value())
-        .addGauge(CacheHitRatio, cacheHitRatio)
-        .addGauge(CacheArenaSize, arenaSize.value())
-        .addGauge(CacheMinAllocationSize, minAllocationSize.value())
-        .addGauge(CacheMaxAllocationSize, maxAllocationSize.value());
+        .addGauge(CacheHitRatio, cacheHitRatio);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorInfo.java
index 941d926..db5fd4f 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorInfo.java
@@ -26,21 +26,32 @@ import com.google.common.base.Objects;
  */
 public enum LlapDaemonExecutorInfo implements MetricsInfo {
   ExecutorMetrics("Llap daemon cache related metrics"),
-  ExecutorThreadCountPerInstance("Total number of executor threads per node"),
+  ExecutorMaxFreeSlots("Sum of wait queue size and number of executors"),
+  ExecutorNumExecutorsPerInstance("Total number of executor threads per node"),
+  ExecutorNumExecutorsAvailable("Total number of executor threads per node that are free"),
+  ExecutorAvailableFreeSlots("Number of free slots available"),
+  ExecutorAvailableFreeSlotsPercent("Percent of free slots available"),
+  ExecutorThreadCPUTime("Cpu time in nanoseconds"),
   ExecutorMemoryPerInstance("Total memory for executors per node in bytes"),
   ExecutorCacheMemoryPerInstance("Total Cache memory per node in bytes"),
   ExecutorJvmMaxMemory("Max memory available for JVM in bytes"),
   ExecutorWaitQueueSize("Size of wait queue per node"),
-  ExecutorRpcNumHandlers("Number of RPC handlers per node"),
-  ExecutorThreadCPUTime("Cpu time in nanoseconds"),
   ExecutorThreadUserTime("User time in nanoseconds"),
   ExecutorTotalRequestsHandled("Total number of requests handled by the container"),
   ExecutorNumQueuedRequests("Number of requests queued by the container for processing"),
+  ExecutorNumPreemptableRequests("Number of queued requests that are pre-emptable"),
+  ExecutorTotalRejectedRequests("Total number of requests rejected as wait queue being full"),
   ExecutorTotalSuccess("Total number of requests handled by the container that succeeded"),
-  ExecutorTotalExecutionFailure("Total number of requests handled by the container that failed execution"),
-  ExecutorTotalInterrupted("Total number of requests handled by the container that got interrupted"),
+  ExecutorTotalFailed("Total number of requests handled by the container that failed execution"),
+  ExecutorTotalKilled("Total number of requests handled by the container that got interrupted"),
   ExecutorTotalAskedToDie("Total number of requests handled by the container that were asked to die"),
-  PreemptionTimeLost("Total time lost due to task preemptions");
+  ExecutorTotalPreemptionTimeToKill("Total amount of time taken for killing tasks due to pre-emption"),
+  ExecutorTotalPreemptionTimeLost("Total useful cluster time lost because of pre-emption"),
+  ExecutorPercentileTimeToKill("Percentile time to kill for pre-empted tasks"),
+  ExecutorPercentileTimeLost("Percentile cluster time wasted due to pre-emption"),
+  ExecutorMaxPreemptionTimeToKill("Max time for killing pre-empted task"),
+  ExecutorMaxPreemptionTimeLost("Max cluster time lost due to pre-emption"),
+  ExecutorTotalEvictedFromWaitQueue("Total number of tasks evicted from wait queue because of low priority");
 
   private final String desc;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java
index 894880f..1110683 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java
@@ -17,22 +17,30 @@
  */
 package org.apache.hadoop.hive.llap.metrics;
 
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorAvailableFreeSlots;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorAvailableFreeSlotsPercent;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorCacheMemoryPerInstance;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorJvmMaxMemory;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorMaxFreeSlots;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorMaxPreemptionTimeLost;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorMaxPreemptionTimeToKill;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorMemoryPerInstance;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorNumExecutorsAvailable;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorNumPreemptableRequests;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorNumQueuedRequests;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorRpcNumHandlers;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorThreadCPUTime;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorThreadCountPerInstance;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorNumExecutorsPerInstance;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorThreadUserTime;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalAskedToDie;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalExecutionFailure;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalInterrupted;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalEvictedFromWaitQueue;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalFailed;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalKilled;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalRejectedRequests;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalRequestsHandled;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalSuccess;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorMetrics;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalPreemptionTimeLost;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorTotalPreemptionTimeToKill;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.ExecutorWaitQueueSize;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorInfo.PreemptionTimeLost;
 import static org.apache.hadoop.metrics2.impl.MsInfo.ProcessName;
 import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
 
@@ -54,6 +62,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MutableQuantiles;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 
 /**
@@ -70,13 +79,23 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
   private final ThreadMXBean threadMXBean;
   private final Map<Integer, MetricsInfo> cpuMetricsInfoMap;
   private final Map<Integer, MetricsInfo> userMetricsInfoMap;
+  private long maxTimeLost = Long.MIN_VALUE;
+  private long maxTimeToKill = Long.MIN_VALUE;
 
   final MutableGaugeLong[] executorThreadCpuTime;
   final MutableGaugeLong[] executorThreadUserTime;
   @Metric
   MutableCounterLong executorTotalRequestHandled;
   @Metric
-  MutableCounterLong executorNumQueuedRequests;
+  MutableGaugeInt executorNumQueuedRequests;
+  @Metric
+  MutableGaugeInt executorNumPreemptableRequests;
+  @Metric
+  MutableGaugeInt numExecutorsAvailable;
+  @Metric
+  MutableCounterLong totalRejectedRequests;
+  @Metric
+  MutableCounterLong totalEvictedFromWaitQueue;
   @Metric
   MutableCounterLong executorTotalSuccess;
   @Metric
@@ -84,8 +103,6 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
   @Metric
   MutableCounterLong executorTotalExecutionFailed;
   @Metric
-  MutableCounterLong preemptionTimeLost;
-  @Metric
   MutableGaugeLong cacheMemoryPerInstance;
   @Metric
   MutableGaugeLong memoryPerInstance;
@@ -94,10 +111,20 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
   @Metric
   MutableGaugeInt waitQueueSize;
   @Metric
-  MutableGaugeInt rpcNumHandlers;
+  MutableCounterLong totalPreemptionTimeToKill;
+  @Metric
+  MutableCounterLong totalPreemptionTimeLost;
+  @Metric
+  MutableGaugeLong maxPreemptionTimeToKill;
+  @Metric
+  MutableGaugeLong maxPreemptionTimeLost;
+  @Metric
+  final MutableQuantiles[] percentileTimeToKill;
+  @Metric
+  final MutableQuantiles[] percentileTimeLost;
 
   private LlapDaemonExecutorMetrics(String displayName, JvmMetrics jm, String sessionId,
-      int numExecutors) {
+      int numExecutors, final int[] intervals) {
     this.name = displayName;
     this.jvmMetrics = jm;
     this.sessionId = sessionId;
@@ -110,6 +137,21 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
     this.cpuMetricsInfoMap = new ConcurrentHashMap<>();
     this.userMetricsInfoMap = new ConcurrentHashMap<>();
 
+    final int len = intervals == null ? 0 : intervals.length;
+    this.percentileTimeToKill = new MutableQuantiles[len];
+    this.percentileTimeLost = new MutableQuantiles[len];
+    for (int i=0; i<len; i++) {
+      int interval = intervals[i];
+      percentileTimeToKill[i] = registry.newQuantiles(
+          LlapDaemonExecutorInfo.ExecutorMaxPreemptionTimeToKill.name() + "_" + interval + "s",
+          LlapDaemonExecutorInfo.ExecutorMaxPreemptionTimeToKill.description(),
+          "ops", "latency", interval);
+      percentileTimeLost[i] = registry.newQuantiles(
+          LlapDaemonExecutorInfo.ExecutorMaxPreemptionTimeLost.name() + "_" + interval + "s",
+          LlapDaemonExecutorInfo.ExecutorMaxPreemptionTimeLost.description(),
+          "ops", "latency", interval);
+    }
+
     for (int i = 0; i < numExecutors; i++) {
       MetricsInfo mic = new LlapDaemonCustomMetricsInfo(ExecutorThreadCPUTime.name() + "_" + i,
           ExecutorThreadCPUTime.description());
@@ -123,11 +165,11 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
   }
 
   public static LlapDaemonExecutorMetrics create(String displayName, String sessionId,
-      int numExecutors) {
+      int numExecutors, final int[] intervals) {
     MetricsSystem ms = LlapMetricsSystem.instance();
     JvmMetrics jm = JvmMetrics.create(MetricsUtils.METRICS_PROCESS_NAME, sessionId, ms);
     return ms.register(displayName, "LlapDaemon Executor Metrics",
-        new LlapDaemonExecutorMetrics(displayName, jm, sessionId, numExecutors));
+        new LlapDaemonExecutorMetrics(displayName, jm, sessionId, numExecutors, intervals));
   }
 
   @Override
@@ -143,12 +185,24 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
     executorTotalRequestHandled.incr();
   }
 
-  public void incrExecutorNumQueuedRequests() {
-    executorNumQueuedRequests.incr();
+  public void setExecutorNumQueuedRequests(int value) {
+    executorNumQueuedRequests.set(value);
+  }
+
+  public void setExecutorNumPreemptableRequests(int value) {
+    executorNumPreemptableRequests.set(value);
+  }
+
+  public void setNumExecutorsAvailable(int value) {
+    numExecutorsAvailable.set(value);
+  }
+
+  public void incrTotalEvictedFromWaitQueue() {
+    totalEvictedFromWaitQueue.incr();
   }
 
-  public void decrExecutorNumQueuedRequests() {
-    executorNumQueuedRequests.incr(-1);
+  public void incrTotalRejectedRequests() {
+    totalRejectedRequests.incr();
   }
 
   public void incrExecutorTotalSuccess() {
@@ -159,8 +213,30 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
     executorTotalExecutionFailed.incr();
   }
 
-  public void incrPreemptionTimeLost(long value) {
-    preemptionTimeLost.incr(value);
+  public void addMetricsPreemptionTimeLost(long value) {
+    totalPreemptionTimeLost.incr(value);
+
+    if (value > maxTimeLost) {
+      maxTimeLost = value;
+      maxPreemptionTimeLost.set(maxTimeLost);
+    }
+
+    for (MutableQuantiles q : percentileTimeLost) {
+      q.add(value);
+    }
+  }
+
+  public void addMetricsPreemptionTimeToKill(long value) {
+    totalPreemptionTimeToKill.incr(value);
+
+    if (value > maxTimeToKill) {
+      maxTimeToKill = value;
+      maxPreemptionTimeToKill.set(maxTimeToKill);
+    }
+
+    for (MutableQuantiles q : percentileTimeToKill) {
+      q.add(value);
+    }
   }
 
   public void incrExecutorTotalKilled() {
@@ -183,25 +259,43 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
     waitQueueSize.set(size);
   }
 
-  public void setRpcNumHandlers(int numHandlers) {
-    rpcNumHandlers.set(numHandlers);
-  }
-
   private void getExecutorStats(MetricsRecordBuilder rb) {
     updateThreadMetrics(rb);
+    final int totalSlots = waitQueueSize.value() + numExecutors;
+    final int slotsAvailableInQueue = waitQueueSize.value() - executorNumQueuedRequests.value();
+    final int slotsAvailableTotal = slotsAvailableInQueue + numExecutorsAvailable.value();
+    final float slotsAvailablePercent = totalSlots <= 0 ? 0.0f :
+        (float) slotsAvailableTotal / (float) totalSlots;
 
     rb.addCounter(ExecutorTotalRequestsHandled, executorTotalRequestHandled.value())
-        .addCounter(ExecutorNumQueuedRequests, executorNumQueuedRequests.value())
         .addCounter(ExecutorTotalSuccess, executorTotalSuccess.value())
-        .addCounter(ExecutorTotalExecutionFailure, executorTotalExecutionFailed.value())
-        .addCounter(ExecutorTotalInterrupted, executorTotalIKilled.value())
-        .addCounter(PreemptionTimeLost, preemptionTimeLost.value())
-        .addGauge(ExecutorThreadCountPerInstance, numExecutors)
+        .addCounter(ExecutorTotalFailed, executorTotalExecutionFailed.value())
+        .addCounter(ExecutorTotalKilled, executorTotalIKilled.value())
+        .addCounter(ExecutorTotalEvictedFromWaitQueue, totalEvictedFromWaitQueue.value())
+        .addCounter(ExecutorTotalRejectedRequests, totalRejectedRequests.value())
+        .addGauge(ExecutorNumQueuedRequests, executorNumQueuedRequests.value())
+        .addGauge(ExecutorNumPreemptableRequests, executorNumPreemptableRequests.value())
         .addGauge(ExecutorMemoryPerInstance, memoryPerInstance.value())
         .addGauge(ExecutorCacheMemoryPerInstance, cacheMemoryPerInstance.value())
         .addGauge(ExecutorJvmMaxMemory, jvmMaxMemory.value())
+        .addGauge(ExecutorMaxFreeSlots, totalSlots)
+        .addGauge(ExecutorNumExecutorsPerInstance, numExecutors)
         .addGauge(ExecutorWaitQueueSize, waitQueueSize.value())
-        .addGauge(ExecutorRpcNumHandlers, rpcNumHandlers.value());
+        .addGauge(ExecutorNumExecutorsAvailable, numExecutorsAvailable.value())
+        .addGauge(ExecutorAvailableFreeSlots, slotsAvailableTotal)
+        .addGauge(ExecutorAvailableFreeSlotsPercent, slotsAvailablePercent)
+        .addCounter(ExecutorTotalPreemptionTimeToKill, totalPreemptionTimeToKill.value())
+        .addCounter(ExecutorTotalPreemptionTimeLost, totalPreemptionTimeLost.value())
+        .addGauge(ExecutorMaxPreemptionTimeToKill, maxPreemptionTimeToKill.value())
+        .addGauge(ExecutorMaxPreemptionTimeLost, maxPreemptionTimeLost.value());
+
+    for (MutableQuantiles q : percentileTimeToKill) {
+      q.snapshot(rb, true);
+    }
+
+    for (MutableQuantiles q : percentileTimeLost) {
+      q.snapshot(rb, true);
+    }
   }
 
   private void updateThreadMetrics(MetricsRecordBuilder rb) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOInfo.java
index 79f004b..f0fde62 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOInfo.java
@@ -26,10 +26,6 @@ import com.google.common.base.Objects;
  */
 public enum LlapDaemonIOInfo implements MetricsInfo {
   IOMetrics("Llap daemon I/O elevator metrics"),
-  IoThreadPoolSize("Size of the thread pool used by IO elevator"),
-  EncodedColumnBatchPoolSize("Size of the object pool that stores encoded column batches"),
-  ColumnStreamDataPoolSize("Size of the object pool that stores column stream data"),
-  ColumnVectorBatchPoolSize("Size of the object pool that stores column vector batches"),
   PercentileDecodingTime("Percentile decoding time for encoded column batch"),
   MaxDecodingTime("Max time for decoding an encoded column batch");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOMetrics.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOMetrics.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOMetrics.java
index f3def75..36eb0e5 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOMetrics.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonIOMetrics.java
@@ -17,12 +17,8 @@
  */
 package org.apache.hadoop.hive.llap.metrics;
 
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonIOInfo.ColumnStreamDataPoolSize;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonIOInfo.ColumnVectorBatchPoolSize;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonIOInfo.EncodedColumnBatchPoolSize;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonIOInfo.IoThreadPoolSize;
-import static org.apache.hadoop.hive.llap.metrics.LlapDaemonIOInfo.MaxDecodingTime;
 import static org.apache.hadoop.hive.llap.metrics.LlapDaemonIOInfo.IOMetrics;
+import static org.apache.hadoop.hive.llap.metrics.LlapDaemonIOInfo.MaxDecodingTime;
 import static org.apache.hadoop.metrics2.impl.MsInfo.ProcessName;
 import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
 
@@ -33,7 +29,6 @@ import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.annotation.Metric;
 import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
-import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableQuantiles;
 import org.apache.hadoop.metrics2.lib.MutableRate;
@@ -52,14 +47,6 @@ public class LlapDaemonIOMetrics implements MetricsSource {
   private long maxTime = Long.MIN_VALUE;
 
   @Metric
-  MutableGaugeInt encodedColumnBatchPoolSize;
-  @Metric
-  MutableGaugeInt columnStreamDataPoolSize;
-  @Metric
-  MutableGaugeInt columnVectorBatchPool;
-  @Metric
-  MutableGaugeInt ioThreadPoolSize;
-  @Metric
   MutableRate rateOfDecoding;
   final MutableQuantiles[] decodingTimes;
   @Metric
@@ -101,22 +88,6 @@ public class LlapDaemonIOMetrics implements MetricsSource {
     return name;
   }
 
-  public void setEncodedColumnBatchPoolSize(int size) {
-    encodedColumnBatchPoolSize.set(size);
-  }
-
-  public void setColumnStreamDataPoolSize(int size) {
-    columnStreamDataPoolSize.set(size);
-  }
-
-  public void setColumnVectorBatchPoolSize(int size) {
-    columnVectorBatchPool.set(size);
-  }
-
-  public void setIoThreadPoolSize(int size) {
-    ioThreadPoolSize.set(size);
-  }
-
   public void addDecodeBatchTime(long latency) {
     rateOfDecoding.add(latency);
     if (latency > maxTime) {
@@ -129,11 +100,7 @@ public class LlapDaemonIOMetrics implements MetricsSource {
   }
 
   private void getIoStats(MetricsRecordBuilder rb) {
-    rb.addGauge(EncodedColumnBatchPoolSize, encodedColumnBatchPoolSize.value())
-        .addGauge(ColumnStreamDataPoolSize, columnStreamDataPoolSize.value())
-        .addGauge(ColumnVectorBatchPoolSize, columnVectorBatchPool.value())
-        .addGauge(IoThreadPoolSize, ioThreadPoolSize.value())
-        .addGauge(MaxDecodingTime, maxDecodingTime.value());
+    rb.addGauge(MaxDecodingTime, maxDecodingTime.value());
     rateOfDecoding.snapshot(rb, true);
 
     for (MutableQuantiles q : decodingTimes) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java
index d1edd12..506f611 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java
@@ -190,7 +190,7 @@ public class TestTaskExecutorService {
     public TaskExecutorServiceForTest(int numExecutors, int waitQueueSize, String waitQueueComparatorClassName,
                                       boolean enablePreemption) {
       super(numExecutors, waitQueueSize, waitQueueComparatorClassName, enablePreemption,
-          Thread.currentThread().getContextClassLoader());
+          Thread.currentThread().getContextClassLoader(), null);
     }
 
     private ConcurrentMap<String, InternalCompletionListenerForTest> completionListeners = new ConcurrentHashMap<>();

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
index 5ecbf79..c3d3a1d 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
@@ -56,10 +56,14 @@ import org.apache.commons.lang.mutable.MutableInt;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.metrics.LlapMetricsSystem;
+import org.apache.hadoop.hive.llap.metrics.MetricsUtils;
 import org.apache.hadoop.hive.llap.registry.ServiceInstance;
 import org.apache.hadoop.hive.llap.registry.ServiceInstanceSet;
 import org.apache.hadoop.hive.llap.registry.ServiceInstanceStateChangeListener;
 import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
+import org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerMetrics;
+import org.apache.hadoop.util.JvmPauseMonitor;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -173,12 +177,16 @@ public class LlapTaskSchedulerService extends TaskScheduler {
   @VisibleForTesting
   StatsPerDag dagStats = new StatsPerDag();
 
+  private final LlapTaskSchedulerMetrics metrics;
+  private final JvmPauseMonitor pauseMonitor;
+
   public LlapTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) {
-    this(taskSchedulerContext, new SystemClock());
+    this(taskSchedulerContext, new SystemClock(), true);
   }
 
   @VisibleForTesting
-  public LlapTaskSchedulerService(TaskSchedulerContext taskSchedulerContext, Clock clock) {
+  public LlapTaskSchedulerService(TaskSchedulerContext taskSchedulerContext, Clock clock,
+      boolean initMetrics) {
     super(taskSchedulerContext);
     this.clock = clock;
     try {
@@ -236,6 +244,24 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LlapScheduler").build());
     schedulerExecutor = MoreExecutors.listeningDecorator(schedulerExecutorServiceRaw);
 
+    if (initMetrics) {
+      // Initialize the metrics system
+      LlapMetricsSystem.initialize("LlapDaemon");
+      this.pauseMonitor = new JvmPauseMonitor(conf);
+      pauseMonitor.start();
+      String displayName = "LlapTaskSchedulerMetrics-" + MetricsUtils.getHostName();
+      String sessionId = conf.get("llap.daemon.metrics.sessionid");
+      // TODO: Not sure about the use of this. Should we instead use workerIdentity as sessionId?
+      this.metrics = LlapTaskSchedulerMetrics.create(displayName, sessionId);
+      this.metrics.setNumExecutors(executorsPerInstance);
+      this.metrics.setMemoryPerInstance(memoryPerInstance * 1024L * 1024L);
+      this.metrics.setCpuCoresPerInstance(coresPerExecutor);
+      this.metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
+    } else {
+      this.metrics = null;
+      this.pauseMonitor = null;
+    }
+
     LOG.info("Running with configuration: " + "memoryPerInstance=" + memoryPerInstance
         + ", vCoresPerInstance=" + coresPerInstance + ", executorsPerInstance="
         + executorsPerInstance + ", resourcePerInstanceInferred=" + resourcePerExecutor
@@ -280,7 +306,8 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       registry.registerStateChangeListener(new NodeStateChangeListener());
       activeInstances = registry.getInstances();
       for (ServiceInstance inst : activeInstances.getAll().values()) {
-        addNode(inst, new NodeInfo(inst, nodeBlacklistConf, clock, numSchedulableTasksPerNode));
+        addNode(inst, new NodeInfo(inst, nodeBlacklistConf, clock, numSchedulableTasksPerNode,
+            metrics));
       }
     } finally {
       writeLock.unlock();
@@ -293,14 +320,14 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     @Override
     public void onCreate(final ServiceInstance serviceInstance) {
       addNode(serviceInstance, new NodeInfo(serviceInstance, nodeBlacklistConf, clock,
-          numSchedulableTasksPerNode));
+          numSchedulableTasksPerNode, metrics));
       LOG.info("Added node with identity: {}", serviceInstance.getWorkerIdentity());
     }
 
     @Override
     public void onUpdate(final ServiceInstance serviceInstance) {
       instanceToNodeMap.put(serviceInstance.getWorkerIdentity(), new NodeInfo(serviceInstance,
-          nodeBlacklistConf, clock, numSchedulableTasksPerNode));
+          nodeBlacklistConf, clock, numSchedulableTasksPerNode, metrics));
       LOG.info("Updated node with identity: {}", serviceInstance.getWorkerIdentity());
     }
 
@@ -309,6 +336,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       // FIXME: disabling this for now
       // instanceToNodeMap.remove(serviceInstance.getWorkerIdentity());
       LOG.info("Removed node with identity: {}", serviceInstance.getWorkerIdentity());
+      if (metrics != null) {
+        metrics.setClusterNodeCount(activeInstances.size());
+      }
       // if there are no more nodes. Signal timeout monitor to start timer
       if (activeInstances.size() == 0) {
         LOG.info("No node found. Signalling scheduler timeout monitor thread to start timer.");
@@ -378,6 +408,15 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         if (registry != null) {
           registry.stop();
         }
+
+        if (pauseMonitor != null) {
+          pauseMonitor.stop();
+        }
+
+        if (metrics != null) {
+          LlapMetricsSystem.shutdown();
+        }
+
       }
     } finally {
       writeLock.unlock();
@@ -454,6 +493,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     // This is effectively DAG completed, and can be used to reset statistics being tracked.
     LOG.info("DAG: " + dagCounter.get() + " completed. Scheduling stats: " + dagStats);
     dagCounter.incrementAndGet();
+    if (metrics != null) {
+      metrics.incrCompletedDagCount();
+    }
     dagStats = new StatsPerDag();
   }
 
@@ -544,9 +586,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
           // Also reset commFailures since a task was able to communicate back and indicate success.
           nodeInfo.enableNode();
           // Re-insert into the queue to force the poll thread to remove the element.
-          if ( disabledNodesQueue.remove(nodeInfo)) {
-            disabledNodesQueue.add(nodeInfo);
-          }
+          reinsertNodeInfo(nodeInfo);
         }
         // In case of success, trigger a scheduling run for pending tasks.
         trySchedulingPendingTasks();
@@ -562,9 +602,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
             // Also reset commFailures since a task was able to communicate back and indicate success.
             nodeInfo.enableNode();
             // Re-insert into the queue to force the poll thread to remove the element.
-            if (disabledNodesQueue.remove(nodeInfo)) {
-              disabledNodesQueue.add(nodeInfo);
-            }
+            reinsertNodeInfo(nodeInfo);
           }
           // In case of success, trigger a scheduling run for pending tasks.
           trySchedulingPendingTasks();
@@ -599,6 +637,15 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     return true;
   }
 
+  private void reinsertNodeInfo(final NodeInfo nodeInfo) {
+    if ( disabledNodesQueue.remove(nodeInfo)) {
+      disabledNodesQueue.add(nodeInfo);
+    }
+    if (metrics != null) {
+      metrics.setDisabledNodeCount(disabledNodesQueue.size());
+    }
+  }
+
   @Override
   public Object deallocateContainer(ContainerId containerId) {
     LOG.debug("Ignoring deallocateContainer for containerId: " + containerId);
@@ -704,7 +751,8 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         if (inst.isAlive() && instanceToNodeMap.containsKey(inst.getWorkerIdentity()) == false) {
           /* that's a good node, not added to the allocations yet */
           LOG.info("Found a new node: " + inst + ".");
-          addNode(inst, new NodeInfo(inst, nodeBlacklistConf, clock, numSchedulableTasksPerNode));
+          addNode(inst, new NodeInfo(inst, nodeBlacklistConf, clock, numSchedulableTasksPerNode,
+              metrics));
         }
       }
     } finally {
@@ -720,6 +768,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       stopTimeoutMonitor();
     }
     instanceToNodeMap.put(inst.getWorkerIdentity(), node);
+    if (metrics != null) {
+      metrics.setClusterNodeCount(activeInstances.size());
+    }
     // Trigger scheduling since a new node became available.
     trySchedulingPendingTasks();
   }
@@ -752,6 +803,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         nodeInfo.disableNode(isCommFailure);
         // TODO: handle task to container map events in case of hard failures
         disabledNodesQueue.add(nodeInfo);
+        if (metrics != null) {
+          metrics.setDisabledNodeCount(disabledNodesQueue.size());
+        }
       }
     } finally {
       writeLock.unlock();
@@ -768,6 +822,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       }
       tasksAtPriority.add(taskInfo);
       knownTasks.putIfAbsent(taskInfo.task, taskInfo);
+      if (metrics != null) {
+        metrics.incrPendingTasksCount();
+      }
     } finally {
       writeLock.unlock();
     }
@@ -799,6 +856,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         runningTasks.put(priority, tasksAtpriority);
       }
       tasksAtpriority.add(taskInfo);
+      if (metrics != null) {
+        metrics.decrPendingTasksCount();
+      }
     } finally {
       writeLock.unlock();
     }
@@ -1034,6 +1094,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     writeLock.lock();
     try {
       pendingPreemptions.incrementAndGet();
+      if (metrics != null) {
+        metrics.incrPendingPreemptionTasksCount();
+      }
       MutableInt val = pendingPreemptionsPerHost.get(host);
       if (val == null) {
         val = new MutableInt(1);
@@ -1049,6 +1112,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     writeLock.lock();
     try {
       pendingPreemptions.decrementAndGet();
+      if (metrics != null) {
+        metrics.decrPendingPreemptionTasksCount();
+      }
       MutableInt val = pendingPreemptionsPerHost.get(host);
       Preconditions.checkNotNull(val);
       val.decrement();
@@ -1199,23 +1265,24 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     private int numPreemptedTasks = 0;
     private int numScheduledTasks = 0;
     private final int numSchedulableTasks;
-
+    private final LlapTaskSchedulerMetrics metrics;
 
     /**
      * Create a NodeInfo bound to a service instance
-     *
-     * @param serviceInstance         the associated serviceInstance
+     *  @param serviceInstance         the associated serviceInstance
      * @param blacklistConf           blacklist configuration
      * @param clock                   clock to use to obtain timing information
      * @param numSchedulableTasksConf number of schedulable tasks on the node. 0 represents auto
-     *                                detect based on the serviceInstance, -1 indicates indicates
-     *                                unlimited capacity
+*                                detect based on the serviceInstance, -1 indicates indicates
+     * @param metrics
      */
-    NodeInfo(ServiceInstance serviceInstance, NodeBlacklistConf blacklistConf, Clock clock, int numSchedulableTasksConf) {
+    NodeInfo(ServiceInstance serviceInstance, NodeBlacklistConf blacklistConf, Clock clock,
+        int numSchedulableTasksConf, final LlapTaskSchedulerMetrics metrics) {
       Preconditions.checkArgument(numSchedulableTasksConf >= -1, "NumSchedulableTasks must be >=-1");
       this.serviceInstance = serviceInstance;
       this.blacklistConf = blacklistConf;
       this.clock = clock;
+      this.metrics = metrics;
 
       if (numSchedulableTasksConf == 0) {
         int pendingQueueuCapacity = 0;
@@ -1234,6 +1301,9 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       } else {
         this.numSchedulableTasks = numSchedulableTasksConf;
       }
+      if (metrics != null) {
+        metrics.incrSchedulableTasksCount(numSchedulableTasks);
+      }
       LOG.info("Setting up node: " + serviceInstance + " with schedulableCapacity=" + this.numSchedulableTasks);
     }
 
@@ -1275,17 +1345,33 @@ public class LlapTaskSchedulerService extends TaskScheduler {
 
     void registerTaskScheduled() {
       numScheduledTasks++;
+      if (metrics != null) {
+        metrics.incrRunningTasksCount();
+        metrics.decrSchedulableTasksCount();
+      }
     }
 
     void registerTaskSuccess() {
       numSuccessfulTasks++;
       numScheduledTasks--;
+      if (metrics != null) {
+        metrics.incrSuccessfulTasksCount();
+        metrics.decrRunningTasksCount();
+        metrics.incrSchedulableTasksCount();
+      }
     }
 
     void registerUnsuccessfulTaskEnd(boolean wasPreempted) {
       numScheduledTasks--;
+      if (metrics != null) {
+        metrics.decrRunningTasksCount();
+        metrics.incrSchedulableTasksCount();
+      }
       if (wasPreempted) {
         numPreemptedTasks++;
+        if (metrics != null) {
+          metrics.incrPreemptedTasksCount();
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerInfo.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerInfo.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerInfo.java
new file mode 100644
index 0000000..c190be8
--- /dev/null
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerInfo.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap.tezplugins.metrics;
+
+import org.apache.hadoop.metrics2.MetricsInfo;
+
+import com.google.common.base.Objects;
+
+/**
+ * Metrics information for llap task scheduler.
+ */
+public enum LlapTaskSchedulerInfo implements MetricsInfo {
+  SchedulerMetrics("Llap task scheduler related metrics"),
+  SchedulerClusterNodeCount("Number of nodes in the cluster"),
+  SchedulerExecutorsPerInstance("Total number of executor threads per node"),
+  SchedulerMemoryPerInstance("Total memory for executors per node in bytes"),
+  SchedulerCpuCoresPerInstance("Total CPU vCores per node"),
+  SchedulerDisabledNodeCount("Number of nodes disabled temporarily"),
+  SchedulerPendingTaskCount("Number of pending tasks"),
+  SchedulerSchedulableTaskCount("Current slots available for scheduling tasks"),
+  SchedulerSuccessfulTaskCount("Total number of successful tasks"),
+  SchedulerRunningTaskCount("Total number of running tasks"),
+  SchedulerPendingPreemptionTaskCount("Total number of tasks pending for pre-emption"),
+  SchedulerPreemptedTaskCount("Total number of tasks pre-empted"),
+  SchedulerCompletedDagCount("Number of DAGs completed");
+
+  private final String desc;
+
+  LlapTaskSchedulerInfo(String desc) {
+    this.desc = desc;
+  }
+
+  @Override
+  public String description() {
+    return desc;
+  }
+
+  @Override
+  public String toString() {
+    return Objects.toStringHelper(this)
+        .add("name", name()).add("description", desc)
+        .toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java
new file mode 100644
index 0000000..b3230e2
--- /dev/null
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java
@@ -0,0 +1,197 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap.tezplugins.metrics;
+
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerClusterNodeCount;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerCompletedDagCount;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerCpuCoresPerInstance;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerDisabledNodeCount;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerExecutorsPerInstance;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerMemoryPerInstance;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerMetrics;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerPendingPreemptionTaskCount;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerPendingTaskCount;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerPreemptedTaskCount;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerRunningTaskCount;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerSchedulableTaskCount;
+import static org.apache.hadoop.hive.llap.tezplugins.metrics.LlapTaskSchedulerInfo.SchedulerSuccessfulTaskCount;
+import static org.apache.hadoop.metrics2.impl.MsInfo.ProcessName;
+import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
+
+import org.apache.hadoop.hive.llap.metrics.LlapMetricsSystem;
+import org.apache.hadoop.hive.llap.metrics.MetricsUtils;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.metrics2.source.JvmMetrics;
+
+/**
+ * Metrics about the llap daemon task scheduler.
+ */
+@Metrics(about = "LlapDaemon Task Scheduler Metrics", context = "scheduler")
+public class LlapTaskSchedulerMetrics implements MetricsSource {
+
+  private final String name;
+  private final JvmMetrics jvmMetrics;
+  private final String sessionId;
+  private final MetricsRegistry registry;
+  @Metric
+  MutableGaugeInt numExecutors;
+  @Metric
+  MutableGaugeLong memoryPerInstance;
+  @Metric
+  MutableGaugeInt cpuCoresPerInstance;
+  @Metric
+  MutableGaugeInt clusterNodeCount;
+  @Metric
+  MutableGaugeInt disabledNodeCount;
+  @Metric
+  MutableCounterInt pendingTasksCount;
+  @Metric
+  MutableCounterInt schedulableTasksCount;
+  @Metric
+  MutableCounterInt runningTasksCount;
+  @Metric
+  MutableCounterInt successfulTasksCount;
+  @Metric
+  MutableCounterInt preemptedTasksCount;
+  @Metric
+  MutableCounterInt completedDagcount;
+  @Metric
+  MutableCounterInt pendingPreemptionTasksCount;
+
+  private LlapTaskSchedulerMetrics(String displayName, JvmMetrics jm, String sessionId) {
+    this.name = displayName;
+    this.jvmMetrics = jm;
+    this.sessionId = sessionId;
+    this.registry = new MetricsRegistry("LlapTaskSchedulerMetricsRegistry");
+    this.registry.tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME).tag(SessionId, sessionId);
+  }
+
+  public static LlapTaskSchedulerMetrics create(String displayName, String sessionId) {
+    MetricsSystem ms = LlapMetricsSystem.instance();
+    JvmMetrics jm = JvmMetrics.create(MetricsUtils.METRICS_PROCESS_NAME, sessionId, ms);
+    return ms.register(displayName, "Llap Task Scheduler Metrics",
+        new LlapTaskSchedulerMetrics(displayName, jm, sessionId));
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean b) {
+    MetricsRecordBuilder rb = collector.addRecord(SchedulerMetrics)
+        .setContext("scheduler")
+        .tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME)
+        .tag(SessionId, sessionId);
+    getTaskSchedulerStats(rb);
+  }
+
+  public void setNumExecutors(int value) {
+    numExecutors.set(value);
+  }
+
+  public void setMemoryPerInstance(long value) {
+    memoryPerInstance.set(value);
+  }
+
+  public void setCpuCoresPerInstance(int value) {
+    cpuCoresPerInstance.set(value);
+  }
+
+  public void setClusterNodeCount(int value) {
+    clusterNodeCount.set(value);
+  }
+
+  public void setDisabledNodeCount(int value) {
+    disabledNodeCount.set(value);
+  }
+
+  public void incrPendingTasksCount() {
+    pendingTasksCount.incr();
+  }
+
+  public void decrPendingTasksCount() {
+    pendingTasksCount.incr(-1);
+  }
+
+  public void incrSchedulableTasksCount(int delta) {
+    schedulableTasksCount.incr(delta);
+  }
+
+  public void incrSchedulableTasksCount() {
+    schedulableTasksCount.incr();
+  }
+
+  public void decrSchedulableTasksCount() {
+    schedulableTasksCount.incr(-1);
+  }
+
+  public void incrSuccessfulTasksCount() {
+    successfulTasksCount.incr();
+  }
+
+  public void incrRunningTasksCount() {
+    runningTasksCount.incr();
+  }
+
+  public void decrRunningTasksCount() {
+    runningTasksCount.incr(-1);
+  }
+
+  public void incrPreemptedTasksCount() {
+    preemptedTasksCount.incr();
+  }
+
+  public void incrCompletedDagCount() {
+    completedDagcount.incr();
+  }
+
+  public void incrPendingPreemptionTasksCount() {
+    pendingPreemptionTasksCount.incr();
+  }
+
+  public void decrPendingPreemptionTasksCount() {
+    pendingPreemptionTasksCount.incr(-1);
+  }
+
+  private void getTaskSchedulerStats(MetricsRecordBuilder rb) {
+    rb.addGauge(SchedulerClusterNodeCount, clusterNodeCount.value())
+        .addGauge(SchedulerExecutorsPerInstance, numExecutors.value())
+        .addGauge(SchedulerMemoryPerInstance, memoryPerInstance.value())
+        .addGauge(SchedulerCpuCoresPerInstance, cpuCoresPerInstance.value())
+        .addGauge(SchedulerDisabledNodeCount, disabledNodeCount.value())
+        .addCounter(SchedulerPendingTaskCount, pendingTasksCount.value())
+        .addCounter(SchedulerSchedulableTaskCount, schedulableTasksCount.value())
+        .addCounter(SchedulerRunningTaskCount, runningTasksCount.value())
+        .addCounter(SchedulerSuccessfulTaskCount, successfulTasksCount.value())
+        .addCounter(SchedulerPendingPreemptionTaskCount, pendingPreemptionTasksCount.value())
+        .addCounter(SchedulerPreemptedTaskCount, preemptedTasksCount.value())
+        .addCounter(SchedulerCompletedDagCount, completedDagcount.value());
+  }
+
+  public JvmMetrics getJvmMetrics() {
+    return jvmMetrics;
+  }
+
+  public String getName() {
+    return name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0ebcd938/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskSchedulerService.java b/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskSchedulerService.java
index 36d8ffd..b2cd55e 100644
--- a/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskSchedulerService.java
+++ b/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskSchedulerService.java
@@ -627,7 +627,7 @@ public class TestLlapTaskSchedulerService {
 
     public LlapTaskSchedulerServiceForTest(
         TaskSchedulerContext appClient, Clock clock) {
-      super(appClient, clock);
+      super(appClient, clock, false);
     }
 
     @Override


[05/20] hive git commit: HIVE-13440 : remove hiveserver1 scripts and thrift generated files (Balint Molnar via Thejas Nair)

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-py/hive_service/ThriftHive.py
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/hive_service/ThriftHive.py b/service/src/gen/thrift/gen-py/hive_service/ThriftHive.py
deleted file mode 100644
index 978c2a3..0000000
--- a/service/src/gen/thrift/gen-py/hive_service/ThriftHive.py
+++ /dev/null
@@ -1,1674 +0,0 @@
-#
-# Autogenerated by Thrift Compiler (0.9.3)
-#
-# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-#
-#  options string: py
-#
-
-from thrift.Thrift import TType, TMessageType, TException, TApplicationException
-import hive_metastore.ThriftHiveMetastore
-import logging
-from ttypes import *
-from thrift.Thrift import TProcessor
-from thrift.transport import TTransport
-from thrift.protocol import TBinaryProtocol, TProtocol
-try:
-  from thrift.protocol import fastbinary
-except:
-  fastbinary = None
-
-
-class Iface(hive_metastore.ThriftHiveMetastore.Iface):
-  def execute(self, query):
-    """
-    Parameters:
-     - query
-    """
-    pass
-
-  def fetchOne(self):
-    pass
-
-  def fetchN(self, numRows):
-    """
-    Parameters:
-     - numRows
-    """
-    pass
-
-  def fetchAll(self):
-    pass
-
-  def getSchema(self):
-    pass
-
-  def getThriftSchema(self):
-    pass
-
-  def getClusterStatus(self):
-    pass
-
-  def getQueryPlan(self):
-    pass
-
-  def clean(self):
-    pass
-
-
-class Client(hive_metastore.ThriftHiveMetastore.Client, Iface):
-  def __init__(self, iprot, oprot=None):
-    hive_metastore.ThriftHiveMetastore.Client.__init__(self, iprot, oprot)
-
-  def execute(self, query):
-    """
-    Parameters:
-     - query
-    """
-    self.send_execute(query)
-    self.recv_execute()
-
-  def send_execute(self, query):
-    self._oprot.writeMessageBegin('execute', TMessageType.CALL, self._seqid)
-    args = execute_args()
-    args.query = query
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_execute(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = execute_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.ex is not None:
-      raise result.ex
-    return
-
-  def fetchOne(self):
-    self.send_fetchOne()
-    return self.recv_fetchOne()
-
-  def send_fetchOne(self):
-    self._oprot.writeMessageBegin('fetchOne', TMessageType.CALL, self._seqid)
-    args = fetchOne_args()
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_fetchOne(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = fetchOne_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.ex is not None:
-      raise result.ex
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchOne failed: unknown result")
-
-  def fetchN(self, numRows):
-    """
-    Parameters:
-     - numRows
-    """
-    self.send_fetchN(numRows)
-    return self.recv_fetchN()
-
-  def send_fetchN(self, numRows):
-    self._oprot.writeMessageBegin('fetchN', TMessageType.CALL, self._seqid)
-    args = fetchN_args()
-    args.numRows = numRows
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_fetchN(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = fetchN_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.ex is not None:
-      raise result.ex
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchN failed: unknown result")
-
-  def fetchAll(self):
-    self.send_fetchAll()
-    return self.recv_fetchAll()
-
-  def send_fetchAll(self):
-    self._oprot.writeMessageBegin('fetchAll', TMessageType.CALL, self._seqid)
-    args = fetchAll_args()
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_fetchAll(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = fetchAll_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.ex is not None:
-      raise result.ex
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchAll failed: unknown result")
-
-  def getSchema(self):
-    self.send_getSchema()
-    return self.recv_getSchema()
-
-  def send_getSchema(self):
-    self._oprot.writeMessageBegin('getSchema', TMessageType.CALL, self._seqid)
-    args = getSchema_args()
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_getSchema(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = getSchema_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.ex is not None:
-      raise result.ex
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getSchema failed: unknown result")
-
-  def getThriftSchema(self):
-    self.send_getThriftSchema()
-    return self.recv_getThriftSchema()
-
-  def send_getThriftSchema(self):
-    self._oprot.writeMessageBegin('getThriftSchema', TMessageType.CALL, self._seqid)
-    args = getThriftSchema_args()
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_getThriftSchema(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = getThriftSchema_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.ex is not None:
-      raise result.ex
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getThriftSchema failed: unknown result")
-
-  def getClusterStatus(self):
-    self.send_getClusterStatus()
-    return self.recv_getClusterStatus()
-
-  def send_getClusterStatus(self):
-    self._oprot.writeMessageBegin('getClusterStatus', TMessageType.CALL, self._seqid)
-    args = getClusterStatus_args()
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_getClusterStatus(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = getClusterStatus_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.ex is not None:
-      raise result.ex
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterStatus failed: unknown result")
-
-  def getQueryPlan(self):
-    self.send_getQueryPlan()
-    return self.recv_getQueryPlan()
-
-  def send_getQueryPlan(self):
-    self._oprot.writeMessageBegin('getQueryPlan', TMessageType.CALL, self._seqid)
-    args = getQueryPlan_args()
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_getQueryPlan(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = getQueryPlan_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.ex is not None:
-      raise result.ex
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getQueryPlan failed: unknown result")
-
-  def clean(self):
-    self.send_clean()
-    self.recv_clean()
-
-  def send_clean(self):
-    self._oprot.writeMessageBegin('clean', TMessageType.CALL, self._seqid)
-    args = clean_args()
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_clean(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = clean_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    return
-
-
-class Processor(hive_metastore.ThriftHiveMetastore.Processor, Iface, TProcessor):
-  def __init__(self, handler):
-    hive_metastore.ThriftHiveMetastore.Processor.__init__(self, handler)
-    self._processMap["execute"] = Processor.process_execute
-    self._processMap["fetchOne"] = Processor.process_fetchOne
-    self._processMap["fetchN"] = Processor.process_fetchN
-    self._processMap["fetchAll"] = Processor.process_fetchAll
-    self._processMap["getSchema"] = Processor.process_getSchema
-    self._processMap["getThriftSchema"] = Processor.process_getThriftSchema
-    self._processMap["getClusterStatus"] = Processor.process_getClusterStatus
-    self._processMap["getQueryPlan"] = Processor.process_getQueryPlan
-    self._processMap["clean"] = Processor.process_clean
-
-  def process(self, iprot, oprot):
-    (name, type, seqid) = iprot.readMessageBegin()
-    if name not in self._processMap:
-      iprot.skip(TType.STRUCT)
-      iprot.readMessageEnd()
-      x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name))
-      oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid)
-      x.write(oprot)
-      oprot.writeMessageEnd()
-      oprot.trans.flush()
-      return
-    else:
-      self._processMap[name](self, seqid, iprot, oprot)
-    return True
-
-  def process_execute(self, seqid, iprot, oprot):
-    args = execute_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = execute_result()
-    try:
-      self._handler.execute(args.query)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except HiveServerException as ex:
-      msg_type = TMessageType.REPLY
-      result.ex = ex
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("execute", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_fetchOne(self, seqid, iprot, oprot):
-    args = fetchOne_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = fetchOne_result()
-    try:
-      result.success = self._handler.fetchOne()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except HiveServerException as ex:
-      msg_type = TMessageType.REPLY
-      result.ex = ex
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("fetchOne", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_fetchN(self, seqid, iprot, oprot):
-    args = fetchN_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = fetchN_result()
-    try:
-      result.success = self._handler.fetchN(args.numRows)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except HiveServerException as ex:
-      msg_type = TMessageType.REPLY
-      result.ex = ex
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("fetchN", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_fetchAll(self, seqid, iprot, oprot):
-    args = fetchAll_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = fetchAll_result()
-    try:
-      result.success = self._handler.fetchAll()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except HiveServerException as ex:
-      msg_type = TMessageType.REPLY
-      result.ex = ex
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("fetchAll", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_getSchema(self, seqid, iprot, oprot):
-    args = getSchema_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getSchema_result()
-    try:
-      result.success = self._handler.getSchema()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except HiveServerException as ex:
-      msg_type = TMessageType.REPLY
-      result.ex = ex
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getSchema", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_getThriftSchema(self, seqid, iprot, oprot):
-    args = getThriftSchema_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getThriftSchema_result()
-    try:
-      result.success = self._handler.getThriftSchema()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except HiveServerException as ex:
-      msg_type = TMessageType.REPLY
-      result.ex = ex
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getThriftSchema", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_getClusterStatus(self, seqid, iprot, oprot):
-    args = getClusterStatus_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getClusterStatus_result()
-    try:
-      result.success = self._handler.getClusterStatus()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except HiveServerException as ex:
-      msg_type = TMessageType.REPLY
-      result.ex = ex
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getClusterStatus", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_getQueryPlan(self, seqid, iprot, oprot):
-    args = getQueryPlan_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getQueryPlan_result()
-    try:
-      result.success = self._handler.getQueryPlan()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except HiveServerException as ex:
-      msg_type = TMessageType.REPLY
-      result.ex = ex
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getQueryPlan", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_clean(self, seqid, iprot, oprot):
-    args = clean_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = clean_result()
-    try:
-      self._handler.clean()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("clean", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-
-# HELPER FUNCTIONS AND STRUCTURES
-
-class execute_args:
-  """
-  Attributes:
-   - query
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.STRING, 'query', None, None, ), # 1
-  )
-
-  def __init__(self, query=None,):
-    self.query = query
-
-  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.query = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('execute_args')
-    if self.query is not None:
-      oprot.writeFieldBegin('query', TType.STRING, 1)
-      oprot.writeString(self.query)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.query)
-    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 execute_result:
-  """
-  Attributes:
-   - ex
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.STRUCT, 'ex', (HiveServerException, HiveServerException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, ex=None,):
-    self.ex = ex
-
-  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.ex = HiveServerException()
-          self.ex.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('execute_result')
-    if self.ex is not None:
-      oprot.writeFieldBegin('ex', TType.STRUCT, 1)
-      self.ex.write(oprot)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.ex)
-    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 fetchOne_args:
-
-  thrift_spec = (
-  )
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('fetchOne_args')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class fetchOne_result:
-  """
-  Attributes:
-   - success
-   - ex
-  """
-
-  thrift_spec = (
-    (0, TType.STRING, 'success', None, None, ), # 0
-    (1, TType.STRUCT, 'ex', (HiveServerException, HiveServerException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, success=None, ex=None,):
-    self.success = success
-    self.ex = ex
-
-  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.STRING:
-          self.success = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.ex = HiveServerException()
-          self.ex.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('fetchOne_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.STRING, 0)
-      oprot.writeString(self.success)
-      oprot.writeFieldEnd()
-    if self.ex is not None:
-      oprot.writeFieldBegin('ex', TType.STRUCT, 1)
-      self.ex.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.ex)
-    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 fetchN_args:
-  """
-  Attributes:
-   - numRows
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.I32, 'numRows', None, None, ), # 1
-  )
-
-  def __init__(self, numRows=None,):
-    self.numRows = numRows
-
-  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.I32:
-          self.numRows = iprot.readI32()
-        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('fetchN_args')
-    if self.numRows is not None:
-      oprot.writeFieldBegin('numRows', TType.I32, 1)
-      oprot.writeI32(self.numRows)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.numRows)
-    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 fetchN_result:
-  """
-  Attributes:
-   - success
-   - ex
-  """
-
-  thrift_spec = (
-    (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0
-    (1, TType.STRUCT, 'ex', (HiveServerException, HiveServerException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, success=None, ex=None,):
-    self.success = success
-    self.ex = ex
-
-  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 = []
-          (_etype3, _size0) = iprot.readListBegin()
-          for _i4 in xrange(_size0):
-            _elem5 = iprot.readString()
-            self.success.append(_elem5)
-          iprot.readListEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.ex = HiveServerException()
-          self.ex.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('fetchN_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.LIST, 0)
-      oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter6 in self.success:
-        oprot.writeString(iter6)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
-    if self.ex is not None:
-      oprot.writeFieldBegin('ex', TType.STRUCT, 1)
-      self.ex.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.ex)
-    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 fetchAll_args:
-
-  thrift_spec = (
-  )
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('fetchAll_args')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class fetchAll_result:
-  """
-  Attributes:
-   - success
-   - ex
-  """
-
-  thrift_spec = (
-    (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0
-    (1, TType.STRUCT, 'ex', (HiveServerException, HiveServerException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, success=None, ex=None,):
-    self.success = success
-    self.ex = ex
-
-  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 = []
-          (_etype10, _size7) = iprot.readListBegin()
-          for _i11 in xrange(_size7):
-            _elem12 = iprot.readString()
-            self.success.append(_elem12)
-          iprot.readListEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.ex = HiveServerException()
-          self.ex.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('fetchAll_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.LIST, 0)
-      oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter13 in self.success:
-        oprot.writeString(iter13)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
-    if self.ex is not None:
-      oprot.writeFieldBegin('ex', TType.STRUCT, 1)
-      self.ex.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.ex)
-    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 getSchema_args:
-
-  thrift_spec = (
-  )
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('getSchema_args')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class getSchema_result:
-  """
-  Attributes:
-   - success
-   - ex
-  """
-
-  thrift_spec = (
-    (0, TType.STRUCT, 'success', (hive_metastore.ttypes.Schema, hive_metastore.ttypes.Schema.thrift_spec), None, ), # 0
-    (1, TType.STRUCT, 'ex', (HiveServerException, HiveServerException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, success=None, ex=None,):
-    self.success = success
-    self.ex = ex
-
-  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 = hive_metastore.ttypes.Schema()
-          self.success.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.ex = HiveServerException()
-          self.ex.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('getSchema_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.STRUCT, 0)
-      self.success.write(oprot)
-      oprot.writeFieldEnd()
-    if self.ex is not None:
-      oprot.writeFieldBegin('ex', TType.STRUCT, 1)
-      self.ex.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.ex)
-    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 getThriftSchema_args:
-
-  thrift_spec = (
-  )
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('getThriftSchema_args')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class getThriftSchema_result:
-  """
-  Attributes:
-   - success
-   - ex
-  """
-
-  thrift_spec = (
-    (0, TType.STRUCT, 'success', (hive_metastore.ttypes.Schema, hive_metastore.ttypes.Schema.thrift_spec), None, ), # 0
-    (1, TType.STRUCT, 'ex', (HiveServerException, HiveServerException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, success=None, ex=None,):
-    self.success = success
-    self.ex = ex
-
-  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 = hive_metastore.ttypes.Schema()
-          self.success.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.ex = HiveServerException()
-          self.ex.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('getThriftSchema_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.STRUCT, 0)
-      self.success.write(oprot)
-      oprot.writeFieldEnd()
-    if self.ex is not None:
-      oprot.writeFieldBegin('ex', TType.STRUCT, 1)
-      self.ex.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.ex)
-    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 getClusterStatus_args:
-
-  thrift_spec = (
-  )
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('getClusterStatus_args')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class getClusterStatus_result:
-  """
-  Attributes:
-   - success
-   - ex
-  """
-
-  thrift_spec = (
-    (0, TType.STRUCT, 'success', (HiveClusterStatus, HiveClusterStatus.thrift_spec), None, ), # 0
-    (1, TType.STRUCT, 'ex', (HiveServerException, HiveServerException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, success=None, ex=None,):
-    self.success = success
-    self.ex = ex
-
-  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 = HiveClusterStatus()
-          self.success.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.ex = HiveServerException()
-          self.ex.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('getClusterStatus_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.STRUCT, 0)
-      self.success.write(oprot)
-      oprot.writeFieldEnd()
-    if self.ex is not None:
-      oprot.writeFieldBegin('ex', TType.STRUCT, 1)
-      self.ex.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.ex)
-    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 getQueryPlan_args:
-
-  thrift_spec = (
-  )
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('getQueryPlan_args')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class getQueryPlan_result:
-  """
-  Attributes:
-   - success
-   - ex
-  """
-
-  thrift_spec = (
-    (0, TType.STRUCT, 'success', (queryplan.ttypes.QueryPlan, queryplan.ttypes.QueryPlan.thrift_spec), None, ), # 0
-    (1, TType.STRUCT, 'ex', (HiveServerException, HiveServerException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, success=None, ex=None,):
-    self.success = success
-    self.ex = ex
-
-  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 = queryplan.ttypes.QueryPlan()
-          self.success.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.ex = HiveServerException()
-          self.ex.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('getQueryPlan_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.STRUCT, 0)
-      self.success.write(oprot)
-      oprot.writeFieldEnd()
-    if self.ex is not None:
-      oprot.writeFieldBegin('ex', TType.STRUCT, 1)
-      self.ex.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.ex)
-    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 clean_args:
-
-  thrift_spec = (
-  )
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('clean_args')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class clean_result:
-
-  thrift_spec = (
-  )
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('clean_result')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-py/hive_service/__init__.py
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/hive_service/__init__.py b/service/src/gen/thrift/gen-py/hive_service/__init__.py
deleted file mode 100644
index 6a0dcc9..0000000
--- a/service/src/gen/thrift/gen-py/hive_service/__init__.py
+++ /dev/null
@@ -1 +0,0 @@
-__all__ = ['ttypes', 'constants', 'ThriftHive']

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-py/hive_service/constants.py
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/hive_service/constants.py b/service/src/gen/thrift/gen-py/hive_service/constants.py
deleted file mode 100644
index 4a6492b..0000000
--- a/service/src/gen/thrift/gen-py/hive_service/constants.py
+++ /dev/null
@@ -1,11 +0,0 @@
-#
-# Autogenerated by Thrift Compiler (0.9.3)
-#
-# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-#
-#  options string: py
-#
-
-from thrift.Thrift import TType, TMessageType, TException, TApplicationException
-from ttypes import *
-

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-py/hive_service/ttypes.py
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/hive_service/ttypes.py b/service/src/gen/thrift/gen-py/hive_service/ttypes.py
deleted file mode 100644
index e19fe2a..0000000
--- a/service/src/gen/thrift/gen-py/hive_service/ttypes.py
+++ /dev/null
@@ -1,260 +0,0 @@
-#
-# Autogenerated by Thrift Compiler (0.9.3)
-#
-# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-#
-#  options string: py
-#
-
-from thrift.Thrift import TType, TMessageType, TException, TApplicationException
-import fb303.ttypes
-import hive_metastore.ttypes
-import queryplan.ttypes
-
-
-from thrift.transport import TTransport
-from thrift.protocol import TBinaryProtocol, TProtocol
-try:
-  from thrift.protocol import fastbinary
-except:
-  fastbinary = None
-
-
-class JobTrackerState:
-  INITIALIZING = 1
-  RUNNING = 2
-
-  _VALUES_TO_NAMES = {
-    1: "INITIALIZING",
-    2: "RUNNING",
-  }
-
-  _NAMES_TO_VALUES = {
-    "INITIALIZING": 1,
-    "RUNNING": 2,
-  }
-
-
-class HiveClusterStatus:
-  """
-  Attributes:
-   - taskTrackers
-   - mapTasks
-   - reduceTasks
-   - maxMapTasks
-   - maxReduceTasks
-   - state
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.I32, 'taskTrackers', None, None, ), # 1
-    (2, TType.I32, 'mapTasks', None, None, ), # 2
-    (3, TType.I32, 'reduceTasks', None, None, ), # 3
-    (4, TType.I32, 'maxMapTasks', None, None, ), # 4
-    (5, TType.I32, 'maxReduceTasks', None, None, ), # 5
-    (6, TType.I32, 'state', None, None, ), # 6
-  )
-
-  def __init__(self, taskTrackers=None, mapTasks=None, reduceTasks=None, maxMapTasks=None, maxReduceTasks=None, state=None,):
-    self.taskTrackers = taskTrackers
-    self.mapTasks = mapTasks
-    self.reduceTasks = reduceTasks
-    self.maxMapTasks = maxMapTasks
-    self.maxReduceTasks = maxReduceTasks
-    self.state = state
-
-  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.I32:
-          self.taskTrackers = iprot.readI32()
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.I32:
-          self.mapTasks = iprot.readI32()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.I32:
-          self.reduceTasks = iprot.readI32()
-        else:
-          iprot.skip(ftype)
-      elif fid == 4:
-        if ftype == TType.I32:
-          self.maxMapTasks = iprot.readI32()
-        else:
-          iprot.skip(ftype)
-      elif fid == 5:
-        if ftype == TType.I32:
-          self.maxReduceTasks = iprot.readI32()
-        else:
-          iprot.skip(ftype)
-      elif fid == 6:
-        if ftype == TType.I32:
-          self.state = iprot.readI32()
-        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('HiveClusterStatus')
-    if self.taskTrackers is not None:
-      oprot.writeFieldBegin('taskTrackers', TType.I32, 1)
-      oprot.writeI32(self.taskTrackers)
-      oprot.writeFieldEnd()
-    if self.mapTasks is not None:
-      oprot.writeFieldBegin('mapTasks', TType.I32, 2)
-      oprot.writeI32(self.mapTasks)
-      oprot.writeFieldEnd()
-    if self.reduceTasks is not None:
-      oprot.writeFieldBegin('reduceTasks', TType.I32, 3)
-      oprot.writeI32(self.reduceTasks)
-      oprot.writeFieldEnd()
-    if self.maxMapTasks is not None:
-      oprot.writeFieldBegin('maxMapTasks', TType.I32, 4)
-      oprot.writeI32(self.maxMapTasks)
-      oprot.writeFieldEnd()
-    if self.maxReduceTasks is not None:
-      oprot.writeFieldBegin('maxReduceTasks', TType.I32, 5)
-      oprot.writeI32(self.maxReduceTasks)
-      oprot.writeFieldEnd()
-    if self.state is not None:
-      oprot.writeFieldBegin('state', TType.I32, 6)
-      oprot.writeI32(self.state)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.taskTrackers)
-    value = (value * 31) ^ hash(self.mapTasks)
-    value = (value * 31) ^ hash(self.reduceTasks)
-    value = (value * 31) ^ hash(self.maxMapTasks)
-    value = (value * 31) ^ hash(self.maxReduceTasks)
-    value = (value * 31) ^ hash(self.state)
-    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 HiveServerException(TException):
-  """
-  Attributes:
-   - message
-   - errorCode
-   - SQLState
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.STRING, 'message', None, None, ), # 1
-    (2, TType.I32, 'errorCode', None, None, ), # 2
-    (3, TType.STRING, 'SQLState', None, None, ), # 3
-  )
-
-  def __init__(self, message=None, errorCode=None, SQLState=None,):
-    self.message = message
-    self.errorCode = errorCode
-    self.SQLState = SQLState
-
-  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.message = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.I32:
-          self.errorCode = iprot.readI32()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.STRING:
-          self.SQLState = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('HiveServerException')
-    if self.message is not None:
-      oprot.writeFieldBegin('message', TType.STRING, 1)
-      oprot.writeString(self.message)
-      oprot.writeFieldEnd()
-    if self.errorCode is not None:
-      oprot.writeFieldBegin('errorCode', TType.I32, 2)
-      oprot.writeI32(self.errorCode)
-      oprot.writeFieldEnd()
-    if self.SQLState is not None:
-      oprot.writeFieldBegin('SQLState', TType.STRING, 3)
-      oprot.writeString(self.SQLState)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __str__(self):
-    return repr(self)
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.message)
-    value = (value * 31) ^ hash(self.errorCode)
-    value = (value * 31) ^ hash(self.SQLState)
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-rb/hive_service_constants.rb
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-rb/hive_service_constants.rb b/service/src/gen/thrift/gen-rb/hive_service_constants.rb
deleted file mode 100644
index ed7d2d7..0000000
--- a/service/src/gen/thrift/gen-rb/hive_service_constants.rb
+++ /dev/null
@@ -1,9 +0,0 @@
-#
-# Autogenerated by Thrift Compiler (0.9.3)
-#
-# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-#
-
-require 'thrift'
-require 'hive_service_types'
-

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-rb/hive_service_types.rb
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-rb/hive_service_types.rb b/service/src/gen/thrift/gen-rb/hive_service_types.rb
deleted file mode 100644
index 9191767..0000000
--- a/service/src/gen/thrift/gen-rb/hive_service_types.rb
+++ /dev/null
@@ -1,68 +0,0 @@
-#
-# Autogenerated by Thrift Compiler (0.9.3)
-#
-# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-#
-
-require 'thrift'
-require 'fb303_types'
-require 'hive_metastore_types'
-require 'queryplan_types'
-
-
-module JobTrackerState
-  INITIALIZING = 1
-  RUNNING = 2
-  VALUE_MAP = {1 => "INITIALIZING", 2 => "RUNNING"}
-  VALID_VALUES = Set.new([INITIALIZING, RUNNING]).freeze
-end
-
-class HiveClusterStatus
-  include ::Thrift::Struct, ::Thrift::Struct_Union
-  TASKTRACKERS = 1
-  MAPTASKS = 2
-  REDUCETASKS = 3
-  MAXMAPTASKS = 4
-  MAXREDUCETASKS = 5
-  STATE = 6
-
-  FIELDS = {
-    TASKTRACKERS => {:type => ::Thrift::Types::I32, :name => 'taskTrackers'},
-    MAPTASKS => {:type => ::Thrift::Types::I32, :name => 'mapTasks'},
-    REDUCETASKS => {:type => ::Thrift::Types::I32, :name => 'reduceTasks'},
-    MAXMAPTASKS => {:type => ::Thrift::Types::I32, :name => 'maxMapTasks'},
-    MAXREDUCETASKS => {:type => ::Thrift::Types::I32, :name => 'maxReduceTasks'},
-    STATE => {:type => ::Thrift::Types::I32, :name => 'state', :enum_class => ::JobTrackerState}
-  }
-
-  def struct_fields; FIELDS; end
-
-  def validate
-    unless @state.nil? || ::JobTrackerState::VALID_VALUES.include?(@state)
-      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field state!')
-    end
-  end
-
-  ::Thrift::Struct.generate_accessors self
-end
-
-class HiveServerException < ::Thrift::Exception
-  include ::Thrift::Struct, ::Thrift::Struct_Union
-  MESSAGE = 1
-  ERRORCODE = 2
-  SQLSTATE = 3
-
-  FIELDS = {
-    MESSAGE => {:type => ::Thrift::Types::STRING, :name => 'message'},
-    ERRORCODE => {:type => ::Thrift::Types::I32, :name => 'errorCode'},
-    SQLSTATE => {:type => ::Thrift::Types::STRING, :name => 'SQLState'}
-  }
-
-  def struct_fields; FIELDS; end
-
-  def validate
-  end
-
-  ::Thrift::Struct.generate_accessors self
-end
-

http://git-wip-us.apache.org/repos/asf/hive/blob/6a4e0806/service/src/gen/thrift/gen-rb/thrift_hive.rb
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-rb/thrift_hive.rb b/service/src/gen/thrift/gen-rb/thrift_hive.rb
deleted file mode 100644
index 2b57cd9..0000000
--- a/service/src/gen/thrift/gen-rb/thrift_hive.rb
+++ /dev/null
@@ -1,555 +0,0 @@
-#
-# Autogenerated by Thrift Compiler (0.9.3)
-#
-# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-#
-
-require 'thrift'
-require 'thrift_hive_metastore'
-require 'hive_service_types'
-
-module ThriftHive
-  class Client < ::ThriftHiveMetastore::Client 
-    include ::Thrift::Client
-
-    def execute(query)
-      send_execute(query)
-      recv_execute()
-    end
-
-    def send_execute(query)
-      send_message('execute', Execute_args, :query => query)
-    end
-
-    def recv_execute()
-      result = receive_message(Execute_result)
-      raise result.ex unless result.ex.nil?
-      return
-    end
-
-    def fetchOne()
-      send_fetchOne()
-      return recv_fetchOne()
-    end
-
-    def send_fetchOne()
-      send_message('fetchOne', FetchOne_args)
-    end
-
-    def recv_fetchOne()
-      result = receive_message(FetchOne_result)
-      return result.success unless result.success.nil?
-      raise result.ex unless result.ex.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'fetchOne failed: unknown result')
-    end
-
-    def fetchN(numRows)
-      send_fetchN(numRows)
-      return recv_fetchN()
-    end
-
-    def send_fetchN(numRows)
-      send_message('fetchN', FetchN_args, :numRows => numRows)
-    end
-
-    def recv_fetchN()
-      result = receive_message(FetchN_result)
-      return result.success unless result.success.nil?
-      raise result.ex unless result.ex.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'fetchN failed: unknown result')
-    end
-
-    def fetchAll()
-      send_fetchAll()
-      return recv_fetchAll()
-    end
-
-    def send_fetchAll()
-      send_message('fetchAll', FetchAll_args)
-    end
-
-    def recv_fetchAll()
-      result = receive_message(FetchAll_result)
-      return result.success unless result.success.nil?
-      raise result.ex unless result.ex.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'fetchAll failed: unknown result')
-    end
-
-    def getSchema()
-      send_getSchema()
-      return recv_getSchema()
-    end
-
-    def send_getSchema()
-      send_message('getSchema', GetSchema_args)
-    end
-
-    def recv_getSchema()
-      result = receive_message(GetSchema_result)
-      return result.success unless result.success.nil?
-      raise result.ex unless result.ex.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'getSchema failed: unknown result')
-    end
-
-    def getThriftSchema()
-      send_getThriftSchema()
-      return recv_getThriftSchema()
-    end
-
-    def send_getThriftSchema()
-      send_message('getThriftSchema', GetThriftSchema_args)
-    end
-
-    def recv_getThriftSchema()
-      result = receive_message(GetThriftSchema_result)
-      return result.success unless result.success.nil?
-      raise result.ex unless result.ex.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'getThriftSchema failed: unknown result')
-    end
-
-    def getClusterStatus()
-      send_getClusterStatus()
-      return recv_getClusterStatus()
-    end
-
-    def send_getClusterStatus()
-      send_message('getClusterStatus', GetClusterStatus_args)
-    end
-
-    def recv_getClusterStatus()
-      result = receive_message(GetClusterStatus_result)
-      return result.success unless result.success.nil?
-      raise result.ex unless result.ex.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'getClusterStatus failed: unknown result')
-    end
-
-    def getQueryPlan()
-      send_getQueryPlan()
-      return recv_getQueryPlan()
-    end
-
-    def send_getQueryPlan()
-      send_message('getQueryPlan', GetQueryPlan_args)
-    end
-
-    def recv_getQueryPlan()
-      result = receive_message(GetQueryPlan_result)
-      return result.success unless result.success.nil?
-      raise result.ex unless result.ex.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'getQueryPlan failed: unknown result')
-    end
-
-    def clean()
-      send_clean()
-      recv_clean()
-    end
-
-    def send_clean()
-      send_message('clean', Clean_args)
-    end
-
-    def recv_clean()
-      result = receive_message(Clean_result)
-      return
-    end
-
-  end
-
-  class Processor < ::ThriftHiveMetastore::Processor 
-    include ::Thrift::Processor
-
-    def process_execute(seqid, iprot, oprot)
-      args = read_args(iprot, Execute_args)
-      result = Execute_result.new()
-      begin
-        @handler.execute(args.query)
-      rescue ::HiveServerException => ex
-        result.ex = ex
-      end
-      write_result(result, oprot, 'execute', seqid)
-    end
-
-    def process_fetchOne(seqid, iprot, oprot)
-      args = read_args(iprot, FetchOne_args)
-      result = FetchOne_result.new()
-      begin
-        result.success = @handler.fetchOne()
-      rescue ::HiveServerException => ex
-        result.ex = ex
-      end
-      write_result(result, oprot, 'fetchOne', seqid)
-    end
-
-    def process_fetchN(seqid, iprot, oprot)
-      args = read_args(iprot, FetchN_args)
-      result = FetchN_result.new()
-      begin
-        result.success = @handler.fetchN(args.numRows)
-      rescue ::HiveServerException => ex
-        result.ex = ex
-      end
-      write_result(result, oprot, 'fetchN', seqid)
-    end
-
-    def process_fetchAll(seqid, iprot, oprot)
-      args = read_args(iprot, FetchAll_args)
-      result = FetchAll_result.new()
-      begin
-        result.success = @handler.fetchAll()
-      rescue ::HiveServerException => ex
-        result.ex = ex
-      end
-      write_result(result, oprot, 'fetchAll', seqid)
-    end
-
-    def process_getSchema(seqid, iprot, oprot)
-      args = read_args(iprot, GetSchema_args)
-      result = GetSchema_result.new()
-      begin
-        result.success = @handler.getSchema()
-      rescue ::HiveServerException => ex
-        result.ex = ex
-      end
-      write_result(result, oprot, 'getSchema', seqid)
-    end
-
-    def process_getThriftSchema(seqid, iprot, oprot)
-      args = read_args(iprot, GetThriftSchema_args)
-      result = GetThriftSchema_result.new()
-      begin
-        result.success = @handler.getThriftSchema()
-      rescue ::HiveServerException => ex
-        result.ex = ex
-      end
-      write_result(result, oprot, 'getThriftSchema', seqid)
-    end
-
-    def process_getClusterStatus(seqid, iprot, oprot)
-      args = read_args(iprot, GetClusterStatus_args)
-      result = GetClusterStatus_result.new()
-      begin
-        result.success = @handler.getClusterStatus()
-      rescue ::HiveServerException => ex
-        result.ex = ex
-      end
-      write_result(result, oprot, 'getClusterStatus', seqid)
-    end
-
-    def process_getQueryPlan(seqid, iprot, oprot)
-      args = read_args(iprot, GetQueryPlan_args)
-      result = GetQueryPlan_result.new()
-      begin
-        result.success = @handler.getQueryPlan()
-      rescue ::HiveServerException => ex
-        result.ex = ex
-      end
-      write_result(result, oprot, 'getQueryPlan', seqid)
-    end
-
-    def process_clean(seqid, iprot, oprot)
-      args = read_args(iprot, Clean_args)
-      result = Clean_result.new()
-      @handler.clean()
-      write_result(result, oprot, 'clean', seqid)
-    end
-
-  end
-
-  # HELPER FUNCTIONS AND STRUCTURES
-
-  class Execute_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    QUERY = 1
-
-    FIELDS = {
-      QUERY => {:type => ::Thrift::Types::STRING, :name => 'query'}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Execute_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    EX = 1
-
-    FIELDS = {
-      EX => {:type => ::Thrift::Types::STRUCT, :name => 'ex', :class => ::HiveServerException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class FetchOne_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-
-    FIELDS = {
-
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class FetchOne_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-    EX = 1
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::STRING, :name => 'success'},
-      EX => {:type => ::Thrift::Types::STRUCT, :name => 'ex', :class => ::HiveServerException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class FetchN_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    NUMROWS = 1
-
-    FIELDS = {
-      NUMROWS => {:type => ::Thrift::Types::I32, :name => 'numRows'}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class FetchN_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-    EX = 1
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRING}},
-      EX => {:type => ::Thrift::Types::STRUCT, :name => 'ex', :class => ::HiveServerException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class FetchAll_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-
-    FIELDS = {
-
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class FetchAll_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-    EX = 1
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRING}},
-      EX => {:type => ::Thrift::Types::STRUCT, :name => 'ex', :class => ::HiveServerException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class GetSchema_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-
-    FIELDS = {
-
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class GetSchema_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-    EX = 1
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::Schema},
-      EX => {:type => ::Thrift::Types::STRUCT, :name => 'ex', :class => ::HiveServerException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class GetThriftSchema_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-
-    FIELDS = {
-
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class GetThriftSchema_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-    EX = 1
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::Schema},
-      EX => {:type => ::Thrift::Types::STRUCT, :name => 'ex', :class => ::HiveServerException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class GetClusterStatus_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-
-    FIELDS = {
-
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class GetClusterStatus_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-    EX = 1
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::HiveClusterStatus},
-      EX => {:type => ::Thrift::Types::STRUCT, :name => 'ex', :class => ::HiveServerException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class GetQueryPlan_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-
-    FIELDS = {
-
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class GetQueryPlan_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-    EX = 1
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::QueryPlan},
-      EX => {:type => ::Thrift::Types::STRUCT, :name => 'ex', :class => ::HiveServerException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Clean_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-
-    FIELDS = {
-
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Clean_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-
-    FIELDS = {
-
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-end
-