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

[01/24] hive git commit: HIVE-13318: Cache the result of getTable from metastore (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/llap a7b0ca733 -> 79c1c691e


HIVE-13318: Cache the result of getTable from metastore (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/255069e4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/255069e4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/255069e4

Branch: refs/heads/llap
Commit: 255069e4f1bb1ac874f5a3472ebed1abf26e8187
Parents: 4e9f95a
Author: Pengcheng Xiong <px...@apache.org>
Authored: Wed Mar 30 14:43:44 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Wed Mar 30 14:43:44 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/ParseContext.java      |  7 +++++
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 29 +++++++++++---------
 .../hadoop/hive/ql/parse/TaskCompiler.java      |  2 +-
 3 files changed, 24 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/255069e4/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
index 95c254c..1bccf20 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
@@ -87,6 +87,7 @@ public class ParseContext {
   // reducer
   private Map<String, PrunedPartitionList> prunedPartitions;
   private Map<String, ReadEntity> viewAliasToInput;
+  private Map<String, Table> tabNameToTabObject;
 
   /**
    * The lineage information.
@@ -162,6 +163,7 @@ public class ParseContext {
       Context ctx, HashMap<String, String> idToTableNameMap, int destTableId,
       UnionProcContext uCtx, List<AbstractMapJoinOperator<? extends MapJoinDesc>> listMapJoinOpsNoReducer,
       Map<String, PrunedPartitionList> prunedPartitions,
+      Map<String, Table> tabNameToTabObject,
       HashMap<TableScanOperator, SampleDesc> opToSamplePruner,
       GlobalLimitCtx globalLimitCtx,
       HashMap<String, SplitSample> nameToSplitSample,
@@ -185,6 +187,7 @@ public class ParseContext {
     this.uCtx = uCtx;
     this.listMapJoinOpsNoReducer = listMapJoinOpsNoReducer;
     this.prunedPartitions = prunedPartitions;
+    this.tabNameToTabObject = tabNameToTabObject;
     this.opToSamplePruner = opToSamplePruner;
     this.nameToSplitSample = nameToSplitSample;
     this.globalLimitCtx = globalLimitCtx;
@@ -577,4 +580,8 @@ public class ParseContext {
   public void setNeedViewColumnAuthorization(boolean needViewColumnAuthorization) {
     this.needViewColumnAuthorization = needViewColumnAuthorization;
   }
+
+  public Map<String, Table> getTabNameToTabObject() {
+    return tabNameToTabObject;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/255069e4/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 adee14b..e81d46e 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
@@ -324,7 +324,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   protected AnalyzeRewriteContext analyzeRewrite;
 
   // A mapping from a tableName to a table object in metastore.
-  Map<String, Table> tableNameToMetaDataTableObject;
+  Map<String, Table> tabNameToTabObject;
 
   // The tokens we should ignore when we are trying to do table masking.
   private final Set<Integer> ignoredTokens = Sets.newHashSet(HiveParser.TOK_GROUPBY,
@@ -359,6 +359,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     listMapJoinOpsNoReducer = new ArrayList<AbstractMapJoinOperator<? extends MapJoinDesc>>();
     groupOpToInputTables = new HashMap<GroupByOperator, Set<String>>();
     prunedPartitions = new HashMap<String, PrunedPartitionList>();
+    tabNameToTabObject = new HashMap<String, Table>();
     unparseTranslator = new UnparseTranslator(conf);
     autogenColAliasPrfxLbl = HiveConf.getVar(conf,
         HiveConf.ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL);
@@ -371,7 +372,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     viewAliasToInput = new HashMap<String, ReadEntity>();
     noscan = partialscan = false;
     tableMask = new TableMask(this, conf);
-    tableNameToMetaDataTableObject = new HashMap<>();
+    tabNameToTabObject = new HashMap<>();
   }
 
   @Override
@@ -380,6 +381,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     if(clearPartsCache) {
       prunedPartitions.clear();
     }
+    tabNameToTabObject.clear();
     loadTableWork.clear();
     loadFileWork.clear();
     topOps.clear();
@@ -429,6 +431,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     uCtx = pctx.getUCtx();
     listMapJoinOpsNoReducer = pctx.getListMapJoinOpsNoReducer();
     prunedPartitions = pctx.getPrunedPartitions();
+    tabNameToTabObject = pctx.getTabNameToTabObject();
     fetchTask = pctx.getFetchTask();
     setLineageInfo(pctx.getLineageInfo());
   }
@@ -440,7 +443,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         new HashSet<JoinOperator>(joinContext.keySet()),
         new HashSet<SMBMapJoinOperator>(smbMapJoinContext.keySet()),
         loadTableWork, loadFileWork, ctx, idToTableNameMap, destTableId, uCtx,
-        listMapJoinOpsNoReducer, prunedPartitions,
+        listMapJoinOpsNoReducer, prunedPartitions, tabNameToTabObject,
         opToSamplePruner, globalLimitCtx, nameToSplitSample, inputs, rootTasks,
         opToPartToSkewedPruner, viewAliasToInput, reduceSinkOperatorsAddedByEnforceBucketingSorting,
         analyzeRewrite, tableDesc, queryProperties, viewProjectToTableSchema);
@@ -1606,7 +1609,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           }
           Table table = null;
           try {
-            table = db.getTable(tableName);
+            table = this.getTableObjectByName(tableName);
           } catch (HiveException ex) {
             throw new SemanticException(ex);
           }
@@ -10344,13 +10347,13 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
-  private Table getMetaDataTableObjectByName(String tableName) throws HiveException {
-    if (!tableNameToMetaDataTableObject.containsKey(tableName)) {
+  private Table getTableObjectByName(String tableName) throws HiveException {
+    if (!tabNameToTabObject.containsKey(tableName)) {
       Table table = db.getTable(tableName);
-      tableNameToMetaDataTableObject.put(tableName, table);
+      tabNameToTabObject.put(tableName, table);
       return table;
     } else {
-      return tableNameToMetaDataTableObject.get(tableName);
+      return tabNameToTabObject.get(tableName);
     }
   }
 
@@ -10400,7 +10403,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         String replacementText = null;
         Table table = null;
         try {
-          table = getMetaDataTableObjectByName(tabIdName);
+          table = getTableObjectByName(tabIdName);
         } catch (HiveException e) {
           throw new SemanticException("Table " + tabIdName + " is not found.");
         }
@@ -10636,7 +10639,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         new HashSet<JoinOperator>(joinContext.keySet()),
         new HashSet<SMBMapJoinOperator>(smbMapJoinContext.keySet()),
         loadTableWork, loadFileWork, ctx, idToTableNameMap, destTableId, uCtx,
-        listMapJoinOpsNoReducer, prunedPartitions, opToSamplePruner,
+        listMapJoinOpsNoReducer, prunedPartitions, tabNameToTabObject, opToSamplePruner,
         globalLimitCtx, nameToSplitSample, inputs, rootTasks, opToPartToSkewedPruner,
         viewAliasToInput, reduceSinkOperatorsAddedByEnforceBucketingSorting,
         analyzeRewrite, tableDesc, queryProperties, viewProjectToTableSchema);
@@ -11671,7 +11674,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       Set<String> tableAliases = qb.getTabAliases();
       for (String alias : tableAliases) {
         try {
-          Table table = db.getTable(qb.getTabNameForAlias(alias));
+          Table table = this.getTableObjectByName(qb.getTabNameForAlias(alias));
           if (table.isTemporary()) {
             throw new SemanticException("View definition references temporary table " + alias);
           }
@@ -11874,7 +11877,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     String tableName = getUnescapedName((ASTNode) tree.getChild(0).getChild(0));
     Table tbl;
     try {
-      tbl = db.getTable(tableName);
+      tbl = this.getTableObjectByName(tableName);
     } catch (InvalidTableException e) {
       throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName), e);
     }
@@ -11903,7 +11906,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     String tableName = getUnescapedName((ASTNode) tree.getChild(0).getChild(0));
     Table tbl;
     try {
-      tbl = db.getTable(tableName);
+      tbl = this.getTableObjectByName(tableName);
     } catch (InvalidTableException e) {
       throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName), e);
     } catch (HiveException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/255069e4/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index 8e64a0b..f7d7a40 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -399,7 +399,7 @@ public abstract class TaskCompiler {
         pCtx.getLoadTableWork(), pCtx.getLoadFileWork(), pCtx.getContext(),
         pCtx.getIdToTableNameMap(), pCtx.getDestTableId(), pCtx.getUCtx(),
         pCtx.getListMapJoinOpsNoReducer(),
-        pCtx.getPrunedPartitions(), pCtx.getOpToSamplePruner(), pCtx.getGlobalLimitCtx(),
+        pCtx.getPrunedPartitions(), pCtx.getTabNameToTabObject(), pCtx.getOpToSamplePruner(), pCtx.getGlobalLimitCtx(),
         pCtx.getNameToSplitSample(), pCtx.getSemanticInputs(), rootTasks,
         pCtx.getOpToPartToSkewedPruner(), pCtx.getViewAliasToInput(),
         pCtx.getReduceSinkOperatorsAddedByEnforceBucketingSorting(),


[07/24] hive git commit: HIVE-13402 : Temporarily disable failing spark tests (Siddharth Seth via Ashutosh Chauhan)

Posted by ss...@apache.org.
HIVE-13402 : Temporarily disable failing spark tests (Siddharth Seth via Ashutosh Chauhan)


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

Branch: refs/heads/llap
Commit: e085b7e9bd059d91aaf013df0db4d71dca90ec6f
Parents: 547c5cf
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Thu Mar 31 15:51:01 2016 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Thu Mar 31 15:51:01 2016 -0700

----------------------------------------------------------------------
 itests/src/test/resources/testconfiguration.properties | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e085b7e9/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 232f84e..cf9e4c9 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -886,7 +886,6 @@ spark.query.files=add_part_multiple.q, \
   input18.q, \
   input1_limit.q, \
   input_part2.q, \
-  insert1.q, \
   insert_into1.q, \
   insert_into2.q, \
   insert_into3.q, \
@@ -946,7 +945,6 @@ spark.query.files=add_part_multiple.q, \
   join_cond_pushdown_unqual2.q, \
   join_cond_pushdown_unqual3.q, \
   join_cond_pushdown_unqual4.q, \
-  join_empty.q, \
   join_filters_overlap.q, \
   join_hive_626.q, \
   join_literals.q, \
@@ -1154,7 +1152,6 @@ spark.query.files=add_part_multiple.q, \
   timestamp_lazy.q, \
   timestamp_null.q, \
   timestamp_udf.q, \
-  transform1.q, \
   transform2.q, \
   transform_ppr1.q, \
   transform_ppr2.q, \


[24/24] hive git commit: HIVE-HIVE-13419 part 2. Merge remote-tracking branch 'origin/master' into llap

Posted by ss...@apache.org.
HIVE-HIVE-13419 part 2. Merge remote-tracking branch 'origin/master' into llap

Conflicts:
	llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java


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

Branch: refs/heads/llap
Commit: 79c1c691e529107cba27eabefb94707645a39987
Parents: a7b0ca7 91ab819
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 4 15:37:10 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Apr 4 15:37:10 2016 -0700

----------------------------------------------------------------------
 beeline/pom.xml.orig                            |  169 +
 .../java/org/apache/hive/beeline/BeeLine.java   |   18 +-
 .../apache/hive/beeline/cli/TestHiveCli.java    |   15 +-
 data/conf/llap/hive-site.xml                    |   44 +
 data/conf/llap/llap-daemon-site.xml             |   61 -
 .../org/apache/hive/minikdc/MiniHiveKdc.java    |   46 +-
 .../minikdc/TestJdbcNonKrbSASLWithMiniKdc.java  |  103 +
 itests/hive-unit/pom.xml                        |   13 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   26 +-
 .../test/resources/testconfiguration.properties |    7 -
 itests/util/pom.xml                             |   11 +
 .../apache/hadoop/hive/llap/LlapItUtils.java    |   90 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   25 +-
 .../impl/LlapZookeeperRegistryImpl.java         |    9 +-
 .../hive/llap/tez/LlapProtocolClientProxy.java  |    4 +-
 .../hive/llap/cli/LlapOptionsProcessor.java     |   25 +-
 .../configuration/LlapDaemonConfiguration.java  |    8 +
 .../hive/llap/daemon/impl/LlapDaemon.java       |   34 +-
 .../daemon/impl/LlapProtocolServerImpl.java     |    7 +-
 .../daemon/services/impl/LlapWebServices.java   |   13 +-
 .../hive/llap/daemon/MiniLlapCluster.java       |  147 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |   37 +-
 .../tezplugins/LlapTaskSchedulerService.java    |    7 +
 .../tezplugins/helpers/SourceStateTracker.java  |    2 +-
 .../tezplugins/TestLlapTaskCommunicator.java    |  304 +-
 .../java/org/apache/orc/impl/IntegerReader.java |    4 +-
 .../apache/orc/impl/RunLengthIntegerReader.java |    7 +-
 .../orc/impl/RunLengthIntegerReaderV2.java      |    7 +-
 .../org/apache/orc/impl/SerializationUtils.java |   34 +-
 .../apache/orc/impl/TestSerializationUtils.java |   45 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    9 +-
 .../ql/exec/spark/HiveSparkClientFactory.java   |    9 +
 .../ql/exec/spark/RemoteHiveSparkClient.java    |   20 +-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |    3 +
 .../exec/spark/status/LocalSparkJobMonitor.java |    2 +-
 .../spark/status/RemoteSparkJobMonitor.java     |    5 +-
 .../ql/exec/vector/VectorizationContext.java    |   12 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   16 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |   46 +-
 .../hadoop/hive/ql/parse/ParseContext.java      |    7 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   29 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      |    2 +-
 .../hive/ql/plan/ExprNodeGenericFuncDesc.java   |    5 +-
 .../hive/ql/exec/TestFunctionRegistry.java      |   18 +-
 .../exec/vector/TestVectorizationContext.java   |   17 +-
 .../queries/clientpositive/cast_on_constant.q   |    7 +
 ql/src/test/queries/clientpositive/macro_1.q    |   29 +
 .../clientpositive/cast_on_constant.q.out       |  160 +
 .../test/results/clientpositive/macro_1.q.out   |   84 +
 .../clientpositive/spark/temp_table.q.out       |    2 +
 service-rpc/src/gen/thrift/gen-py/__init__.py   |    0
 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 +
 service/src/gen/thrift/gen-py/__init__.py       |    0
 .../gen-py/hive_service/ThriftHive-remote       | 1242 +++
 .../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 ++
 .../hive/service/auth/HiveAuthFactory.java      |   13 +-
 .../service/cli/thrift/ThriftCLIService.java    |   15 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |   66 +-
 .../apache/hadoop/hive/shims/HadoopShims.java   |    2 +-
 .../hive/spark/client/SparkClientImpl.java      |    9 +-
 78 files changed, 22374 insertions(+), 387 deletions(-)
----------------------------------------------------------------------


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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --cc llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index e9213ee,2fe59a2..418d87a
--- 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
@@@ -290,9 -292,23 +293,24 @@@ public class LlapDaemon extends Composi
      ShuffleHandler.initializeAndStart(shuffleHandlerConf);
      LOG.info("Setting shuffle port to: " + ShuffleHandler.get().getPort());
      this.shufflePort.set(ShuffleHandler.get().getPort());
+     getConfig()
+         .setInt(ConfVars.LLAP_DAEMON_YARN_SHUFFLE_PORT.varname, ShuffleHandler.get().getPort());
      super.serviceStart();
+ 
+     // Setup the actual ports in the configuration.
+     getConfig().setInt(ConfVars.LLAP_DAEMON_RPC_PORT.varname, server.getBindAddress().getPort());
+     getConfig().setInt(ConfVars.LLAP_MANAGEMENT_RPC_PORT.varname, server.getManagementBindAddress().getPort());
+     if (webServices != null) {
+       getConfig().setInt(ConfVars.LLAP_DAEMON_WEB_PORT.varname, webServices.getPort());
+     }
 +    LlapOutputFormatService.get();
-     LOG.info("LlapDaemon serviceStart complete");
+ 
+     this.registry.init(getConfig());
+     this.registry.start();
+     LOG.info(
+         "LlapDaemon serviceStart complete. RPC Port={}, ManagementPort={}, ShuflePort={}, WebPort={}",
+         server.getBindAddress().getPort(), server.getManagementBindAddress().getPort(),
+         ShuffleHandler.get().getPort(), (webServices == null ? "" : webServices.getPort()));
    }
  
    public void serviceStop() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------


[13/24] hive git commit: HIVE-13401:Kerberized HS2 with LDAP auth enabled fails kerberos/delegation token authentication (Chaoyu Tang, reviewed by Szehon Ho)

Posted by ss...@apache.org.
HIVE-13401:Kerberized HS2 with LDAP auth enabled fails kerberos/delegation token authentication (Chaoyu Tang, 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/6a1f8a83
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6a1f8a83
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6a1f8a83

Branch: refs/heads/llap
Commit: 6a1f8a835e13d0e0d3500fce02da8c14de34a023
Parents: d94e8d0
Author: ctang <ct...@cloudera.com>
Authored: Sat Apr 2 21:26:22 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Sat Apr 2 21:26:22 2016 -0400

----------------------------------------------------------------------
 .../org/apache/hive/minikdc/MiniHiveKdc.java    |  46 +++++++--
 .../minikdc/TestJdbcNonKrbSASLWithMiniKdc.java  | 103 +++++++++++++++++++
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |  14 ++-
 .../hive/service/auth/HiveAuthFactory.java      |  13 ++-
 .../service/cli/thrift/ThriftCLIService.java    |  15 +--
 5 files changed, 162 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java
----------------------------------------------------------------------
diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java
index 4e3a9c5..bbec37e 100644
--- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java
+++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java
@@ -50,6 +50,7 @@ public class MiniHiveKdc {
   public static String HIVE_TEST_USER_1 = "user1";
   public static String HIVE_TEST_USER_2 = "user2";
   public static String HIVE_TEST_SUPER_USER = "superuser";
+  public static String AUTHENTICATION_TYPE = "KERBEROS";
 
   private final MiniKdc miniKdc;
   private final File workDir;
@@ -170,16 +171,29 @@ public class MiniHiveKdc {
    * @throws Exception
    */
   public static MiniHS2 getMiniHS2WithKerb(MiniHiveKdc miniHiveKdc, HiveConf hiveConf) throws Exception {
-    String hivePrincipal =
-        miniHiveKdc.getFullyQualifiedServicePrincipal(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL);
-    String hiveKeytab = miniHiveKdc.getKeyTabFile(
-        miniHiveKdc.getServicePrincipalForUser(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL));
-
-    return new MiniHS2.Builder().withConf(hiveConf).
-        withMiniKdc(hivePrincipal, hiveKeytab).build();
+    return getMiniHS2WithKerb(miniHiveKdc, hiveConf, AUTHENTICATION_TYPE);
   }
 
   /**
+  * Create a MiniHS2 with the hive service principal and keytab in MiniHiveKdc
+  * @param miniHiveKdc
+  * @param hiveConf
+  * @param authType
+  * @return new MiniHS2 instance
+  * @throws Exception
+  */
+ public static MiniHS2 getMiniHS2WithKerb(MiniHiveKdc miniHiveKdc, HiveConf hiveConf,
+     String authType) throws Exception {
+   String hivePrincipal =
+       miniHiveKdc.getFullyQualifiedServicePrincipal(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL);
+   String hiveKeytab = miniHiveKdc.getKeyTabFile(
+       miniHiveKdc.getServicePrincipalForUser(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL));
+
+   return new MiniHS2.Builder().withConf(hiveConf).withMiniKdc(hivePrincipal, hiveKeytab).
+       withAuthenticationType(authType).build();
+ }
+
+  /**
    * Create a MiniHS2 with the hive service principal and keytab in MiniHiveKdc
    * @param miniHiveKdc
    * @param hiveConf
@@ -187,12 +201,26 @@ public class MiniHiveKdc {
    * @throws Exception
    */
   public static MiniHS2 getMiniHS2WithKerbWithRemoteHMS(MiniHiveKdc miniHiveKdc, HiveConf hiveConf) throws Exception {
+    return getMiniHS2WithKerbWithRemoteHMS(miniHiveKdc, hiveConf, AUTHENTICATION_TYPE);
+  }
+
+  /**
+   * Create a MiniHS2 with the hive service principal and keytab in MiniHiveKdc. It uses remote HMS
+   * and can support a different Sasl authType
+   * @param miniHiveKdc
+   * @param hiveConf
+   * @param authType
+   * @return new MiniHS2 instance
+   * @throws Exception
+   */
+  public static MiniHS2 getMiniHS2WithKerbWithRemoteHMS(MiniHiveKdc miniHiveKdc, HiveConf hiveConf,
+      String authType) throws Exception {
     String hivePrincipal =
         miniHiveKdc.getFullyQualifiedServicePrincipal(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL);
     String hiveKeytab = miniHiveKdc.getKeyTabFile(
         miniHiveKdc.getServicePrincipalForUser(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL));
 
     return new MiniHS2.Builder().withConf(hiveConf).withRemoteMetastore().
-        withMiniKdc(hivePrincipal, hiveKeytab).build();
+        withMiniKdc(hivePrincipal, hiveKeytab).withAuthenticationType(authType).build();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcNonKrbSASLWithMiniKdc.java
----------------------------------------------------------------------
diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcNonKrbSASLWithMiniKdc.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcNonKrbSASLWithMiniKdc.java
new file mode 100644
index 0000000..1c1beda
--- /dev/null
+++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcNonKrbSASLWithMiniKdc.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.minikdc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+
+import javax.security.sasl.AuthenticationException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.hive.service.auth.PasswdAuthenticationProvider;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestJdbcNonKrbSASLWithMiniKdc extends TestJdbcWithMiniKdc{
+
+  public static class CustomAuthenticator implements PasswdAuthenticationProvider {
+    @Override
+    public void Authenticate(String user, String password) throws AuthenticationException {
+      if (!("nonkrbuser".equals(user) && "mypwd".equals(password))) {
+        throw new AuthenticationException("Authentication failed");
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    Class.forName(MiniHS2.getJdbcDriverName());
+    confOverlay.put(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname,
+        SessionHookTest.class.getName());
+    confOverlay.put(ConfVars.HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS.varname,
+        CustomAuthenticator.class.getName());
+    HiveConf hiveConf = new HiveConf();
+    miniHiveKdc = MiniHiveKdc.getMiniHiveKdc(hiveConf);
+    miniHS2 = MiniHiveKdc.getMiniHS2WithKerbWithRemoteHMS(miniHiveKdc, hiveConf, "CUSTOM");
+    miniHS2.start(confOverlay);
+  }
+
+  /***
+   * Test a nonkrb user could login the kerberized HS2 with authentication type SASL NONE
+   * @throws Exception
+   */
+  @Test
+  public void testNonKrbSASLAuth() throws Exception {
+    hs2Conn = DriverManager.getConnection(miniHS2.getBaseJdbcURL() + "default;user=nonkrbuser;password=mypwd");
+    verifyProperty(SESSION_USER_NAME, "nonkrbuser");
+    hs2Conn.close();
+  }
+
+  /***
+   * Negative test, verify that connection to secure HS2 fails if it is noSasl
+   * @throws Exception
+   */
+  @Test
+  public void testNoSaslConnectionNeg() throws Exception {
+    try {
+      String url = miniHS2.getBaseJdbcURL() + "default;auth=noSasl";
+      hs2Conn = DriverManager.getConnection(url);
+      fail("noSasl connection should fail");
+    } catch (SQLException e) {
+      // expected error
+      assertEquals("08S01", e.getSQLState().trim());
+    }
+  }
+
+  /***
+   * Negative test, verify that NonKrb connection to secure HS2 fails if it is
+   * user/pwd do not match.
+   * @throws Exception
+   */
+  @Test
+  public void testNoKrbConnectionNeg() throws Exception {
+    try {
+      String url = miniHS2.getBaseJdbcURL() + "default;user=wronguser;pwd=mypwd";
+      hs2Conn = DriverManager.getConnection(url);
+      fail("noSasl connection should fail");
+    } catch (SQLException e) {
+      // expected error
+      assertEquals("08S01", e.getSQLState().trim());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index eca2317..6141a1a 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -85,6 +85,7 @@ public class MiniHS2 extends AbstractHiveService {
     private boolean isHTTPTransMode = false;
     private boolean isMetastoreRemote;
     private boolean usePortsFromConf = false;
+    private String authType = "KERBEROS";
 
     public Builder() {
     }
@@ -101,6 +102,11 @@ public class MiniHS2 extends AbstractHiveService {
       return this;
     }
 
+    public Builder withAuthenticationType(String authType) {
+      this.authType = authType;
+      return this;
+    }
+
     public Builder withRemoteMetastore() {
       this.isMetastoreRemote = true;
       return this;
@@ -131,7 +137,7 @@ public class MiniHS2 extends AbstractHiveService {
         hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_BINARY_MODE);
       }
       return new MiniHS2(hiveConf, miniClusterType, useMiniKdc, serverPrincipal, serverKeytab,
-          isMetastoreRemote, usePortsFromConf);
+          isMetastoreRemote, usePortsFromConf, authType);
     }
   }
 
@@ -169,7 +175,7 @@ public class MiniHS2 extends AbstractHiveService {
 
   private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean useMiniKdc,
       String serverPrincipal, String serverKeytab, boolean isMetastoreRemote,
-      boolean usePortsFromConf) throws Exception {
+      boolean usePortsFromConf, String authType) throws Exception {
     super(hiveConf, "localhost",
         (usePortsFromConf ? hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT) : MetaStoreUtils.findFreePort()),
         (usePortsFromConf ? hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT) : MetaStoreUtils.findFreePort()));
@@ -217,7 +223,7 @@ public class MiniHS2 extends AbstractHiveService {
     if (useMiniKdc) {
       hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL, serverPrincipal);
       hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB, serverKeytab);
-      hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, "KERBEROS");
+      hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, authType);
     }
     String metaStoreURL =  "jdbc:derby:" + baseDir.getAbsolutePath() + File.separator + "test_metastore-" +
         hs2Counter.incrementAndGet() + ";create=true";
@@ -260,7 +266,7 @@ public class MiniHS2 extends AbstractHiveService {
 
   public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType,
       boolean usePortsFromConf) throws Exception {
-    this(hiveConf, clusterType, false, null, null, false, usePortsFromConf);
+    this(hiveConf, clusterType, false, null, null, false, usePortsFromConf, "KERBEROS");
   }
 
   public void start(Map<String, String> confOverlay) throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java b/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
index 6992f62..8af9d0a 100644
--- a/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
+++ b/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
@@ -116,8 +116,7 @@ public class HiveAuthFactory {
         authTypeStr = AuthTypes.NONE.getAuthName();
       }
     }
-    if (hadoopAuth.equalsIgnoreCase("kerberos")
-        && !authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) {
+    if (isSASLWithKerberizedHadoop()) {
       saslServer =
           ShimLoader.getHadoopThriftAuthBridge().createServer(
               conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB),
@@ -162,8 +161,7 @@ public class HiveAuthFactory {
     TTransportFactory transportFactory;
     TSaslServerTransport.Factory serverTransportFactory;
 
-    if (hadoopAuth.equalsIgnoreCase("kerberos") && !authTypeStr.equalsIgnoreCase(
-          AuthTypes.NOSASL.getAuthName())) {
+    if (isSASLWithKerberizedHadoop()) {
       try {
         serverTransportFactory = saslServer.createSaslServerTransportFactory(
             getSaslProperties());
@@ -207,7 +205,7 @@ public class HiveAuthFactory {
    * @throws LoginException
    */
   public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException {
-    if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
+    if (isSASLWithKerberizedHadoop()) {
       return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service);
     } else {
       return PlainSaslHelper.getPlainProcessorFactory(service);
@@ -226,6 +224,11 @@ public class HiveAuthFactory {
     }
   }
 
+  public boolean isSASLWithKerberizedHadoop() {
+    return "kerberos".equalsIgnoreCase(hadoopAuth)
+        && !authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName());
+  }
+
   // Perform kerberos login using the hadoop shim API if the configuration is available
   public static void loginFromKeytab(HiveConf hiveConf) throws IOException {
     String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL);

http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
index 62fcde5..0a2a761 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
@@ -387,11 +387,10 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       clientIpAddress = SessionManager.getIpAddress();
     }
     else {
-      // Kerberos
-      if (isKerberosAuthMode()) {
+      if (hiveAuthFactory != null && hiveAuthFactory.isSASLWithKerberizedHadoop()) {
         clientIpAddress = hiveAuthFactory.getIpAddress();
       }
-      // Except kerberos, NOSASL
+      // NOSASL
       else {
         clientIpAddress = TSetIpAddressProcessor.getUserIpAddress();
       }
@@ -412,11 +411,10 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
    */
   private String getUserName(TOpenSessionReq req) throws HiveSQLException, IOException {
     String userName = null;
-    // Kerberos
-    if (isKerberosAuthMode()) {
+    if (hiveAuthFactory != null && hiveAuthFactory.isSASLWithKerberizedHadoop()) {
       userName = hiveAuthFactory.getRemoteUser();
     }
-    // Except kerberos, NOSASL
+    // NOSASL
     if (userName == null) {
       userName = TSetIpAddressProcessor.getUserName();
     }
@@ -803,9 +801,4 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
     LOG.debug("Verified proxy user: " + proxyUser);
     return proxyUser;
   }
-
-  private boolean isKerberosAuthMode() {
-    return cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION)
-        .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString());
-  }
 }


[03/24] hive git commit: HIVE-13255: FloatTreeReader.nextVector is expensive (Prasanth Jayachandran reviewed by Gopal V)

Posted by ss...@apache.org.
HIVE-13255: FloatTreeReader.nextVector is expensive (Prasanth Jayachandran reviewed by Gopal V)


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

Branch: refs/heads/llap
Commit: 8225cb6aedba7e49515da44f092405994f9a22b6
Parents: 4008845
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Thu Mar 31 02:48:01 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Thu Mar 31 02:48:01 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/orc/impl/IntegerReader.java |  4 +-
 .../apache/orc/impl/RunLengthIntegerReader.java |  7 +--
 .../orc/impl/RunLengthIntegerReaderV2.java      |  7 +--
 .../org/apache/orc/impl/SerializationUtils.java | 34 ++++++++++-----
 .../apache/orc/impl/TestSerializationUtils.java | 45 +++++++++++++++++--
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java | 16 +++----
 .../hive/ql/io/orc/TreeReaderFactory.java       | 46 ++++++++++----------
 7 files changed, 99 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8225cb6a/orc/src/java/org/apache/orc/impl/IntegerReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/IntegerReader.java b/orc/src/java/org/apache/orc/impl/IntegerReader.java
index b928559..7dfd289 100644
--- a/orc/src/java/org/apache/orc/impl/IntegerReader.java
+++ b/orc/src/java/org/apache/orc/impl/IntegerReader.java
@@ -60,8 +60,6 @@ public interface IntegerReader {
    * @return
    * @throws IOException
    */
-   void nextVector(LongColumnVector previous, long previousLen)
+   void nextVector(LongColumnVector previous, final int previousLen)
       throws IOException;
-
-  void setInStream(InStream data);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8225cb6a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java b/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java
index f129c86..0c90cde 100644
--- a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java
+++ b/orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java
@@ -99,7 +99,7 @@ public class RunLengthIntegerReader implements IntegerReader {
   }
 
   @Override
-  public void nextVector(LongColumnVector previous, long previousLen) throws IOException {
+  public void nextVector(LongColumnVector previous, final int previousLen) throws IOException {
     previous.isRepeating = true;
     for (int i = 0; i < previousLen; i++) {
       if (!previous.isNull[i]) {
@@ -122,11 +122,6 @@ public class RunLengthIntegerReader implements IntegerReader {
   }
 
   @Override
-  public void setInStream(InStream data) {
-    input = data;
-  }
-
-  @Override
   public void seek(PositionProvider index) throws IOException {
     input.seek(index);
     int consumed = (int) index.getNext();

http://git-wip-us.apache.org/repos/asf/hive/blob/8225cb6a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java b/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java
index 5f2a673..c6d685a 100644
--- a/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java
+++ b/orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java
@@ -360,7 +360,7 @@ public class RunLengthIntegerReaderV2 implements IntegerReader {
   }
 
   @Override
-  public void nextVector(LongColumnVector previous, long previousLen) throws IOException {
+  public void nextVector(LongColumnVector previous, final int previousLen) throws IOException {
     previous.isRepeating = true;
     for (int i = 0; i < previousLen; i++) {
       if (!previous.isNull[i]) {
@@ -382,9 +382,4 @@ public class RunLengthIntegerReaderV2 implements IntegerReader {
       }
     }
   }
-
-  @Override
-  public void setInStream(InStream data) {
-    input = data;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8225cb6a/orc/src/java/org/apache/orc/impl/SerializationUtils.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/SerializationUtils.java b/orc/src/java/org/apache/orc/impl/SerializationUtils.java
index c1162e4..2e5a59b 100644
--- a/orc/src/java/org/apache/orc/impl/SerializationUtils.java
+++ b/orc/src/java/org/apache/orc/impl/SerializationUtils.java
@@ -18,8 +18,6 @@
 
 package org.apache.orc.impl;
 
-import org.apache.orc.impl.InStream;
-
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -77,18 +75,22 @@ public final class SerializationUtils {
   }
 
   public float readFloat(InputStream in) throws IOException {
-    int ser = in.read() | (in.read() << 8) | (in.read() << 16) |
-      (in.read() << 24);
-    return Float.intBitsToFloat(ser);
+    readFully(in, readBuffer, 0, 4);
+    int val = (((readBuffer[0] & 0xff) << 0)
+        + ((readBuffer[1] & 0xff) << 8)
+        + ((readBuffer[2] & 0xff) << 16)
+        + ((readBuffer[3] & 0xff) << 24));
+    return Float.intBitsToFloat(val);
   }
 
   public void writeFloat(OutputStream output,
                          float value) throws IOException {
     int ser = Float.floatToIntBits(value);
-    output.write(ser & 0xff);
-    output.write((ser >> 8) & 0xff);
-    output.write((ser >> 16) & 0xff);
-    output.write((ser >> 24) & 0xff);
+    writeBuffer[0] = (byte) ((ser >> 0)  & 0xff);
+    writeBuffer[1] = (byte) ((ser >> 8)  & 0xff);
+    writeBuffer[2] = (byte) ((ser >> 16) & 0xff);
+    writeBuffer[3] = (byte) ((ser >> 24) & 0xff);
+    output.write(writeBuffer, 0, 4);
   }
 
   public double readDouble(InputStream in) throws IOException {
@@ -96,7 +98,7 @@ public final class SerializationUtils {
   }
 
   public long readLongLE(InputStream in) throws IOException {
-    in.read(readBuffer, 0, 8);
+    readFully(in, readBuffer, 0, 8);
     return (((readBuffer[0] & 0xff) << 0)
         + ((readBuffer[1] & 0xff) << 8)
         + ((readBuffer[2] & 0xff) << 16)
@@ -107,6 +109,18 @@ public final class SerializationUtils {
         + ((long) (readBuffer[7] & 0xff) << 56));
   }
 
+  private void readFully(final InputStream in, final byte[] buffer, final int off, final int len)
+      throws IOException {
+    int n = 0;
+    while (n < len) {
+      int count = in.read(buffer, off + n, len - n);
+      if (count < 0) {
+        throw new EOFException("Read past EOF for " + in);
+      }
+      n += count;
+    }
+  }
+
   public void writeDouble(OutputStream output,
                           double value) throws IOException {
     writeLongLE(output, Double.doubleToLongBits(value));

http://git-wip-us.apache.org/repos/asf/hive/blob/8225cb6a/orc/src/test/org/apache/orc/impl/TestSerializationUtils.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestSerializationUtils.java b/orc/src/test/org/apache/orc/impl/TestSerializationUtils.java
index 0785412..4a8a0f2 100644
--- a/orc/src/test/org/apache/orc/impl/TestSerializationUtils.java
+++ b/orc/src/test/org/apache/orc/impl/TestSerializationUtils.java
@@ -25,6 +25,9 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.InputStream;
 import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
 
 import org.junit.Test;
 
@@ -156,9 +159,43 @@ public class TestSerializationUtils {
     assertEquals(Long.MIN_VALUE, LongMath.checkedSubtract(Long.MIN_VALUE, 0));
   }
 
-  public static void main(String[] args) throws Exception {
-    TestSerializationUtils test = new TestSerializationUtils();
-    test.testDoubles();
-    test.testBigIntegers();
+  @Test
+  public void testRandomFloats() throws Exception {
+    float tolerance = 0.0000000000000001f;
+    ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+    SerializationUtils utils = new SerializationUtils();
+    Random rand = new Random();
+    int n = 100_000;
+    float[] expected = new float[n];
+    for (int i = 0; i < n; i++) {
+      float f = rand.nextFloat();
+      expected[i] = f;
+      utils.writeFloat(buffer, f);
+    }
+    InputStream newBuffer = fromBuffer(buffer);
+    for (int i = 0; i < n; i++) {
+      float got = utils.readFloat(newBuffer);
+      assertEquals(expected[i], got, tolerance);
+    }
+  }
+
+  @Test
+  public void testRandomDoubles() throws Exception {
+    double tolerance = 0.0000000000000001;
+    ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+    SerializationUtils utils = new SerializationUtils();
+    Random rand = new Random();
+    int n = 100_000;
+    double[] expected = new double[n];
+    for (int i = 0; i < n; i++) {
+      double d = rand.nextDouble();
+      expected[i] = d;
+      utils.writeDouble(buffer, d);
+    }
+    InputStream newBuffer = fromBuffer(buffer);
+    for (int i = 0; i < n; i++) {
+      double got = utils.readDouble(newBuffer);
+      assertEquals(expected[i], got, tolerance);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8225cb6a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index aa835ae..3975409 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -1060,7 +1060,7 @@ public class RecordReaderImpl implements RecordReader {
         readStripe();
       }
 
-      long batchSize = computeBatchSize(VectorizedRowBatch.DEFAULT_SIZE);
+      final int batchSize = computeBatchSize(VectorizedRowBatch.DEFAULT_SIZE);
 
       rowInStripe += batchSize;
       if (previous == null) {
@@ -1068,13 +1068,13 @@ public class RecordReaderImpl implements RecordReader {
         result = new VectorizedRowBatch(cols.length);
         result.cols = cols;
       } else {
-        result = (VectorizedRowBatch) previous;
+        result = previous;
         result.selectedInUse = false;
         reader.setVectorColumnCount(result.getDataColumnCount());
-        reader.nextVector(result.cols, (int) batchSize);
+        reader.nextVector(result.cols, batchSize);
       }
 
-      result.size = (int) batchSize;
+      result.size = batchSize;
       advanceToNextRow(reader, rowInStripe + rowBaseInStripe, true);
       return result;
     } catch (IOException e) {
@@ -1083,8 +1083,8 @@ public class RecordReaderImpl implements RecordReader {
     }
   }
 
-  private long computeBatchSize(long targetBatchSize) {
-    long batchSize = 0;
+  private int computeBatchSize(long targetBatchSize) {
+    final int batchSize;
     // In case of PPD, batch size should be aware of row group boundaries. If only a subset of row
     // groups are selected then marker position is set to the end of range (subset of row groups
     // within strip). Batch size computed out of marker position makes sure that batch size is
@@ -1106,13 +1106,13 @@ public class RecordReaderImpl implements RecordReader {
       final long markerPosition =
           (endRowGroup * rowIndexStride) < rowCountInStripe ? (endRowGroup * rowIndexStride)
               : rowCountInStripe;
-      batchSize = Math.min(targetBatchSize, (markerPosition - rowInStripe));
+      batchSize = (int) Math.min(targetBatchSize, (markerPosition - rowInStripe));
 
       if (isLogDebugEnabled && batchSize < targetBatchSize) {
         LOG.debug("markerPosition: " + markerPosition + " batchSize: " + batchSize);
       }
     } else {
-      batchSize = Math.min(targetBatchSize, (rowCountInStripe - rowInStripe));
+      batchSize = (int) Math.min(targetBatchSize, (rowCountInStripe - rowInStripe));
     }
     return batchSize;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/8225cb6a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
index 620ad53..d74a854 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
@@ -239,7 +239,7 @@ public class TreeReaderFactory {
      * @return next column vector
      * @throws IOException
      */
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       ColumnVector result = (ColumnVector) previousVector;
       if (present != null) {
         // Set noNulls and isNull vector of the ColumnVector based on
@@ -322,7 +322,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final LongColumnVector result;
       if (previousVector == null) {
         result = new LongColumnVector();
@@ -387,7 +387,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final LongColumnVector result;
       if (previousVector == null) {
         result = new LongColumnVector();
@@ -473,7 +473,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final LongColumnVector result;
       if (previousVector == null) {
         result = new LongColumnVector();
@@ -559,7 +559,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final LongColumnVector result;
       if (previousVector == null) {
         result = new LongColumnVector();
@@ -646,7 +646,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final LongColumnVector result;
       if (previousVector == null) {
         result = new LongColumnVector();
@@ -719,7 +719,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, final long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final DoubleColumnVector result;
       if (previousVector == null) {
         result = new DoubleColumnVector();
@@ -832,7 +832,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, final long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final DoubleColumnVector result;
       if (previousVector == null) {
         result = new DoubleColumnVector();
@@ -974,7 +974,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final BytesColumnVector result;
       if (previousVector == null) {
         result = new BytesColumnVector();
@@ -1144,7 +1144,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final TimestampColumnVector result;
       if (previousVector == null) {
         result = new TimestampColumnVector();
@@ -1253,7 +1253,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final LongColumnVector result;
       if (previousVector == null) {
         result = new LongColumnVector();
@@ -1352,7 +1352,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final DecimalColumnVector result;
       if (previousVector == null) {
         result = new DecimalColumnVector(precision, scale);
@@ -1481,7 +1481,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       return reader.nextVector(previousVector, batchSize);
     }
 
@@ -1498,7 +1498,7 @@ public class TreeReaderFactory {
 
     private static byte[] commonReadByteArrays(InStream stream, IntegerReader lengths,
         LongColumnVector scratchlcv,
-        BytesColumnVector result, long batchSize) throws IOException {
+        BytesColumnVector result, final int batchSize) throws IOException {
       // Read lengths
       scratchlcv.isNull = result.isNull;  // Notice we are replacing the isNull vector here...
       lengths.nextVector(scratchlcv, batchSize);
@@ -1534,7 +1534,7 @@ public class TreeReaderFactory {
     // This method has the common code for reading in bytes into a BytesColumnVector.
     public static void readOrcByteArrays(InStream stream, IntegerReader lengths,
         LongColumnVector scratchlcv,
-        BytesColumnVector result, long batchSize) throws IOException {
+        BytesColumnVector result, final int batchSize) throws IOException {
 
       byte[] allBytes = commonReadByteArrays(stream, lengths, scratchlcv, result, batchSize);
 
@@ -1641,7 +1641,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final BytesColumnVector result;
       if (previousVector == null) {
         result = new BytesColumnVector();
@@ -1815,7 +1815,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final BytesColumnVector result;
       int offset;
       int length;
@@ -1926,7 +1926,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       // Get the vector of strings from StringTreeReader, then make a 2nd pass to
       // adjust down the length (right trim and truncate) if necessary.
       BytesColumnVector result = (BytesColumnVector) super.nextVector(previousVector, batchSize);
@@ -2000,7 +2000,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       // Get the vector of strings from StringTreeReader, then make a 2nd pass to
       // adjust down the length (truncate) if necessary.
       BytesColumnVector result = (BytesColumnVector) super.nextVector(previousVector, batchSize);
@@ -2137,7 +2137,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       final ColumnVector[] result;
       if (previousVector == null) {
         result = new ColumnVector[readColumnCount];
@@ -2242,7 +2242,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previousVector, long batchSize) throws IOException {
+    public Object nextVector(Object previousVector, final int batchSize) throws IOException {
       throw new UnsupportedOperationException(
           "NextVector is not supported operation for Union type");
     }
@@ -2325,7 +2325,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previous, long batchSize) throws IOException {
+    public Object nextVector(Object previous, final int batchSize) throws IOException {
       throw new UnsupportedOperationException(
           "NextVector is not supported operation for List type");
     }
@@ -2419,7 +2419,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    public Object nextVector(Object previous, long batchSize) throws IOException {
+    public Object nextVector(Object previous, final int batchSize) throws IOException {
       throw new UnsupportedOperationException(
           "NextVector is not supported operation for Map type");
     }


[08/24] hive git commit: HIVE-12650: Improve error messages for Hive on Spark in case the cluster has no resources available (Rui reviewed by Xuefu)

Posted by ss...@apache.org.
HIVE-12650: Improve error messages for Hive on Spark in case the cluster has no resources available (Rui reviewed by Xuefu)


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

Branch: refs/heads/llap
Commit: 03b81bc9c40b6de4f238f6b7660488e711b869c4
Parents: e085b7e
Author: Rui Li <ru...@intel.com>
Authored: Fri Apr 1 14:36:18 2016 +0800
Committer: Rui Li <ru...@intel.com>
Committed: Fri Apr 1 14:38:14 2016 +0800

----------------------------------------------------------------------
 .../ql/exec/spark/RemoteHiveSparkClient.java    | 20 +++++++++++++++-----
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |  3 +++
 .../exec/spark/status/LocalSparkJobMonitor.java |  2 +-
 .../spark/status/RemoteSparkJobMonitor.java     |  5 ++++-
 .../hive/spark/client/SparkClientImpl.java      |  9 ++++++++-
 5 files changed, 31 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/03b81bc9/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java
index 30e53d2..3a1577f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
@@ -110,7 +111,12 @@ public class RemoteHiveSparkClient implements HiveSparkClient {
       int curExecutors = 0;
       long ts = System.currentTimeMillis();
       do {
-        curExecutors = getExecutorCount();
+        try {
+          curExecutors = getExecutorCount(MAX_PREWARM_TIME, TimeUnit.MILLISECONDS);
+        } catch (TimeoutException e) {
+          // let's don't fail on future timeout since we have a timeout for pre-warm
+          LOG.warn("Timed out getting executor count.", e);
+        }
         if (curExecutors >= minExecutors) {
           LOG.info("Finished prewarming Spark executors. The current number of executors is " + curExecutors);
           return;
@@ -118,8 +124,8 @@ public class RemoteHiveSparkClient implements HiveSparkClient {
         Thread.sleep(500); // sleep half a second
       } while (System.currentTimeMillis() - ts < MAX_PREWARM_TIME);
 
-      LOG.info("Timeout (" + MAX_PREWARM_TIME + 
-          "s) occurred while prewarming executors. The current number of executors is " + curExecutors);
+      LOG.info("Timeout (" + MAX_PREWARM_TIME / 1000 + "s) occurred while prewarming executors. " +
+          "The current number of executors is " + curExecutors);
     }
   }
 
@@ -143,6 +149,11 @@ public class RemoteHiveSparkClient implements HiveSparkClient {
     return minExecutors;
   }
 
+  private int getExecutorCount(long timeout, TimeUnit unit) throws Exception {
+    Future<Integer> handler = remoteClient.getExecutorCount();
+    return handler.get(timeout, unit);
+  }
+
   @Override
   public SparkConf getSparkConf() {
     return sparkConf;
@@ -150,8 +161,7 @@ public class RemoteHiveSparkClient implements HiveSparkClient {
 
   @Override
   public int getExecutorCount() throws Exception {
-    Future<Integer> handler = remoteClient.getExecutorCount();
-    return handler.get(sparkClientTimtout, TimeUnit.SECONDS).intValue();
+    return getExecutorCount(sparkClientTimtout, TimeUnit.SECONDS);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/03b81bc9/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
index 26cce1b..7f87adf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
@@ -105,6 +105,9 @@ public class SparkTask extends Task<SparkWork> {
         }
         LOG.info("Execution completed successfully");
       } else if (rc == 2) { // Cancel job if the monitor found job submission timeout.
+        // TODO: If the timeout is because of lack of resources in the cluster, we should
+        // ideally also cancel the app request here. But w/o facilities from Spark or YARN,
+        // it's difficult to do it on hive side alone. See HIVE-12650.
         jobRef.cancelJob();
       }
       sparkJobStatus.cleanup();

http://git-wip-us.apache.org/repos/asf/hive/blob/03b81bc9/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java
index d109c6f..5f0352a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java
@@ -59,7 +59,7 @@ public class LocalSparkJobMonitor extends SparkJobMonitor {
         if (state == null) {
           long timeCount = (System.currentTimeMillis() - startTime)/1000;
           if (timeCount > monitorTimeoutInteval) {
-            LOG.info("Job hasn't been submitted after " + timeCount + "s. Aborting it.");
+            console.printError("Job hasn't been submitted after " + timeCount + "s. Aborting it.");
             console.printError("Status: " + state);
             running = false;
             done = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/03b81bc9/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
index 6990e80..11f263b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
@@ -66,7 +66,10 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor {
         case QUEUED:
           long timeCount = (System.currentTimeMillis() - startTime) / 1000;
           if ((timeCount > monitorTimeoutInteval)) {
-            LOG.info("Job hasn't been submitted after " + timeCount + "s. Aborting it.");
+            console.printError("Job hasn't been submitted after " + timeCount + "s." +
+                " Aborting it.\nPossible reasons include network issues, " +
+                "errors in remote driver or the cluster has no available resources, etc.\n" +
+                "Please check YARN or Spark driver's logs for further information.");
             console.printError("Status: " + state);
             running = false;
             done = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/03b81bc9/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
index 3d682a0..ae78bc3 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
@@ -47,6 +47,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -100,7 +101,13 @@ class SparkClientImpl implements SparkClient {
       // The RPC server will take care of timeouts here.
       this.driverRpc = rpcServer.registerClient(clientId, secret, protocol).get();
     } catch (Throwable e) {
-      LOG.warn("Error while waiting for client to connect.", e);
+      if (e.getCause() instanceof TimeoutException) {
+        LOG.error("Timed out waiting for client to connect.\nPossible reasons include network " +
+            "issues, errors in remote driver or the cluster has no available resources, etc." +
+            "\nPlease check YARN or Spark driver's logs for further information.", e);
+      } else {
+        LOG.error("Error while waiting for client to connect.", e);
+      }
       driverThread.interrupt();
       try {
         driverThread.join();


[12/24] hive git commit: HIVE-10280. LLAP: Handle errors while sending source state updates to the daemons. (Siddharth Seth, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
HIVE-10280. LLAP: Handle errors while sending source state updates to the daemons. (Siddharth Seth, 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/d94e8d08
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d94e8d08
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d94e8d08

Branch: refs/heads/llap
Commit: d94e8d08dd1d92c9eee99f60273e895a4a633b23
Parents: 21f18ad
Author: Siddharth Seth <ss...@apache.org>
Authored: Sat Apr 2 15:06:34 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Sat Apr 2 15:06:34 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/tez/LlapProtocolClientProxy.java  |   4 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |  37 ++-
 .../tezplugins/LlapTaskSchedulerService.java    |   7 +
 .../tezplugins/helpers/SourceStateTracker.java  |   2 +-
 .../tezplugins/TestLlapTaskCommunicator.java    | 304 ++++++++++++++++++-
 5 files changed, 340 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d94e8d08/llap-client/src/java/org/apache/hadoop/hive/llap/tez/LlapProtocolClientProxy.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/tez/LlapProtocolClientProxy.java b/llap-client/src/java/org/apache/hadoop/hive/llap/tez/LlapProtocolClientProxy.java
index e8d4148..f48a1cb 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/tez/LlapProtocolClientProxy.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/tez/LlapProtocolClientProxy.java
@@ -139,10 +139,8 @@ public class LlapProtocolClientProxy extends AbstractService {
     requestManager.queueRequest(new SubmitWorkCallable(nodeId, request, callback));
   }
 
-  public void sendSourceStateUpdate(final SourceStateUpdatedRequestProto request, final String host,
-                                    final int port,
+  public void sendSourceStateUpdate(final SourceStateUpdatedRequestProto request, final LlapNodeId nodeId,
                                     final ExecuteRequestCallback<SourceStateUpdatedResponseProto> callback) {
-    LlapNodeId nodeId = LlapNodeId.getInstance(host, port);
     requestManager.queueRequest(
         new SendSourceStateUpdateCallable(nodeId, request, callback));
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/d94e8d08/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
index 456121b..799367b 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
@@ -150,7 +150,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
     super.initialize();
     Configuration conf = getConf();
     int numThreads = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_COMMUNICATOR_NUM_THREADS);
-    this.communicator = new LlapProtocolClientProxy(numThreads, conf, token);
+    this.communicator = createLlapProtocolClientProxy(numThreads, conf);
     this.deleteDelayOnDagComplete = HiveConf.getTimeVar(
         conf, ConfVars.LLAP_FILE_CLEANUP_DELAY_SECONDS, TimeUnit.SECONDS);
     LOG.info("Running LlapTaskCommunicator with "
@@ -205,6 +205,10 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
     }
   }
 
+  protected LlapProtocolClientProxy createLlapProtocolClientProxy(int numThreads, Configuration conf) {
+    return new LlapProtocolClientProxy(numThreads, conf, token);
+  }
+
   @Override
   public void registerRunningContainer(ContainerId containerId, String hostname, int port) {
     super.registerRunningContainer(containerId, hostname, port);
@@ -413,9 +417,9 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
         .sourceStateUpdated(vertexStateUpdate.getVertexName(), vertexStateUpdate.getVertexState());
   }
 
-  public void sendStateUpdate(final String host, final int port,
+  public void sendStateUpdate(final LlapNodeId nodeId,
                               final SourceStateUpdatedRequestProto request) {
-    communicator.sendSourceStateUpdate(request, host, port,
+    communicator.sendSourceStateUpdate(request, nodeId,
         new LlapProtocolClientProxy.ExecuteRequestCallback<SourceStateUpdatedResponseProto>() {
           @Override
           public void setResponse(SourceStateUpdatedResponseProto response) {
@@ -423,12 +427,29 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
 
           @Override
           public void indicateError(Throwable t) {
-            // TODO HIVE-10280.
-            // Ideally, this should be retried for a while, after which the node should be marked as failed.
-            // Considering tasks are supposed to run fast. Failing the task immediately may be a good option.
+
+            // Re-attempts are left upto the RPC layer. If there's a failure reported after this,
+            // mark all attempts running on this node as KILLED. The node itself cannot be killed from
+            // here, that's only possible via the scheduler.
+            // The assumption is that if there's a failure to communicate with the node - it will
+            // eventually timeout - and no more tasks will be allocated on it.
+
             LOG.error(
-                "Failed to send state update to node: " + host + ":" + port + ", StateUpdate=" +
-                    request, t);
+                "Failed to send state update to node: {}, Killing all attempts running on node. Attempted StateUpdate={}",
+                nodeId, request, t);
+            BiMap<ContainerId, TezTaskAttemptID> biMap =
+                entityTracker.getContainerAttemptMapForNode(nodeId);
+            if (biMap != null) {
+              synchronized (biMap) {
+                for (Map.Entry<ContainerId, TezTaskAttemptID> entry : biMap.entrySet()) {
+                  LOG.info(
+                      "Sending a kill for attempt {}, due to a communication failure while sending a finishable state update",
+                      entry.getValue());
+                  getContext().taskKilled(entry.getValue(), TaskAttemptEndReason.NODE_FAILED,
+                      "Failed to send finishable state update to node " + nodeId);
+                }
+              }
+            }
           }
         });
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/d94e8d08/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 0cb770b..b57ae1a 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
@@ -516,6 +516,13 @@ public class LlapTaskSchedulerService extends TaskScheduler {
               dagStats.registerTaskRejected(taskInfo.assignedInstance.getHost());
             }
           }
+          if (endReason != null && endReason == TaskAttemptEndReason.NODE_FAILED) {
+            LOG.info(
+                "Task {} ended on {} nodeInfo.toString() with a NODE_FAILED message." +
+                    " An message should come in from the registry to disable this node unless" +
+                    " this was a temporary communication failure",
+                task, assignedInstance);
+          }
           boolean commFailure =
               endReason != null && endReason == TaskAttemptEndReason.COMMUNICATION_ERROR;
           disableInstance(assignedInstance, commFailure);

http://git-wip-us.apache.org/repos/asf/hive/blob/d94e8d08/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/SourceStateTracker.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/SourceStateTracker.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/SourceStateTracker.java
index d8f7574..3dd73f6 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/SourceStateTracker.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/SourceStateTracker.java
@@ -280,7 +280,7 @@ public class SourceStateTracker {
 
 
   void sendStateUpdateToNode(LlapNodeId nodeId, String sourceName, VertexState state) {
-    taskCommunicator.sendStateUpdate(nodeId.getHostname(), nodeId.getPort(),
+    taskCommunicator.sendStateUpdate(nodeId,
         SourceStateUpdatedRequestProto.newBuilder().setQueryIdentifier(currentQueryIdentifier)
             .setSrcName(sourceName).setState(Converters.fromVertexState(state)).build());
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/d94e8d08/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java b/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java
index 8f3d104..1ee6a50 100644
--- a/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java
+++ b/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java
@@ -17,13 +17,49 @@ package org.apache.hadoop.hive.llap.tezplugins;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.llap.LlapNodeId;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
+import org.apache.hadoop.hive.llap.tez.LlapProtocolClientProxy;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskCommunicatorContext;
 import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestLlapTaskCommunicator {
 
@@ -32,8 +68,6 @@ public class TestLlapTaskCommunicator {
     LlapTaskCommunicator.EntityTracker entityTracker = new LlapTaskCommunicator.EntityTracker();
 
     String host1 = "host1";
-    String host2 = "host2";
-    String host3 = "host3";
     int port = 1451;
 
 
@@ -84,6 +118,240 @@ public class TestLlapTaskCommunicator {
   }
 
 
+  @Test(timeout = 5000)
+  public void testFinishableStateUpdateFailure() throws Exception {
+
+    LlapTaskCommunicatorWrapperForTest wrapper = null;
+
+    Lock lock = new ReentrantLock();
+    Condition condition = lock.newCondition();
+    final AtomicBoolean opDone = new AtomicBoolean(false);
+
+    LlapProtocolClientProxy proxy = mock(LlapProtocolClientProxy.class,
+        new FinishableStatusUpdateTestAnswer(lock, condition, opDone));
+
+    try {
+      wrapper = new LlapTaskCommunicatorWrapperForTest(proxy);
+
+      // Register tasks on 2 nodes, with a dependency on vertex1 completing.
+      ContainerId cId11 = wrapper.registerContainer(1, 0);
+      TaskSpec ts11 = wrapper.registerRunningTaskAttemptWithSourceVertex(cId11, 1);
+
+      ContainerId cId12 = wrapper.registerContainer(2, 0);
+      TaskSpec ts12 = wrapper.registerRunningTaskAttemptWithSourceVertex(cId12, 2);
+
+      ContainerId cId21 = wrapper.registerContainer(3, 1);
+      TaskSpec ts21 = wrapper.registerRunningTaskAttemptWithSourceVertex(cId21, 3);
+
+      // Send a state update for vertex1 completion. This triggers a status update to be sent out.
+      VertexStateUpdate vertexStateUpdate =
+          new VertexStateUpdate(LlapTaskCommunicatorWrapperForTest.VERTEX_NAME1,
+              VertexState.SUCCEEDED);
+      wrapper.getTaskCommunicator().onVertexStateUpdated(vertexStateUpdate);
+
+      // Wait for all invocations to complete.
+      lock.lock();
+      try {
+        while (!opDone.get()) {
+          condition.await();
+        }
+      } finally {
+        lock.unlock();
+      }
+      // Verify that a task kill went out for all nodes running on the specified host.
+
+      verify(wrapper.getTaskCommunicatorContext(), times(2))
+          .taskKilled(any(TezTaskAttemptID.class), any(TaskAttemptEndReason.class),
+              any(String.class));
+
+      verify(wrapper.getTaskCommunicatorContext()).taskKilled(eq(ts11.getTaskAttemptID()),
+          eq(TaskAttemptEndReason.NODE_FAILED), any(String.class));
+      verify(wrapper.getTaskCommunicatorContext()).taskKilled(eq(ts12.getTaskAttemptID()),
+          eq(TaskAttemptEndReason.NODE_FAILED), any(String.class));
+
+      wrapper.getTaskCommunicator().sendStateUpdate(LlapNodeId
+              .getInstance(LlapTaskCommunicatorWrapperForTest.HOSTS[1],
+                  LlapTaskCommunicatorWrapperForTest.RPC_PORT),
+          LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto.getDefaultInstance());
+
+      // Verify no more invocations in case of success.
+      verify(wrapper.getTaskCommunicatorContext(), times(2))
+          .taskKilled(any(TezTaskAttemptID.class), any(TaskAttemptEndReason.class),
+              any(String.class));
+
+    } finally {
+      if (wrapper != null) {
+        wrapper.shutdown();
+      }
+    }
+  }
+
+  static class FinishableStatusUpdateTestAnswer implements Answer<Void> {
+
+    final Lock lock;
+    final Condition condition;
+    final AtomicBoolean opDone;
+
+    final AtomicBoolean successInvoked = new AtomicBoolean(false);
+    final AtomicBoolean failInvoked = new AtomicBoolean(false);
+
+
+    FinishableStatusUpdateTestAnswer(Lock lock, Condition condition, AtomicBoolean opDone) {
+      this.lock = lock;
+      this.condition = condition;
+      this.opDone = opDone;
+    }
+
+    void reset() {
+      opDone.set(false);
+      successInvoked.set(false);
+      failInvoked.set(false);
+    }
+
+    @Override
+    public Void answer(InvocationOnMock invocation) throws Throwable {
+      if (invocation.getMethod().getName().equals("sendSourceStateUpdate")) {
+        LlapNodeId nodeId = (LlapNodeId) invocation.getArguments()[1];
+        final LlapProtocolClientProxy.ExecuteRequestCallback callback =
+            (LlapProtocolClientProxy.ExecuteRequestCallback) invocation.getArguments()[2];
+
+        if (nodeId.getHostname().equals(LlapTaskCommunicatorWrapperForTest.HOSTS[0])) {
+          new Thread() {
+            public void run() {
+              callback.indicateError(
+                  new IOException("Force failing " + LlapTaskCommunicatorWrapperForTest.HOSTS[0]));
+              successInvoked.set(true);
+              signalOpDoneIfBothInvoked();
+            }
+          }.start();
+        } else {
+          new Thread() {
+            public void run() {
+              // Report success for all other cases.
+              callback.setResponse(
+                  LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto.getDefaultInstance());
+              failInvoked.set(true);
+              signalOpDoneIfBothInvoked();
+            }
+          }.start();
+        }
+      }
+      return null;
+    }
+
+    private void signalOpDoneIfBothInvoked() {
+      lock.lock();
+      try {
+        if (failInvoked.get() && successInvoked.get()) {
+          opDone.set(true);
+          condition.signal();
+        }
+      } finally {
+        lock.unlock();
+      }
+    }
+  }
+
+
+  /**
+   * Wrapper class which is responsible for setting up various mocks required for different tests.
+   */
+  private static class LlapTaskCommunicatorWrapperForTest {
+
+    static final String[] HOSTS = new String[]{"host1", "host2", "host3"};
+    static final int RPC_PORT = 15002;
+    static final String DAG_NAME = "dagName";
+    static final String VERTEX_NAME1 = "vertexName1";
+    static final String VERTEX_NAME2 = "vertexName2";
+
+    final TaskCommunicatorContext taskCommunicatorContext = mock(TaskCommunicatorContext.class);
+
+    final ApplicationId appId = ApplicationId.newInstance(1000, 1);
+    final ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 100);
+    final TezDAGID dagid = TezDAGID.getInstance(appId, 200);
+    final TezVertexID vertexId1 = TezVertexID.getInstance(dagid, 300);
+    final TezVertexID vertexId2 = TezVertexID.getInstance(dagid, 301);
+    final Configuration conf = new Configuration(false);
+    final UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    final LlapTaskCommunicatorForTest taskCommunicator;
+
+    public LlapTaskCommunicatorWrapperForTest(LlapProtocolClientProxy llapProxy) throws Exception {
+      doReturn(appAttemptId).when(taskCommunicatorContext).getApplicationAttemptId();
+      doReturn(new Credentials()).when(taskCommunicatorContext).getCredentials();
+      doReturn(userPayload).when(taskCommunicatorContext).getInitialUserPayload();
+      doReturn(appId.toString()).when(taskCommunicatorContext).getCurrentAppIdentifier();
+      doReturn(new LinkedList<String>()).when(taskCommunicatorContext)
+          .getInputVertexNames(any(String.class));
+
+
+      this.taskCommunicator = new LlapTaskCommunicatorForTest(taskCommunicatorContext, llapProxy);
+      this.taskCommunicator.initialize();
+      this.taskCommunicator.start();
+    }
+
+    void shutdown() {
+      this.taskCommunicator.shutdown();
+    }
+
+    TaskCommunicatorContext getTaskCommunicatorContext() {
+      return taskCommunicatorContext;
+    }
+
+    LlapTaskCommunicatorForTest getTaskCommunicator() {
+      return taskCommunicator;
+    }
+
+    ContainerId registerContainer(int containerIdx, int hostIdx) {
+      ContainerId containerId = ContainerId.newInstance(appAttemptId, containerIdx);
+      taskCommunicator.registerRunningContainer(containerId, HOSTS[hostIdx], RPC_PORT);
+      return containerId;
+    }
+
+    /*
+    Sets up a TaskSpec which has vertex1 as it's input, and tasks belonging to vertex2
+     */
+    TaskSpec registerRunningTaskAttemptWithSourceVertex(ContainerId containerId, int taskIdx) {
+      TaskSpec taskSpec = createBaseTaskSpec(VERTEX_NAME2, vertexId2, taskIdx);
+
+      InputSpec inputSpec =
+          new InputSpec(VERTEX_NAME1, InputDescriptor.create("fakeInputClassName"), 3);
+      List<InputSpec> inputs = Lists.newArrayList(inputSpec);
+
+      doReturn(inputs).when(taskSpec).getInputs();
+
+      taskCommunicator
+          .registerRunningTaskAttempt(containerId, taskSpec, new HashMap<String, LocalResource>(),
+              new Credentials(), false, 2);
+      return taskSpec;
+    }
+
+    /*
+    Sets up a TaskSpec with no inputs, and tasks belonging to vertex1
+     */
+    TaskSpec registerRunningTaskAttempt(ContainerId containerId, int taskIdx) {
+
+      TaskSpec taskSpec = createBaseTaskSpec(VERTEX_NAME1, vertexId1, taskIdx);
+
+      taskCommunicator
+          .registerRunningTaskAttempt(containerId, taskSpec, new HashMap<String, LocalResource>(),
+              new Credentials(), false, 2);
+      return taskSpec;
+    }
+
+    private TaskSpec createBaseTaskSpec(String vertexName, TezVertexID vertexId, int taskIdx) {
+      TaskSpec taskSpec = mock(TaskSpec.class);
+      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(
+          TezTaskID.getInstance(vertexId, taskIdx), 0);
+      doReturn(taskAttemptId).when(taskSpec).getTaskAttemptID();
+      doReturn(DAG_NAME).when(taskSpec).getDAGName();
+      doReturn(vertexName).when(taskSpec).getVertexName();
+      return taskSpec;
+    }
+  }
+
+
+
   private ContainerId constructContainerId(int id) {
     ContainerId containerId = mock(ContainerId.class);
     doReturn(id).when(containerId).getId();
@@ -97,4 +365,36 @@ public class TestLlapTaskCommunicator {
     return taskAttemptId;
   }
 
+
+  private static class LlapTaskCommunicatorForTest extends LlapTaskCommunicator {
+
+    private final LlapProtocolClientProxy llapProxy;
+
+    public LlapTaskCommunicatorForTest(
+        TaskCommunicatorContext taskCommunicatorContext) {
+      this(taskCommunicatorContext, mock(LlapProtocolClientProxy.class));
+    }
+
+    public LlapTaskCommunicatorForTest(
+        TaskCommunicatorContext taskCommunicatorContext, LlapProtocolClientProxy llapProxy) {
+      super(taskCommunicatorContext);
+      this.llapProxy = llapProxy;
+    }
+
+    @Override
+    protected void startRpcServer() {
+    }
+
+    @Override
+    protected LlapProtocolClientProxy createLlapProtocolClientProxy(int numThreads,
+                                                                    Configuration conf) {
+      return llapProxy;
+    }
+
+    @Override
+    public InetSocketAddress getAddress() {
+      return InetSocketAddress.createUnresolved("localhost", 15001);
+    }
+  }
+
 }


[14/24] hive git commit: HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/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
new file mode 100644
index 0000000..978c2a3
--- /dev/null
+++ b/service/src/gen/thrift/gen-py/hive_service/ThriftHive.py
@@ -0,0 +1,1674 @@
+#
+# 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/98303635/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
new file mode 100644
index 0000000..6a0dcc9
--- /dev/null
+++ b/service/src/gen/thrift/gen-py/hive_service/__init__.py
@@ -0,0 +1 @@
+__all__ = ['ttypes', 'constants', 'ThriftHive']

http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/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
new file mode 100644
index 0000000..4a6492b
--- /dev/null
+++ b/service/src/gen/thrift/gen-py/hive_service/constants.py
@@ -0,0 +1,11 @@
+#
+# 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/98303635/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
new file mode 100644
index 0000000..e19fe2a
--- /dev/null
+++ b/service/src/gen/thrift/gen-py/hive_service/ttypes.py
@@ -0,0 +1,260 @@
+#
+# 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/98303635/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
new file mode 100644
index 0000000..ed7d2d7
--- /dev/null
+++ b/service/src/gen/thrift/gen-rb/hive_service_constants.rb
@@ -0,0 +1,9 @@
+#
+# 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/98303635/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
new file mode 100644
index 0000000..9191767
--- /dev/null
+++ b/service/src/gen/thrift/gen-rb/hive_service_types.rb
@@ -0,0 +1,68 @@
+#
+# 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/98303635/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
new file mode 100644
index 0000000..2b57cd9
--- /dev/null
+++ b/service/src/gen/thrift/gen-rb/thrift_hive.rb
@@ -0,0 +1,555 @@
+#
+# 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
+


[16/24] hive git commit: HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/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
new file mode 100644
index 0000000..23dc8fd
--- /dev/null
+++ b/service/src/gen/thrift/gen-php/ThriftHive.php
@@ -0,0 +1,1943 @@
+<?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/98303635/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
new file mode 100644
index 0000000..cbf4c29
--- /dev/null
+++ b/service/src/gen/thrift/gen-php/Types.php
@@ -0,0 +1,338 @@
+<?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;
+  }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/service/src/gen/thrift/gen-py/__init__.py
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/__init__.py b/service/src/gen/thrift/gen-py/__init__.py
new file mode 100644
index 0000000..e69de29


[17/24] hive git commit: HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/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
new file mode 100644
index 0000000..934a8a5
--- /dev/null
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/ThriftHive.java
@@ -0,0 +1,7784 @@
+/**
+ * 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;
+          }
+          swit

<TRUNCATED>

[19/24] hive git commit: HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/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
new file mode 100644
index 0000000..902bd4b
--- /dev/null
+++ b/service/src/gen/thrift/gen-cpp/ThriftHive.h
@@ -0,0 +1,1224 @@
+/**
+ * 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/98303635/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
new file mode 100644
index 0000000..3b61b76
--- /dev/null
+++ b/service/src/gen/thrift/gen-cpp/ThriftHive_server.skeleton.cpp
@@ -0,0 +1,84 @@
+// 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/98303635/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
new file mode 100644
index 0000000..e2bbe71
--- /dev/null
+++ b/service/src/gen/thrift/gen-cpp/hive_service_constants.cpp
@@ -0,0 +1,17 @@
+/**
+ * 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/98303635/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
new file mode 100644
index 0000000..e0887f4
--- /dev/null
+++ b/service/src/gen/thrift/gen-cpp/hive_service_constants.h
@@ -0,0 +1,24 @@
+/**
+ * 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/98303635/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
new file mode 100644
index 0000000..9ddf7c1
--- /dev/null
+++ b/service/src/gen/thrift/gen-cpp/hive_service_types.cpp
@@ -0,0 +1,351 @@
+/**
+ * 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/98303635/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
new file mode 100644
index 0000000..266f8ea
--- /dev/null
+++ b/service/src/gen/thrift/gen-cpp/hive_service_types.h
@@ -0,0 +1,176 @@
+/**
+ * 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/98303635/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
new file mode 100644
index 0000000..ad89867
--- /dev/null
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveClusterStatus.java
@@ -0,0 +1,901 @@
+/**
+ * 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);
+      }
+    }
+  }
+
+}
+


[15/24] hive git commit: HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/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
new file mode 100755
index 0000000..9a2322f
--- /dev/null
+++ b/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
@@ -0,0 +1,1242 @@
+#!/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 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('  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 == '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 == '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()


[09/24] hive git commit: HIVE-12612: beeline always exits with 0 status when reading query from standard input (Reuben Kuhnert, reviewed by Sergio Pena)

Posted by ss...@apache.org.
HIVE-12612: beeline always exits with 0 status when reading query from standard input (Reuben Kuhnert, reviewed by Sergio Pena)


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

Branch: refs/heads/llap
Commit: ac273b672de402027181b71fb192930645bd5cc0
Parents: 03b81bc
Author: Sergio Pena <se...@cloudera.com>
Authored: Fri Apr 1 10:38:27 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Fri Apr 1 10:38:27 2016 -0500

----------------------------------------------------------------------
 beeline/pom.xml.orig                            | 169 +++++++++++++++++++
 .../java/org/apache/hive/beeline/BeeLine.java   |  18 +-
 .../apache/hive/beeline/cli/TestHiveCli.java    |  15 +-
 3 files changed, 189 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ac273b67/beeline/pom.xml.orig
----------------------------------------------------------------------
diff --git a/beeline/pom.xml.orig b/beeline/pom.xml.orig
new file mode 100644
index 0000000..8ac83f5
--- /dev/null
+++ b/beeline/pom.xml.orig
@@ -0,0 +1,169 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive</artifactId>
+    <version>2.1.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>hive-beeline</artifactId>
+  <packaging>jar</packaging>
+  <name>Hive Beeline</name>
+
+  <properties>
+    <hive.path.to.root>..</hive.path.to.root>
+  </properties>
+
+  <dependencies>
+    <!-- dependencies are always listed in sorted order by groupId, artifectId -->
+    <!-- intra-project -->
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-metastore</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-shims</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-jdbc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <!-- inter-project -->
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+      <version>${commons-cli.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <version>${commons-lang.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <version>${commons-io.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>jline</groupId>
+      <artifactId>jline</artifactId>
+      <version>${jline.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libthrift</artifactId>
+      <version>${libthrift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>net.sf.supercsv</groupId>
+      <artifactId>super-csv</artifactId>
+      <version>${super-csv.version}</version>
+    </dependency>
+    <!-- test intra-project -->
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-service</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <!-- test inter-project -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>postgresql</groupId>
+      <artifactId>postgresql</artifactId>
+      <version>9.1-901.jdbc4</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>sources</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-source-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>attach-sources</id>
+                <goals>
+                  <goal>test-jar</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <build>
+    <sourceDirectory>${basedir}/src/java</sourceDirectory>
+    <testSourceDirectory>${basedir}/src/test</testSourceDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hive/blob/ac273b67/beeline/src/java/org/apache/hive/beeline/BeeLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
index 4ab6aa8..a4a9558 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -953,26 +953,32 @@ public class BeeLine implements Closeable {
   }
 
   private int execute(ConsoleReader reader, boolean exitOnError) {
-    String line;
+    int lastExecutionResult = ERRNO_OK;
     while (!exit) {
       try {
         // Execute one instruction; terminate on executing a script if there is an error
         // in silent mode, prevent the query and prompt being echoed back to terminal
-        line = (getOpts().isSilent() && getOpts().getScriptFile() != null) ? reader
+        String line = (getOpts().isSilent() && getOpts().getScriptFile() != null) ? reader
             .readLine(null, ConsoleReader.NULL_MASK) : reader.readLine(getPrompt());
 
         // trim line
-        line = (line == null) ? null : line.trim();
+        if (line != null) {
+          line = line.trim();
+        }
 
-        if (!dispatch(line) && exitOnError) {
-          return ERRNO_OTHER;
+        if (!dispatch(line)) {
+          lastExecutionResult = ERRNO_OTHER;
+          if (exitOnError) break;
+        } else if (line != null) {
+          lastExecutionResult = ERRNO_OK;
         }
+
       } catch (Throwable t) {
         handleException(t);
         return ERRNO_OTHER;
       }
     }
-    return ERRNO_OK;
+    return lastExecutionResult;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/ac273b67/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
index 275036f..d306e29 100644
--- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
+++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
@@ -38,6 +38,7 @@ public class TestHiveCli {
   private static final Logger LOG = LoggerFactory.getLogger(TestHiveCli.class.getName());
   private static final int ERRNO_OK = 0;
   private static final int ERRNO_ARGS = 1;
+  private static final int ERRNO_OTHER = 2;
 
   private final static String SOURCE_CONTEXT =
       "create table if not exists test.testSrcTbl(sc1 string);";
@@ -101,7 +102,7 @@ public class TestHiveCli {
 
   @Test
   public void testInValidCmd() {
-    verifyCMD("!lss\n", "Failed to execute lss", errS, null, ERRNO_OK, true);
+    verifyCMD("!lss\n", "Failed to execute lss", errS, null, ERRNO_OTHER, true);
   }
 
   @Test
@@ -159,7 +160,7 @@ public class TestHiveCli {
   public void testSourceCmd3() {
     File f = generateTmpFile(SOURCE_CONTEXT4);
     verifyCMD("source " + f.getPath() + ";" + "desc testSrcTbl4;\nquit;\n", "src", os,
-        new String[] { "--database", "test" }, ERRNO_OK, true);
+        new String[] { "--database", "test" }, ERRNO_OTHER, true);
     f.delete();
   }
 
@@ -205,34 +206,34 @@ public class TestHiveCli {
   @Test
   public void testErrOutput() {
     verifyCMD("show tables;set system:xxx=5;set system:yyy=${system:xxx};\nlss;",
-        "cannot recognize input near 'lss' '<EOF>' '<EOF>'", errS, null, ERRNO_OK, true);
+        "cannot recognize input near 'lss' '<EOF>' '<EOF>'", errS, null, ERRNO_OTHER, true);
   }
 
   @Test
   public void testUseCurrentDB1() {
     verifyCMD(
         "create database if not exists testDB; set hive.cli.print.current.db=true;use testDB;\n"
-            + "use default;drop if exists testDB;", "hive (testDB)>", os, null, ERRNO_OK, true);
+            + "use default;drop if exists testDB;", "hive (testDB)>", os, null, ERRNO_OTHER, true);
   }
 
   @Test
   public void testUseCurrentDB2() {
     verifyCMD(
         "create database if not exists testDB; set hive.cli.print.current.db=true;use\ntestDB;\nuse default;drop if exists testDB;",
-        "hive (testDB)>", os, null, ERRNO_OK, true);
+        "hive (testDB)>", os, null, ERRNO_OTHER, true);
   }
 
   @Test
   public void testUseCurrentDB3() {
     verifyCMD(
         "create database if not exists testDB; set hive.cli.print.current.db=true;use  testDB;\n"
-            + "use default;drop if exists testDB;", "hive (testDB)>", os, null, ERRNO_OK, true);
+            + "use default;drop if exists testDB;", "hive (testDB)>", os, null, ERRNO_OTHER, true);
   }
 
   @Test
   public void testUseInvalidDB() {
     verifyCMD("set hive.cli.print.current.db=true;use invalidDB;",
-        "hive (invalidDB)>", os, null, ERRNO_OK, false);
+        "hive (invalidDB)>", os, null, ERRNO_OTHER, false);
   }
 
   @Test


[23/24] hive git commit: HIVE-13365. Change the MiniLLAPCluster to work with a MiniZKCluster, and potentially allow multiple instances of LLAP within the MiniLlapCluster. (Siddharth Seth, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
HIVE-13365. Change the MiniLLAPCluster to work with a MiniZKCluster, and potentially allow multiple instances of LLAP within the MiniLlapCluster. (Siddharth Seth, 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/91ab819a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/91ab819a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/91ab819a

Branch: refs/heads/llap
Commit: 91ab819a18d6271a6c8905d085ad90b1b184ecae
Parents: b446502
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 4 15:23:37 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Apr 4 15:23:37 2016 -0700

----------------------------------------------------------------------
 itests/hive-unit/pom.xml                        |   2 -
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   2 +-
 .../apache/hadoop/hive/llap/LlapItUtils.java    |  10 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |  10 +-
 .../hive/llap/daemon/MiniLlapCluster.java       | 145 ++++++++++++-------
 5 files changed, 109 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/91ab819a/itests/hive-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml
index 7219f1d..ae231de 100644
--- a/itests/hive-unit/pom.xml
+++ b/itests/hive-unit/pom.xml
@@ -210,14 +210,12 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
       <version>${hbase.version}</version>
-      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
       <version>${hbase.version}</version>
       <type>test-jar</type>
-      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/91ab819a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index 6141a1a..6b337d2 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -202,7 +202,7 @@ public class MiniHS2 extends AbstractHiveService {
         if (usePortsFromConf) {
           hiveConf.setBoolean("minillap.usePortsFromConf", true);
         }
-        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(hiveConf, null);
+        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(hiveConf, null, null);
 
         mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 4, uriString);
         break;

http://git-wip-us.apache.org/repos/asf/hive/blob/91ab819a/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java b/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java
index cb4aba5..c1a32c9 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java
@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
 import org.apache.hadoop.hive.llap.daemon.MiniLlapCluster;
@@ -36,7 +37,9 @@ public class LlapItUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapItUtils.class);
 
-  public static MiniLlapCluster startAndGetMiniLlapCluster(Configuration conf, String confDir) throws
+  public static MiniLlapCluster startAndGetMiniLlapCluster(Configuration conf,
+                                                           MiniZooKeeperCluster miniZkCluster,
+                                                           String confDir) throws
       IOException {
     MiniLlapCluster llapCluster;
     LOG.info("Using conf dir: {}", confDir);
@@ -57,11 +60,14 @@ public class LlapItUtils {
     // enabling this will cause test failures in Mac OS X
     final boolean directMemoryEnabled = false;
     final int numLocalDirs = 1;
-    LOG.info("MiniLlap Configs - maxMemory: " + maxMemory + " memoryForCache: " + memoryForCache
+    LOG.info("MiniLlap Configs -  maxMemory: " + maxMemory +
+        " memoryForCache: " + memoryForCache
         + " totalExecutorMemory: " + totalExecutorMemory + " numExecutors: " + numExecutors
         + " asyncIOEnabled: " + asyncIOEnabled + " directMemoryEnabled: " + directMemoryEnabled
         + " numLocalDirs: " + numLocalDirs);
     llapCluster = MiniLlapCluster.create(clusterName,
+        miniZkCluster,
+        1,
         numExecutors,
         totalExecutorMemory,
         asyncIOEnabled,

http://git-wip-us.apache.org/repos/asf/hive/blob/91ab819a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 5ccbcba..8473436 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -429,6 +429,9 @@ public class QTestUtil {
         fs = dfs.getFileSystem();
       }
 
+      setup = new QTestSetup();
+      setup.preTest(conf);
+
       String uriString = WindowsPathUtil.getHdfsUriString(fs.getUri().toString());
       if (clusterType == MiniClusterType.tez) {
         if (confDir != null && !confDir.isEmpty()) {
@@ -437,13 +440,16 @@ public class QTestUtil {
         }
         mr = shims.getMiniTezCluster(conf, 4, uriString);
       } else if (clusterType == MiniClusterType.llap) {
-        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(conf, confDir);
+        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(conf, setup.zooKeeperCluster, confDir);
         mr = shims.getMiniTezCluster(conf, 2, uriString);
       } else if (clusterType == MiniClusterType.miniSparkOnYarn) {
         mr = shims.getMiniSparkCluster(conf, 4, uriString, 1);
       } else {
         mr = shims.getMiniMrCluster(conf, 4, uriString, 1);
       }
+    } else {
+      setup = new QTestSetup();
+      setup.preTest(conf);
     }
 
     initConf();
@@ -471,8 +477,6 @@ public class QTestUtil {
 
     overWrite = "true".equalsIgnoreCase(System.getProperty("test.output.overwrite"));
 
-    setup = new QTestSetup();
-    setup.preTest(conf);
     init();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/91ab819a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
index a09c0b2..9871702 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
@@ -14,12 +14,11 @@
 
 package org.apache.hadoop.hive.llap.daemon;
 
+import javax.annotation.Nullable;
 import java.io.File;
 import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Iterator;
-import java.util.Map;
 
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -41,47 +40,57 @@ public class MiniLlapCluster extends AbstractService {
   private static final Logger LOG = LoggerFactory.getLogger(MiniLlapCluster.class);
 
   private final File testWorkDir;
+  private final String clusterNameTrimmed;
+  private final long numInstances;
   private final long execBytesPerService;
   private final boolean llapIoEnabled;
   private final boolean ioIsDirect;
   private final long ioBytesPerService;
   private final int numExecutorsPerService;
+  private final File zkWorkDir;
   private final String[] localDirs;
   private final Configuration clusterSpecificConfiguration = new Configuration(false);
 
-  private LlapDaemon llapDaemon;
+  private final LlapDaemon [] llapDaemons;
+  private MiniZooKeeperCluster miniZooKeeperCluster;
+  private final boolean ownZkCluster;
 
-  public static MiniLlapCluster create(String clusterName, int numExecutorsPerService,
-      long execBytePerService, boolean llapIoEnabled, boolean ioIsDirect, long ioBytesPerService,
-      int numLocalDirs) {
-    return new MiniLlapCluster(clusterName, numExecutorsPerService, execBytePerService,
+
+  public static MiniLlapCluster create(String clusterName,
+                                       @Nullable MiniZooKeeperCluster miniZkCluster,
+                                       int numInstances,
+                                       int numExecutorsPerService,
+                                       long execBytePerService, boolean llapIoEnabled,
+                                       boolean ioIsDirect, long ioBytesPerService,
+                                       int numLocalDirs) {
+    return new MiniLlapCluster(clusterName, miniZkCluster, numInstances, numExecutorsPerService,
+        execBytePerService,
         llapIoEnabled, ioIsDirect, ioBytesPerService, numLocalDirs);
   }
 
-  public static MiniLlapCluster createAndLaunch(Configuration conf, String clusterName,
-      int numExecutorsPerService, long execBytePerService, boolean llapIoEnabled,
-      boolean ioIsDirect, long ioBytesPerService, int numLocalDirs) {
-    MiniLlapCluster miniLlapCluster = create(clusterName, numExecutorsPerService,
-        execBytePerService, llapIoEnabled, ioIsDirect, ioBytesPerService, numLocalDirs);
-    miniLlapCluster.init(conf);
-    miniLlapCluster.start();
-    Configuration llapConf = miniLlapCluster.getClusterSpecificConfiguration();
-    Iterator<Map.Entry<String, String>> confIter = llapConf.iterator();
-    while (confIter.hasNext()) {
-      Map.Entry<String, String> entry = confIter.next();
-      conf.set(entry.getKey(), entry.getValue());
-    }
-    return miniLlapCluster;
+  public static MiniLlapCluster create(String clusterName,
+                                       @Nullable MiniZooKeeperCluster miniZkCluster,
+                                       int numExecutorsPerService,
+                                       long execBytePerService, boolean llapIoEnabled,
+                                       boolean ioIsDirect, long ioBytesPerService,
+                                       int numLocalDirs) {
+    return create(clusterName, miniZkCluster, 1, numExecutorsPerService, execBytePerService,
+        llapIoEnabled,
+        ioIsDirect, ioBytesPerService, numLocalDirs);
   }
 
-  // TODO Add support for multiple instances
-  private MiniLlapCluster(String clusterName, int numExecutorsPerService, long execMemoryPerService,
-                          boolean llapIoEnabled, boolean ioIsDirect, long ioBytesPerService, int numLocalDirs) {
+  private MiniLlapCluster(String clusterName, @Nullable MiniZooKeeperCluster miniZkCluster,
+                          int numInstances, int numExecutorsPerService, long execMemoryPerService,
+                          boolean llapIoEnabled, boolean ioIsDirect, long ioBytesPerService,
+                          int numLocalDirs) {
     super(clusterName + "_" + MiniLlapCluster.class.getSimpleName());
     Preconditions.checkArgument(numExecutorsPerService > 0);
     Preconditions.checkArgument(execMemoryPerService > 0);
     Preconditions.checkArgument(numLocalDirs > 0);
-    String clusterNameTrimmed = clusterName.replace("$", "") + "_" + MiniLlapCluster.class.getSimpleName();
+    this.numInstances = numInstances;
+
+    this.clusterNameTrimmed = clusterName.replace("$", "") + "_" + MiniLlapCluster.class.getSimpleName();
+    this.llapDaemons = new LlapDaemon[numInstances];
     File targetWorkDir = new File("target", clusterNameTrimmed);
     try {
       FileContext.getLocalFSFileContext().delete(
@@ -123,8 +132,18 @@ public class MiniLlapCluster extends AbstractService {
 
       this.testWorkDir = link;
     } else {
+      targetWorkDir.mkdir();
       this.testWorkDir = targetWorkDir;
     }
+    if (miniZkCluster == null) {
+      ownZkCluster = true;
+      this.zkWorkDir = new File(testWorkDir, "mini-zk-cluster");
+      zkWorkDir.mkdir();
+    } else {
+      miniZooKeeperCluster = miniZkCluster;
+      ownZkCluster = false;
+      this.zkWorkDir = null;
+    }
     this.numExecutorsPerService = numExecutorsPerService;
     this.execBytesPerService = execMemoryPerService;
     this.ioIsDirect = ioIsDirect;
@@ -142,12 +161,13 @@ public class MiniLlapCluster extends AbstractService {
   }
 
   @Override
-  public void serviceInit(Configuration conf) {
+  public void serviceInit(Configuration conf) throws IOException, InterruptedException {
     int rpcPort = 0;
     int mngPort = 0;
     int shufflePort = 0;
     int webPort = 0;
     boolean usePortsFromConf = conf.getBoolean("minillap.usePortsFromConf", false);
+    LOG.info("MiniLlap configured to use ports from conf: {}", usePortsFromConf);
     if (usePortsFromConf) {
       rpcPort = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_RPC_PORT);
       mngPort = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_MANAGEMENT_RPC_PORT);
@@ -155,43 +175,61 @@ public class MiniLlapCluster extends AbstractService {
       webPort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_WEB_PORT);
     }
 
-    llapDaemon = new LlapDaemon(conf, numExecutorsPerService, execBytesPerService, llapIoEnabled,
-        ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort);
-    llapDaemon.init(conf);
+    if (ownZkCluster) {
+      miniZooKeeperCluster = new MiniZooKeeperCluster();
+      miniZooKeeperCluster.startup(zkWorkDir);
+    } else {
+      // Already setup in the create method
+    } 
+
+    conf.set(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname, "@" + clusterNameTrimmed);
+    conf.set(ConfVars.HIVE_ZOOKEEPER_QUORUM.varname, "localhost");
+    conf.setInt(ConfVars.HIVE_ZOOKEEPER_CLIENT_PORT.varname, miniZooKeeperCluster.getClientPort());
+  
+    LOG.info("Initializing {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);
+    for (int i = 0 ;i < numInstances ; i++) {
+      llapDaemons[i] = new LlapDaemon(conf, numExecutorsPerService, execBytesPerService, llapIoEnabled,
+          ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort);
+      llapDaemons[i].init(new Configuration(conf));
+    }
+    LOG.info("Initialized {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);
   }
 
   @Override
   public void serviceStart() {
-    llapDaemon.start();
-
-    clusterSpecificConfiguration.set(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname,
-        getServiceAddress().getHostName());
-    clusterSpecificConfiguration.setInt(ConfVars.LLAP_DAEMON_RPC_PORT.varname,
-        getServiceAddress().getPort());
-
-    clusterSpecificConfiguration.setInt(
-        ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname,
-        numExecutorsPerService);
-    clusterSpecificConfiguration.setLong(
-        ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, execBytesPerService);
+    LOG.info("Starting {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);
+    for (int i = 0 ;i < numInstances ; i++) {
+      llapDaemons[i].start();
+    }
+    LOG.info("Started {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);
+
     // Optimize local fetch does not work with LLAP due to different local directories
     // used by containers and LLAP
     clusterSpecificConfiguration
         .setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false);
+    clusterSpecificConfiguration.set(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname, "@" + clusterNameTrimmed);
   }
 
   @Override
-  public void serviceStop() {
-    if (llapDaemon != null) {
-      llapDaemon.stop();
-      llapDaemon = null;
+  public void serviceStop() throws IOException {
+    for (int i = 0 ; i < numInstances ; i++) {
+      if (llapDaemons[i] != null) {
+        llapDaemons[i].stop();
+        llapDaemons[i] = null;
+      }
+    }
+    if (ownZkCluster) {
+      if (miniZooKeeperCluster != null) {
+        LOG.info("Stopping MiniZooKeeper cluster");
+        miniZooKeeperCluster.shutdown();
+        miniZooKeeperCluster = null;
+        LOG.info("Stopped MiniZooKeeper cluster");
+      }
+    } else {
+      LOG.info("Not stopping MiniZK cluster since it is now owned by us"); 
     }
   }
 
-  private InetSocketAddress getServiceAddress() {
-    Preconditions.checkState(getServiceState() == Service.STATE.STARTED);
-    return llapDaemon.getListenerAddress();
-  }
 
   public Configuration getClusterSpecificConfiguration() {
     Preconditions.checkState(getServiceState() == Service.STATE.STARTED);
@@ -200,7 +238,10 @@ public class MiniLlapCluster extends AbstractService {
 
   // Mainly for verification
   public long getNumSubmissions() {
-    return llapDaemon.getNumSubmissions();
+    int numSubmissions = 0;
+    for (int i = 0 ; i < numInstances ; i++) {
+      numSubmissions += llapDaemons[i].getNumSubmissions();
+    }
+    return numSubmissions;
   }
-
 }


[02/24] hive git commit: HIVE-13372: Hive Macro overwritten when multiple macros are used in one column (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by ss...@apache.org.
HIVE-13372: Hive Macro overwritten when multiple macros are used in one column (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/40088453
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/40088453
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/40088453

Branch: refs/heads/llap
Commit: 40088453b57a9bc6abfa7d0a23c1a78d39390212
Parents: 255069e
Author: Pengcheng Xiong <px...@apache.org>
Authored: Wed Mar 30 21:34:01 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Wed Mar 30 21:34:01 2016 -0700

----------------------------------------------------------------------
 .../hive/ql/plan/ExprNodeGenericFuncDesc.java   |  5 +-
 ql/src/test/queries/clientpositive/macro_1.q    | 29 +++++++
 .../test/results/clientpositive/macro_1.q.out   | 84 ++++++++++++++++++++
 3 files changed, 117 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/40088453/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java
index b7c1445..9e0159c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java
@@ -284,7 +284,10 @@ public class ExprNodeGenericFuncDesc extends ExprNodeDesc implements
     }
 
     if (genericUDF instanceof GenericUDFMacro) {
-      if (funcText != null && !funcText.equals(dest.funcText)) {
+      // if getMacroName is null, we always treat it different from others.
+      if (((GenericUDFMacro) genericUDF).getMacroName() == null
+          || !(((GenericUDFMacro) genericUDF).getMacroName()
+              .equals(((GenericUDFMacro) dest.genericUDF).getMacroName()))) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/40088453/ql/src/test/queries/clientpositive/macro_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/macro_1.q b/ql/src/test/queries/clientpositive/macro_1.q
new file mode 100644
index 0000000..dddc8e2
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/macro_1.q
@@ -0,0 +1,29 @@
+set hive.mapred.mode=nonstrict;
+
+CREATE TEMPORARY MACRO STRING_LEN(x string) length(x);
+CREATE TEMPORARY MACRO STRING_LEN_PLUS_ONE(x string) length(x)+1;
+CREATE TEMPORARY MACRO STRING_LEN_PLUS_TWO(x string) length(x)+2;
+
+create table macro_test (x string);
+
+insert into table macro_test values ("bb"), ("a"), ("ccc");
+
+SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test;
+
+SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test
+sort by a;
+
+
+SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test
+sort by a desc;
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/40088453/ql/src/test/results/clientpositive/macro_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/macro_1.q.out b/ql/src/test/results/clientpositive/macro_1.q.out
new file mode 100644
index 0000000..82b4ad9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/macro_1.q.out
@@ -0,0 +1,84 @@
+PREHOOK: query: CREATE TEMPORARY MACRO STRING_LEN(x string) length(x)
+PREHOOK: type: CREATEMACRO
+PREHOOK: Output: database:default
+POSTHOOK: query: CREATE TEMPORARY MACRO STRING_LEN(x string) length(x)
+POSTHOOK: type: CREATEMACRO
+POSTHOOK: Output: database:default
+PREHOOK: query: CREATE TEMPORARY MACRO STRING_LEN_PLUS_ONE(x string) length(x)+1
+PREHOOK: type: CREATEMACRO
+PREHOOK: Output: database:default
+POSTHOOK: query: CREATE TEMPORARY MACRO STRING_LEN_PLUS_ONE(x string) length(x)+1
+POSTHOOK: type: CREATEMACRO
+POSTHOOK: Output: database:default
+PREHOOK: query: CREATE TEMPORARY MACRO STRING_LEN_PLUS_TWO(x string) length(x)+2
+PREHOOK: type: CREATEMACRO
+PREHOOK: Output: database:default
+POSTHOOK: query: CREATE TEMPORARY MACRO STRING_LEN_PLUS_TWO(x string) length(x)+2
+POSTHOOK: type: CREATEMACRO
+POSTHOOK: Output: database:default
+PREHOOK: query: create table macro_test (x string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@macro_test
+POSTHOOK: query: create table macro_test (x string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@macro_test
+PREHOOK: query: insert into table macro_test values ("bb"), ("a"), ("ccc")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@macro_test
+POSTHOOK: query: insert into table macro_test values ("bb"), ("a"), ("ccc")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@macro_test
+POSTHOOK: Lineage: macro_test.x SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test
+PREHOOK: type: QUERY
+PREHOOK: Input: default@macro_test
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@macro_test
+#### A masked pattern was here ####
+2:3:4
+1:2:3
+3:4:5
+PREHOOK: query: SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test
+sort by a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@macro_test
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test
+sort by a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@macro_test
+#### A masked pattern was here ####
+1:2:3
+2:3:4
+3:4:5
+PREHOOK: query: SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test
+sort by a desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@macro_test
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT
+    CONCAT(STRING_LEN(x), ":", STRING_LEN_PLUS_ONE(x), ":", STRING_LEN_PLUS_TWO(x)) a
+FROM macro_test
+sort by a desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@macro_test
+#### A masked pattern was here ####
+3:4:5
+2:3:4
+1:2:3


[18/24] hive git commit: HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/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
new file mode 100644
index 0000000..97b1219
--- /dev/null
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/HiveServerException.java
@@ -0,0 +1,601 @@
+/**
+ * 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/98303635/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
new file mode 100644
index 0000000..46c71ee
--- /dev/null
+++ b/service/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/service/JobTrackerState.java
@@ -0,0 +1,45 @@
+/**
+ * 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;
+    }
+  }
+}


[22/24] hive git commit: HIVE-13381 : Timestamp & date should have precedence in type hierarchy than string group (Ashutosh Chauhan via Jason Dere)

Posted by ss...@apache.org.
HIVE-13381 : Timestamp & date should have precedence in type hierarchy than string group (Ashutosh Chauhan via Jason Dere)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/llap
Commit: b44650231ad2708fa73346164ae9c329ad36d6cb
Parents: 9830363
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Tue Mar 29 19:01:24 2016 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Mon Apr 4 13:11:07 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   9 +-
 .../ql/exec/vector/VectorizationContext.java    |  12 +-
 .../hive/ql/exec/TestFunctionRegistry.java      |  18 ++-
 .../exec/vector/TestVectorizationContext.java   |  17 +-
 .../queries/clientpositive/cast_on_constant.q   |   7 +
 .../clientpositive/cast_on_constant.q.out       | 160 +++++++++++++++++++
 6 files changed, 198 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b4465023/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 56b96b4..1343b39 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -737,7 +737,14 @@ public final class FunctionRegistry {
       return getTypeInfoForPrimitiveCategory(
           (PrimitiveTypeInfo)a, (PrimitiveTypeInfo)b,PrimitiveCategory.STRING);
     }
-
+    // timestamp/date is higher precedence than String_GROUP
+    if (pgA == PrimitiveGrouping.STRING_GROUP && pgB == PrimitiveGrouping.DATE_GROUP) {
+      return b;
+    }
+    // date/timestamp is higher precedence than String_GROUP
+    if (pgB == PrimitiveGrouping.STRING_GROUP && pgA == PrimitiveGrouping.DATE_GROUP) {
+      return a;
+    }
     // Another special case, because timestamp is not implicitly convertible to numeric types.
     if ((pgA == PrimitiveGrouping.NUMERIC_GROUP || pgB == PrimitiveGrouping.NUMERIC_GROUP)
         && (pcA == PrimitiveCategory.TIMESTAMP || pcB == PrimitiveCategory.TIMESTAMP)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b4465023/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 1eb960d..30a0f5a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -155,7 +155,7 @@ public class VectorizationContext {
 
   VectorExpressionDescriptor vMap;
 
-  private List<String> initialColumnNames;
+  private final List<String> initialColumnNames;
 
   private List<Integer> projectedColumns;
   private List<String> projectionColumnNames;
@@ -712,7 +712,7 @@ public class VectorizationContext {
         genericUdf = new GenericUDFToDate();
         break;
       case TIMESTAMP:
-        genericUdf = new GenericUDFToUnixTimeStamp();
+        genericUdf = new GenericUDFTimestamp();
         break;
       case INTERVAL_YEAR_MONTH:
         genericUdf = new GenericUDFToIntervalYearMonth();
@@ -1329,7 +1329,7 @@ public class VectorizationContext {
     case INT:
     case LONG:
       return InConstantType.INT_FAMILY;
-  
+
     case DATE:
       return InConstantType.TIMESTAMP;
 
@@ -1339,16 +1339,16 @@ public class VectorizationContext {
     case FLOAT:
     case DOUBLE:
       return InConstantType.FLOAT_FAMILY;
-  
+
     case STRING:
     case CHAR:
     case VARCHAR:
     case BINARY:
       return InConstantType.STRING_FAMILY;
-  
+
     case DECIMAL:
       return InConstantType.DECIMAL;
-  
+
 
     case INTERVAL_YEAR_MONTH:
     case INTERVAL_DAY_TIME:

http://git-wip-us.apache.org/repos/asf/hive/blob/b4465023/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
index 6a83c32..8488c21 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
@@ -253,9 +253,13 @@ public class TestFunctionRegistry extends TestCase {
                TypeInfoFactory.doubleTypeInfo);
 
     comparison(TypeInfoFactory.dateTypeInfo, TypeInfoFactory.stringTypeInfo,
-        TypeInfoFactory.stringTypeInfo);
+        TypeInfoFactory.dateTypeInfo);
     comparison(TypeInfoFactory.stringTypeInfo, TypeInfoFactory.dateTypeInfo,
-        TypeInfoFactory.stringTypeInfo);
+        TypeInfoFactory.dateTypeInfo);
+    comparison(TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.stringTypeInfo,
+        TypeInfoFactory.timestampTypeInfo);
+    comparison(TypeInfoFactory.stringTypeInfo, TypeInfoFactory.timestampTypeInfo,
+        TypeInfoFactory.timestampTypeInfo);
 
     comparison(TypeInfoFactory.intTypeInfo, TypeInfoFactory.timestampTypeInfo,
         TypeInfoFactory.doubleTypeInfo);
@@ -364,15 +368,15 @@ public class TestFunctionRegistry extends TestCase {
 
     // non-qualified types should simply return the TypeInfo associated with that type
     assertEquals(TypeInfoFactory.stringTypeInfo, FunctionRegistry.getTypeInfoForPrimitiveCategory(
-        (PrimitiveTypeInfo) varchar10, (PrimitiveTypeInfo) TypeInfoFactory.stringTypeInfo,
+        (PrimitiveTypeInfo) varchar10, TypeInfoFactory.stringTypeInfo,
         PrimitiveCategory.STRING));
     assertEquals(TypeInfoFactory.stringTypeInfo, FunctionRegistry.getTypeInfoForPrimitiveCategory(
-        (PrimitiveTypeInfo) TypeInfoFactory.stringTypeInfo,
-        (PrimitiveTypeInfo) TypeInfoFactory.stringTypeInfo,
+        TypeInfoFactory.stringTypeInfo,
+        TypeInfoFactory.stringTypeInfo,
         PrimitiveCategory.STRING));
     assertEquals(TypeInfoFactory.doubleTypeInfo, FunctionRegistry.getTypeInfoForPrimitiveCategory(
-        (PrimitiveTypeInfo) TypeInfoFactory.doubleTypeInfo,
-        (PrimitiveTypeInfo) TypeInfoFactory.stringTypeInfo,
+        TypeInfoFactory.doubleTypeInfo,
+        TypeInfoFactory.stringTypeInfo,
         PrimitiveCategory.DOUBLE));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b4465023/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
index e4c7529..bb37a04 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
@@ -23,14 +23,9 @@ import static org.junit.Assert.assertTrue;
 
 import java.sql.Timestamp;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
-
-import junit.framework.Assert;
 
 import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.BRoundWithNumDigitsDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ColAndCol;
@@ -73,11 +68,12 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringUpper;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampDate;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampTimestamp;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFYearDate;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterStringColumnInList;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterLongColumnInList;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterDoubleColumnInList;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFYearTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterTimestampColumnBetween;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterTimestampColumnNotBetween;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprLongColumnLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprLongScalarLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprLongScalarLongColumn;
@@ -144,13 +140,12 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFPower;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFRound;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPPlus;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDecimal;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUnixTimeStamp;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFTimestamp;
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class TestVectorizationContext {
@@ -1215,12 +1210,12 @@ public class TestVectorizationContext {
     children1.set(2, new ExprNodeConstantDesc("2013-11-05 00:00:00.000"));
     children1.set(3, new ExprNodeConstantDesc("2013-11-06 00:00:00.000"));
     ve = vc.getVectorExpression(exprDesc, VectorExpressionDescriptor.Mode.FILTER);
-    assertEquals(FilterStringColumnBetween.class, ve.getClass());
+    assertEquals(FilterTimestampColumnBetween.class, ve.getClass());
 
     // timestamp NOT BETWEEN
     children1.set(0, new ExprNodeConstantDesc(new Boolean(true)));
     ve = vc.getVectorExpression(exprDesc, VectorExpressionDescriptor.Mode.FILTER);
-    assertEquals(FilterStringColumnNotBetween.class, ve.getClass());
+    assertEquals(FilterTimestampColumnNotBetween.class, ve.getClass());
   }
 
   // Test translation of both IN filters and boolean-valued IN expressions (non-filters).
@@ -1468,7 +1463,7 @@ public class TestVectorizationContext {
     children1.set(2,  col3Expr);
     ve = vc.getVectorExpression(exprDesc);
     assertTrue(ve instanceof IfExprCharScalarStringGroupColumn);
- 
+
     // test for VARCHAR type
     VarcharTypeInfo varcharTypeInfo = new VarcharTypeInfo(10);
     constDesc2 = new ExprNodeConstantDesc(varcharTypeInfo, new HiveVarchar("Alpha", 10));

http://git-wip-us.apache.org/repos/asf/hive/blob/b4465023/ql/src/test/queries/clientpositive/cast_on_constant.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cast_on_constant.q b/ql/src/test/queries/clientpositive/cast_on_constant.q
new file mode 100644
index 0000000..aabb9c6
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/cast_on_constant.q
@@ -0,0 +1,7 @@
+create table t1(ts_field timestamp, date_field date);
+explain select * from t1 where ts_field = "2016-01-23 00:00:00";
+explain select * from t1 where date_field = "2016-01-23";
+explain select * from t1 where ts_field = timestamp '2016-01-23 00:00:00';
+explain select * from t1 where date_field = date '2016-01-23';
+
+drop table t1;

http://git-wip-us.apache.org/repos/asf/hive/blob/b4465023/ql/src/test/results/clientpositive/cast_on_constant.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cast_on_constant.q.out b/ql/src/test/results/clientpositive/cast_on_constant.q.out
new file mode 100644
index 0000000..f8d6a0d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/cast_on_constant.q.out
@@ -0,0 +1,160 @@
+PREHOOK: query: create table t1(ts_field timestamp, date_field date)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+POSTHOOK: query: create table t1(ts_field timestamp, date_field date)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+PREHOOK: query: explain select * from t1 where ts_field = "2016-01-23 00:00:00"
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from t1 where ts_field = "2016-01-23 00:00:00"
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (ts_field = 2016-01-23 00:00:00.0) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 2016-01-23 00:00:00.0 (type: timestamp), date_field (type: date)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select * from t1 where date_field = "2016-01-23"
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from t1 where date_field = "2016-01-23"
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (date_field = 2016-01-23) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: ts_field (type: timestamp), 2016-01-23 (type: date)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select * from t1 where ts_field = timestamp '2016-01-23 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from t1 where ts_field = timestamp '2016-01-23 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (ts_field = 2016-01-23 00:00:00.0) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 2016-01-23 00:00:00.0 (type: timestamp), date_field (type: date)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select * from t1 where date_field = date '2016-01-23'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from t1 where date_field = date '2016-01-23'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (date_field = 2016-01-23) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: ts_field (type: timestamp), 2016-01-23 (type: date)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: drop table t1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t1
+PREHOOK: Output: default@t1
+POSTHOOK: query: drop table t1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: default@t1


[05/24] hive git commit: HIVE-11766. LLAP: Remove MiniLlapCluster from shim layer after hadoop-1 removal. (Siddharth Seth, reviewed by Prasanth Jayachandran)

Posted by ss...@apache.org.
HIVE-11766. LLAP: Remove MiniLlapCluster from shim layer after hadoop-1
removal. (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/184e0e1d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/184e0e1d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/184e0e1d

Branch: refs/heads/llap
Commit: 184e0e1da55c576cd8766c52423bcbcd4a2be69b
Parents: f41cbea
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Mar 31 14:53:59 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Mar 31 14:53:59 2016 -0700

----------------------------------------------------------------------
 data/conf/llap/hive-site.xml                    | 44 ++++++++++
 data/conf/llap/llap-daemon-site.xml             | 61 --------------
 itests/hive-unit/pom.xml                        | 11 ++-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   | 12 ++-
 itests/util/pom.xml                             | 11 +++
 .../apache/hadoop/hive/llap/LlapItUtils.java    | 84 ++++++++++++++++++++
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 17 ++--
 .../configuration/LlapDaemonConfiguration.java  |  8 ++
 .../apache/hadoop/hive/shims/Hadoop23Shims.java | 66 +--------------
 .../apache/hadoop/hive/shims/HadoopShims.java   |  2 +-
 10 files changed, 175 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/data/conf/llap/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/llap/hive-site.xml b/data/conf/llap/hive-site.xml
index c2bef58..72bdcfb 100644
--- a/data/conf/llap/hive-site.xml
+++ b/data/conf/llap/hive-site.xml
@@ -273,4 +273,48 @@
   <value>false</value>
 </property>
 
+<!-- llap properties -->
+
+<property>
+  <name>hive.llap.daemon.service.hosts</name>
+  <value>localhost</value>
+</property>
+
+<property>
+  <name>hive.llap.daemon.service.port</name>
+  <value>0</value>
+</property>
+
+<property>
+  <name>hive.llap.daemon.num.executors</name>
+  <value>4</value>
+</property>
+
+<property>
+  <name>hive.llap.daemon.task.scheduler.wait.queue.size</name>
+  <value>4</value>
+</property>
+
+<property>
+  <name>hive.llap.cache.allow.synthetic.fileid</name>
+  <value>true</value>
+</property>
+
+<!-- hadoop IPC options -->
+<property>
+  <name>ipc.client.low-latency</name>
+  <value>true</value>
+  <description> </description>
+</property>
+<property>
+  <name>ipc.client.tcpnodelay</name>
+  <value>true</value>
+  <description> </description>
+</property>
+<property>
+  <name>ipc.clients-per-factory</name>
+  <value>4</value>
+  <description> </description>
+</property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/data/conf/llap/llap-daemon-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/llap/llap-daemon-site.xml b/data/conf/llap/llap-daemon-site.xml
deleted file mode 100644
index 98c0f2b..0000000
--- a/data/conf/llap/llap-daemon-site.xml
+++ /dev/null
@@ -1,61 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-<configuration>
-<property>
-  <name>hive.llap.daemon.service.hosts</name>
-  <value>localhost</value>
-</property>
-
-<property>
-  <name>hive.llap.daemon.service.port</name>
-  <value>0</value>
-</property>
-
-<property>
-  <name>hive.llap.daemon.num.executors</name>
-  <value>4</value>
-</property>
-
-<property>
-  <name>hive.llap.daemon.task.scheduler.wait.queue.size</name>
-  <value>4</value>
-</property>
-
-<property>
-  <name>hive.llap.cache.allow.synthetic.fileid</name>
-  <value>true</value>
-</property>
-
-<!-- hadoop IPC options -->
-<property>
-  <name>ipc.client.low-latency</name>
-  <value>true</value>
-  <description> </description>
-</property>
-<property>
-  <name>ipc.client.tcpnodelay</name>
-  <value>true</value>
-  <description> </description>
-</property>
-<property>
-  <name>ipc.clients-per-factory</name>
-  <value>4</value>
-  <description> </description>
-</property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/itests/hive-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml
index 97786d9..7219f1d 100644
--- a/itests/hive-unit/pom.xml
+++ b/itests/hive-unit/pom.xml
@@ -75,6 +75,11 @@
       <artifactId>hive-hcatalog-streaming</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-it-util</artifactId>
+      <version>${project.version}</version>
+    </dependency>
     <!-- inter-project -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>
@@ -123,12 +128,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
-      <artifactId>hive-it-util</artifactId>
-      <version>${project.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hive</groupId>
       <artifactId>hive-jdbc</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index a9d9c76..eca2317 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.LlapItUtils;
+import org.apache.hadoop.hive.llap.daemon.MiniLlapCluster;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.WindowsPathUtil;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -60,6 +62,7 @@ public class MiniHS2 extends AbstractHiveService {
   private static final AtomicLong hs2Counter = new AtomicLong();
   private MiniMrShim mr;
   private MiniDFSShim dfs;
+  private MiniLlapCluster llapCluster = null;
   private final FileSystem localFS;
   private boolean useMiniKdc = false;
   private final String serverPrincipal;
@@ -187,13 +190,15 @@ public class MiniHS2 extends AbstractHiveService {
       // Initialize the execution engine based on cluster type
       switch (miniClusterType) {
       case TEZ:
-        mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 4, uriString, false);
+        mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 4, uriString);
         break;
       case LLAP:
         if (usePortsFromConf) {
           hiveConf.setBoolean("minillap.usePortsFromConf", true);
         }
-        mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 4, uriString, true);
+        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(hiveConf, null);
+
+        mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 4, uriString);
         break;
       case MR:
         mr = ShimLoader.getHadoopShims().getMiniMrCluster(hiveConf, 4, uriString, 1);
@@ -285,6 +290,9 @@ public class MiniHS2 extends AbstractHiveService {
     hiveServer2.stop();
     setStarted(false);
     try {
+      if (llapCluster != null) {
+        llapCluster.stop();
+      }
       if (mr != null) {
         mr.shutdown();
         mr = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/itests/util/pom.xml
----------------------------------------------------------------------
diff --git a/itests/util/pom.xml b/itests/util/pom.xml
index aaafc0a..4789586 100644
--- a/itests/util/pom.xml
+++ b/itests/util/pom.xml
@@ -97,6 +97,17 @@
       <classifier>tests</classifier>
     </dependency>
     <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-llap-server</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-llap-server</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java b/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java
new file mode 100644
index 0000000..cb4aba5
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/llap/LlapItUtils.java
@@ -0,0 +1,84 @@
+/**
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
+import org.apache.hadoop.hive.llap.daemon.MiniLlapCluster;
+import org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LlapItUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapItUtils.class);
+
+  public static MiniLlapCluster startAndGetMiniLlapCluster(Configuration conf, String confDir) throws
+      IOException {
+    MiniLlapCluster llapCluster;
+    LOG.info("Using conf dir: {}", confDir);
+    if (confDir != null && !confDir.isEmpty()) {
+      conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
+          + "/tez-site.xml"));
+    }
+
+    Configuration daemonConf = new LlapDaemonConfiguration(conf);
+    final String clusterName = "llap";
+    final long maxMemory = LlapDaemon.getTotalHeapSize();
+    // 15% for io cache
+    final long memoryForCache = (long) (0.15f * maxMemory);
+    // 75% for 4 executors
+    final long totalExecutorMemory = (long) (0.75f * maxMemory);
+    final int numExecutors = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_NUM_EXECUTORS);
+    final boolean asyncIOEnabled = true;
+    // enabling this will cause test failures in Mac OS X
+    final boolean directMemoryEnabled = false;
+    final int numLocalDirs = 1;
+    LOG.info("MiniLlap Configs - maxMemory: " + maxMemory + " memoryForCache: " + memoryForCache
+        + " totalExecutorMemory: " + totalExecutorMemory + " numExecutors: " + numExecutors
+        + " asyncIOEnabled: " + asyncIOEnabled + " directMemoryEnabled: " + directMemoryEnabled
+        + " numLocalDirs: " + numLocalDirs);
+    llapCluster = MiniLlapCluster.create(clusterName,
+        numExecutors,
+        totalExecutorMemory,
+        asyncIOEnabled,
+        directMemoryEnabled,
+        memoryForCache,
+        numLocalDirs);
+    llapCluster.init(daemonConf);
+    llapCluster.start();
+
+    // Augment conf with the settings from the started llap configuration.
+    Configuration llapConf = llapCluster.getClusterSpecificConfiguration();
+    Iterator<Map.Entry<String, String>> confIter = llapConf.iterator();
+    while (confIter.hasNext()) {
+      Map.Entry<String, String> entry = confIter.next();
+      conf.set(entry.getKey(), entry.getValue());
+    }
+    return llapCluster;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 5e81e98..5ccbcba 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -51,6 +51,7 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.Deque;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -82,6 +83,10 @@ import org.apache.hadoop.hive.common.io.SortAndDigestPrintStream;
 import org.apache.hadoop.hive.common.io.SortPrintStream;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.LlapItUtils;
+import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
+import org.apache.hadoop.hive.llap.daemon.MiniLlapCluster;
+import org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.Index;
@@ -166,6 +171,7 @@ public class QTestUtil {
   private HadoopShims.MiniMrShim mr = null;
   private HadoopShims.MiniDFSShim dfs = null;
   private HadoopShims.HdfsEncryptionShim hes = null;
+  private MiniLlapCluster llapCluster = null;
   private String hadoopVer = null;
   private QTestSetup setup = null;
   private TezSessionState tezSessionState = null;
@@ -429,15 +435,10 @@ public class QTestUtil {
           conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
               + "/tez-site.xml"));
         }
-        mr = shims.getMiniTezCluster(conf, 4, uriString, false);
+        mr = shims.getMiniTezCluster(conf, 4, uriString);
       } else if (clusterType == MiniClusterType.llap) {
-        if (confDir != null && !confDir.isEmpty()) {
-          conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
-              + "/tez-site.xml"));
-          conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
-              + "/llap-daemon-site.xml"));
-        }
-        mr = shims.getMiniTezCluster(conf, 2, uriString, true);
+        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(conf, confDir);
+        mr = shims.getMiniTezCluster(conf, 2, uriString);
       } else if (clusterType == MiniClusterType.miniSparkOnYarn) {
         mr = shims.getMiniSparkCluster(conf, 4, uriString, 1);
       } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/llap-server/src/java/org/apache/hadoop/hive/llap/configuration/LlapDaemonConfiguration.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/configuration/LlapDaemonConfiguration.java b/llap-server/src/java/org/apache/hadoop/hive/llap/configuration/LlapDaemonConfiguration.java
index 51e8509..88f3b19 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/configuration/LlapDaemonConfiguration.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/configuration/LlapDaemonConfiguration.java
@@ -14,6 +14,7 @@
 
 package org.apache.hadoop.hive.llap.configuration;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -21,6 +22,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 /**
  * Configuration for LLAP daemon processes only. This should not be used by any clients.
  */
+@InterfaceAudience.Private
 public class LlapDaemonConfiguration extends Configuration {
 
   @InterfaceAudience.Private
@@ -46,4 +48,10 @@ public class LlapDaemonConfiguration extends Configuration {
     }
     addResource(LLAP_DAEMON_SITE);
   }
+
+  @VisibleForTesting
+  public LlapDaemonConfiguration(Configuration conf) {
+    this();
+    addResource(conf);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
index 9a3a31c..e028212 100644
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
+++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
@@ -370,8 +370,8 @@ public class Hadoop23Shims extends HadoopShimsSecure {
    */
   @Override
   public MiniMrShim getMiniTezCluster(Configuration conf, int numberOfTaskTrackers,
-      String nameNode, boolean isLlap) throws IOException {
-    return new MiniTezShim(conf, numberOfTaskTrackers, nameNode, isLlap);
+      String nameNode) throws IOException {
+    return new MiniTezShim(conf, numberOfTaskTrackers, nameNode);
   }
 
   /**
@@ -381,11 +381,8 @@ public class Hadoop23Shims extends HadoopShimsSecure {
 
     private final MiniTezCluster mr;
     private final Configuration conf;
-    private Class<?> miniLlapKlass;
-    private Object miniLlapCluster;
 
-    public MiniTezShim(Configuration conf, int numberOfTaskTrackers, String nameNode,
-        boolean isLlap) throws IOException {
+    public MiniTezShim(Configuration conf, int numberOfTaskTrackers, String nameNode) throws IOException {
       mr = new MiniTezCluster("hive", numberOfTaskTrackers);
       conf.set("fs.defaultFS", nameNode);
       conf.set("tez.am.log.level", "DEBUG");
@@ -393,54 +390,6 @@ public class Hadoop23Shims extends HadoopShimsSecure {
       mr.init(conf);
       mr.start();
       this.conf = mr.getConfig();
-      if (isLlap) {
-        createAndLaunchLlapDaemon(this.conf);
-      } else {
-        miniLlapCluster = null;
-      }
-    }
-
-    private void createAndLaunchLlapDaemon(final Configuration conf)
-        throws IOException {
-      try {
-        final String clusterName = "llap";
-        Class<?> llapDaemonKlass =
-            Class.forName("org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon",
-                false, ShimLoader.class.getClassLoader());
-        Method totalMemMethod = llapDaemonKlass.getMethod("getTotalHeapSize");
-        final long maxMemory = (long) totalMemMethod.invoke(null);
-        // 15% for io cache
-        final long memoryForCache = (long) (0.15f * maxMemory);
-        // 75% for executors
-        final long totalExecutorMemory = (long) (0.75f * maxMemory);
-        final int numExecutors = conf.getInt("llap.daemon.num.executors", 4);
-        final boolean asyncIOEnabled = true;
-        // enabling this will cause test failures in Mac OS X
-        final boolean directMemoryEnabled = false;
-        final int numLocalDirs = 1;
-        LOG.info("MiniLlap Configs - maxMemory: " + maxMemory + " memoryForCache: " + memoryForCache
-            + " totalExecutorMemory: " + totalExecutorMemory + " numExecutors: " + numExecutors
-            + " asyncIOEnabled: " + asyncIOEnabled + " directMemoryEnabled: " + directMemoryEnabled
-            + " numLocalDirs: " + numLocalDirs);
-
-        miniLlapKlass = Class.forName("org.apache.hadoop.hive.llap.daemon.MiniLlapCluster",
-            false, ShimLoader.class.getClassLoader());
-        Method create = miniLlapKlass.getMethod("createAndLaunch", new Class[]{Configuration.class,
-            String.class, Integer.TYPE, Long.TYPE, Boolean.TYPE, Boolean.TYPE,
-            Long.TYPE, Integer.TYPE});
-        miniLlapCluster = create.invoke(null,
-            conf,
-            clusterName,
-            numExecutors,
-            totalExecutorMemory,
-            asyncIOEnabled,
-            directMemoryEnabled,
-            memoryForCache,
-            numLocalDirs);
-      } catch (Exception e) {
-        LOG.error("Unable to create MiniLlapCluster. Exception: " + e.getMessage());
-        throw new IOException(e);
-      }
     }
 
     @Override
@@ -458,15 +407,6 @@ public class Hadoop23Shims extends HadoopShimsSecure {
     @Override
     public void shutdown() throws IOException {
       mr.stop();
-
-      if (miniLlapKlass != null && miniLlapCluster != null) {
-        try {
-          Method stop = miniLlapKlass.getMethod("stop", new Class[]{});
-          stop.invoke(miniLlapCluster);
-        } catch (Exception e) {
-          LOG.error("Unable to stop llap daemon. Exception: " + e.getMessage());
-        }
-      }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/184e0e1d/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
index 7a5a9b5..a44d0c0 100644
--- a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
@@ -90,7 +90,7 @@ public interface HadoopShims {
       String nameNode, int numDir) throws IOException;
 
   public MiniMrShim getMiniTezCluster(Configuration conf, int numberOfTaskTrackers,
-      String nameNode, boolean isLlap) throws IOException;
+      String nameNode) throws IOException;
 
   public MiniMrShim getMiniSparkCluster(Configuration conf, int numberOfTaskTrackers,
       String nameNode, int numDir) throws IOException;


[21/24] hive git commit: HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, 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/98303635
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/98303635
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/98303635

Branch: refs/heads/llap
Commit: 983036358633cfbb6aec30003faac8280372b2c9
Parents: 6a1f8a8
Author: Wei Zheng <we...@apache.org>
Authored: Mon Apr 4 11:18:25 2016 -0700
Committer: Wei Zheng <we...@apache.org>
Committed: Mon Apr 4 11:18:25 2016 -0700

----------------------------------------------------------------------
 service-rpc/src/gen/thrift/gen-py/__init__.py   |    0
 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 +
 service/src/gen/thrift/gen-py/__init__.py       |    0
 .../gen-py/hive_service/ThriftHive-remote       | 1242 +++
 .../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 ++
 23 files changed, 20852 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/service-rpc/src/gen/thrift/gen-py/__init__.py
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-py/__init__.py b/service-rpc/src/gen/thrift/gen-py/__init__.py
new file mode 100644
index 0000000..e69de29


[11/24] hive git commit: HIVE-13402 : Temporarily disable failing spark tests (Addendum)

Posted by ss...@apache.org.
HIVE-13402 : Temporarily disable failing spark tests (Addendum)


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

Branch: refs/heads/llap
Commit: 21f18adac0f10008e0fe18be8e2d8b7070399066
Parents: 9a0dabd
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Fri Apr 1 17:14:52 2016 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Apr 1 17:14:52 2016 -0700

----------------------------------------------------------------------
 itests/src/test/resources/testconfiguration.properties    | 4 ----
 ql/src/test/results/clientpositive/spark/temp_table.q.out | 2 ++
 2 files changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/21f18ada/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index cf9e4c9..889884c 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -719,7 +719,6 @@ spark.query.files=add_part_multiple.q, \
   auto_join3.q, \
   auto_join30.q, \
   auto_join31.q, \
-  auto_join32.q, \
   auto_join4.q, \
   auto_join5.q, \
   auto_join6.q, \
@@ -740,7 +739,6 @@ spark.query.files=add_part_multiple.q, \
   auto_sortmerge_join_14.q, \
   auto_sortmerge_join_15.q, \
   auto_sortmerge_join_16.q, \
-  auto_sortmerge_join_2.q, \
   auto_sortmerge_join_3.q, \
   auto_sortmerge_join_4.q, \
   auto_sortmerge_join_5.q, \
@@ -926,7 +924,6 @@ spark.query.files=add_part_multiple.q, \
   join38.q, \
   join39.q, \
   join4.q, \
-  join40.q, \
   join41.q, \
   join5.q, \
   join6.q, \
@@ -1029,7 +1026,6 @@ spark.query.files=add_part_multiple.q, \
   ppd_join.q, \
   ppd_join2.q, \
   ppd_join3.q, \
-  ppd_join4.q, \
   ppd_join5.q, \
   ppd_join_filter.q, \
   ppd_multi_insert.q, \

http://git-wip-us.apache.org/repos/asf/hive/blob/21f18ada/ql/src/test/results/clientpositive/spark/temp_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/temp_table.q.out b/ql/src/test/results/clientpositive/spark/temp_table.q.out
index 1bc8b19..c2ec3b4 100644
--- a/ql/src/test/results/clientpositive/spark/temp_table.q.out
+++ b/ql/src/test/results/clientpositive/spark/temp_table.q.out
@@ -461,6 +461,8 @@ POSTHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@s
+POSTHOOK: Lineage: s.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: s.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select count(*) from s
 PREHOOK: type: QUERY
 PREHOOK: Input: default@s


[20/24] hive git commit: HIVE-13388 : Fix inconsistent content due to Thrift changes (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/98303635/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
new file mode 100644
index 0000000..a5448f0
--- /dev/null
+++ b/service/src/gen/thrift/gen-cpp/ThriftHive.cpp
@@ -0,0 +1,3544 @@
+/**
+ * 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>

[10/24] hive git commit: HIVE-13376 : HoS emits too many logs with application state (Szehon, via Rui Li and Xuefu)

Posted by ss...@apache.org.
HIVE-13376 : HoS emits too many logs with application state (Szehon, via Rui Li and Xuefu)


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

Branch: refs/heads/llap
Commit: 9a0dabdf439e11cccf9aa02e5356ab21617e1f6e
Parents: ac273b6
Author: Szehon Ho <sz...@cloudera.com>
Authored: Fri Apr 1 11:47:52 2016 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Fri Apr 1 11:49:09 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java   | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9a0dabdf/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
index 2427321..b36c60e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
@@ -53,6 +53,7 @@ public class HiveSparkClientFactory {
   private static final String SPARK_DEFAULT_APP_NAME = "Hive on Spark";
   private static final String SPARK_DEFAULT_SERIALIZER = "org.apache.spark.serializer.KryoSerializer";
   private static final String SPARK_DEFAULT_REFERENCE_TRACKING = "false";
+  private static final String SPARK_YARN_REPORT_INTERVAL = "spark.yarn.report.interval";
 
   public static HiveSparkClient createHiveSparkClient(HiveConf hiveconf) throws Exception {
     Map<String, String> sparkConf = initiateSparkConf(hiveconf);
@@ -187,6 +188,14 @@ public class HiveSparkClientFactory {
       }
     }
 
+    //The application reports tend to spam the hive logs.  This is controlled by spark, and the default seems to be 1s.
+    //If it is not specified, set it to a much higher number.  It can always be overriden by user.
+    String sparkYarnReportInterval = sparkConf.get(SPARK_YARN_REPORT_INTERVAL);
+    if (sparkMaster.startsWith("yarn") && sparkYarnReportInterval == null) {
+      //the new version of spark also takes time-units, but old versions do not.
+      sparkConf.put(SPARK_YARN_REPORT_INTERVAL, "60000");
+    }
+
     return sparkConf;
   }
 


[06/24] hive git commit: HIVE-13364. Allow llap to work with dynamic ports for rpc, shuffle, ui. (Siddharth Seth, reviewed by Prasanth Jayachandran)

Posted by ss...@apache.org.
HIVE-13364. Allow llap to work with dynamic ports for rpc, shuffle, ui. (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/547c5cfc
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/547c5cfc
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/547c5cfc

Branch: refs/heads/llap
Commit: 547c5cfce9587de31a58622589a63eba62a4b120
Parents: 184e0e1
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Mar 31 14:54:53 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Mar 31 14:54:53 2016 -0700

----------------------------------------------------------------------
 .../impl/LlapZookeeperRegistryImpl.java         |  9 +++---
 .../hive/llap/daemon/impl/LlapDaemon.java       | 34 ++++++++++++++++----
 .../daemon/impl/LlapProtocolServerImpl.java     |  7 ++--
 .../daemon/services/impl/LlapWebServices.java   | 13 ++++++--
 .../hive/llap/daemon/MiniLlapCluster.java       |  4 ++-
 5 files changed, 50 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/547c5cfc/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 c611d1a..ba38fb8 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
@@ -31,7 +31,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
@@ -68,8 +67,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.client.ZooKeeperSaslClient;
 import org.apache.zookeeper.data.ACL;
@@ -285,8 +282,10 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
         // No node exists, throw exception
         throw new Exception("Unable to create znode for this LLAP instance on ZooKeeper.");
       }
-      LOG.info("Created a znode on ZooKeeper for LLAP instance: {} znodePath: {}", rpcEndpoint,
-          znodePath);
+      LOG.info(
+          "Registered node. Created a znode on ZooKeeper for LLAP instance: rpc: {}, shuffle: {}," +
+              " webui: {}, mgmt: {}, znodePath: {} ",
+          rpcEndpoint, getShuffleEndpoint(), getServicesEndpoint(), getMngEndpoint(), znodePath);
     } catch (Exception e) {
       LOG.error("Unable to create a znode for this server instance", e);
       CloseableUtils.closeQuietly(znode);

http://git-wip-us.apache.org/repos/asf/hive/blob/547c5cfc/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 c8734a5..2fe59a2 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
@@ -100,7 +100,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
 
   public LlapDaemon(Configuration daemonConf, int numExecutors, long executorMemoryBytes,
       boolean ioEnabled, boolean isDirectCache, long ioMemoryBytes, String[] localDirs, int srvPort,
-      int mngPort, int shufflePort) {
+      int mngPort, int shufflePort, int webPort) {
     super("LlapDaemon");
 
     initializeLogging();
@@ -140,6 +140,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
         "numExecutors=" + numExecutors +
         ", rpcListenerPort=" + srvPort +
         ", mngListenerPort=" + mngPort +
+        ", webPort=" + webPort +
         ", workDirs=" + Arrays.toString(localDirs) +
         ", shufflePort=" + shufflePort +
         ", executorMemory=" + executorMemoryBytes +
@@ -206,12 +207,11 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
         amReporter, executorClassLoader);
     addIfService(containerRunner);
 
-    this.registry = new LlapRegistryService(true);
-    addIfService(registry);
+
     if (HiveConf.getBoolVar(daemonConf, HiveConf.ConfVars.HIVE_IN_TEST)) {
       this.webServices = null;
     } else {
-      this.webServices = new LlapWebServices();
+      this.webServices = new LlapWebServices(webPort);
       addIfService(webServices);
     }
     // Bring up the server only after all other components have started.
@@ -219,6 +219,9 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     // AMReporter after the server so that it gets the correct address. It knows how to deal with
     // requests before it is started.
     addIfService(amReporter);
+
+    // Not adding the registry as a service, since we need to control when it is initialized - conf used to pickup properties.
+    this.registry = new LlapRegistryService(true);
   }
 
   private void initializeLogging() {
@@ -289,11 +292,29 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     ShuffleHandler.initializeAndStart(shuffleHandlerConf);
     LOG.info("Setting shuffle port to: " + ShuffleHandler.get().getPort());
     this.shufflePort.set(ShuffleHandler.get().getPort());
+    getConfig()
+        .setInt(ConfVars.LLAP_DAEMON_YARN_SHUFFLE_PORT.varname, ShuffleHandler.get().getPort());
     super.serviceStart();
-    LOG.info("LlapDaemon serviceStart complete");
+
+    // Setup the actual ports in the configuration.
+    getConfig().setInt(ConfVars.LLAP_DAEMON_RPC_PORT.varname, server.getBindAddress().getPort());
+    getConfig().setInt(ConfVars.LLAP_MANAGEMENT_RPC_PORT.varname, server.getManagementBindAddress().getPort());
+    if (webServices != null) {
+      getConfig().setInt(ConfVars.LLAP_DAEMON_WEB_PORT.varname, webServices.getPort());
+    }
+
+    this.registry.init(getConfig());
+    this.registry.start();
+    LOG.info(
+        "LlapDaemon serviceStart complete. RPC Port={}, ManagementPort={}, ShuflePort={}, WebPort={}",
+        server.getBindAddress().getPort(), server.getManagementBindAddress().getPort(),
+        ShuffleHandler.get().getPort(), (webServices == null ? "" : webServices.getPort()));
   }
 
   public void serviceStop() throws Exception {
+    if (registry != null) {
+      this.registry.stop();
+    }
     super.serviceStop();
     ShuffleHandler.shutdown();
     shutdown();
@@ -341,6 +362,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       int mngPort = HiveConf.getIntVar(daemonConf, ConfVars.LLAP_MANAGEMENT_RPC_PORT);
       int shufflePort = daemonConf
           .getInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, ShuffleHandler.DEFAULT_SHUFFLE_PORT);
+      int webPort = HiveConf.getIntVar(daemonConf, ConfVars.LLAP_DAEMON_WEB_PORT);
       long executorMemoryBytes = HiveConf.getIntVar(
           daemonConf, ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB) * 1024l * 1024l;
 
@@ -348,7 +370,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       boolean isDirectCache = HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_ALLOCATOR_DIRECT);
       boolean isLlapIo = HiveConf.getBoolVar(daemonConf, HiveConf.ConfVars.LLAP_IO_ENABLED, true);
       llapDaemon = new LlapDaemon(daemonConf, numExecutors, executorMemoryBytes, isLlapIo,
-              isDirectCache, ioMemoryBytes, localDirs, rpcPort, mngPort, shufflePort);
+              isDirectCache, ioMemoryBytes, localDirs, rpcPort, mngPort, shufflePort, webPort);
 
       LOG.info("Adding shutdown hook for LlapDaemon");
       ShutdownHookManager.addShutdownHook(new CompositeServiceShutdownHook(llapDaemon), 1);

http://git-wip-us.apache.org/repos/asf/hive/blob/547c5cfc/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
index 3a25a66..e99e689 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
@@ -19,7 +19,6 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.util.concurrent.atomic.AtomicReference;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.io.ByteArrayDataOutput;
 import com.google.common.io.ByteStreams;
 import com.google.protobuf.BlockingService;
@@ -189,11 +188,15 @@ public class LlapProtocolServerImpl extends AbstractService
   }
 
   @InterfaceAudience.Private
-  @VisibleForTesting
   InetSocketAddress getBindAddress() {
     return srvAddress.get();
   }
 
+  @InterfaceAudience.Private
+  InetSocketAddress getManagementBindAddress() {
+    return mngAddress.get();
+  }
+
   private RPC.Server createServer(Class<?> pbProtocol, InetSocketAddress addr, Configuration conf,
                                   int numHandlers, BlockingService blockingService) throws
       IOException {

http://git-wip-us.apache.org/repos/asf/hive/blob/547c5cfc/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
index afb59c0..e4c622e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
@@ -20,9 +20,9 @@ package org.apache.hadoop.hive.llap.daemon.services.impl;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hive.http.HttpServer;
@@ -38,13 +38,14 @@ public class LlapWebServices extends AbstractService {
   private boolean useSSL = false;
   private boolean useSPNEGO = false;
 
-  public LlapWebServices() {
+  public LlapWebServices(int port) {
     super("LlapWebServices");
+    this.port = port;
   }
 
   @Override
   public void serviceInit(Configuration conf) {
-    this.port = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_WEB_PORT);
+
     this.useSSL = HiveConf.getBoolVar(conf, ConfVars.LLAP_DAEMON_WEB_SSL);
     this.useSPNEGO = HiveConf.getBoolVar(conf, ConfVars.LLAP_WEB_AUTO_AUTH);
     String bindAddress = "0.0.0.0";
@@ -69,6 +70,11 @@ public class LlapWebServices extends AbstractService {
     }
   }
 
+  @InterfaceAudience.Private
+  public int getPort() {
+    return this.http.getPort();
+  }
+
   @Override
   public void serviceStart() throws Exception {
     if (this.http != null) {
@@ -76,6 +82,7 @@ public class LlapWebServices extends AbstractService {
     }
   }
 
+  @Override
   public void serviceStop() throws Exception {
     if (this.http != null) {
       this.http.stop();

http://git-wip-us.apache.org/repos/asf/hive/blob/547c5cfc/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
index c920c24..a09c0b2 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
@@ -146,15 +146,17 @@ public class MiniLlapCluster extends AbstractService {
     int rpcPort = 0;
     int mngPort = 0;
     int shufflePort = 0;
+    int webPort = 0;
     boolean usePortsFromConf = conf.getBoolean("minillap.usePortsFromConf", false);
     if (usePortsFromConf) {
       rpcPort = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_RPC_PORT);
       mngPort = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_MANAGEMENT_RPC_PORT);
       shufflePort = conf.getInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, ShuffleHandler.DEFAULT_SHUFFLE_PORT);
+      webPort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_WEB_PORT);
     }
 
     llapDaemon = new LlapDaemon(conf, numExecutorsPerService, execBytesPerService, llapIoEnabled,
-        ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort);
+        ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort);
     llapDaemon.init(conf);
   }
 


[04/24] hive git commit: HIVE-13379 : HIVE-12851 args do not work (slider-keytab-dir, etc.) (Sergey Shelukhin, reviewed by Siddharth Seth)

Posted by ss...@apache.org.
HIVE-13379 : HIVE-12851 args do not work (slider-keytab-dir, etc.) (Sergey Shelukhin, 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/f41cbea8
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f41cbea8
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f41cbea8

Branch: refs/heads/llap
Commit: f41cbea80dc8ca8765b02c19e6aeba772323895c
Parents: 8225cb6
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Mar 31 11:53:39 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Mar 31 11:53:39 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/cli/LlapOptionsProcessor.java     | 25 +++++++++++++++++---
 1 file changed, 22 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f41cbea8/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java
index b7f019c..cdc919e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java
@@ -39,9 +39,6 @@ public class LlapOptionsProcessor {
   public static final String OPTION_INSTANCES = "instances"; //forward as arg
   public static final String OPTION_NAME = "name"; // forward as arg
   public static final String OPTION_DIRECTORY = "directory"; // work-dir
-  public static final String OPTION_ARGS = "args"; // forward as arg
-  public static final String OPTION_LOGLEVEL = "loglevel"; // forward as arg
-  public static final String OPTION_CHAOS_MONKEY = "chaosmonkey"; // forward as arg
   public static final String OPTION_EXECUTORS = "executors"; // llap-daemon-site
   public static final String OPTION_CACHE = "cache"; // llap-daemon-site
   public static final String OPTION_SIZE = "size"; // forward via config.json
@@ -54,6 +51,16 @@ public class LlapOptionsProcessor {
   public static final String OPTION_LLAP_QUEUE = "queue"; // forward via config.json
   public static final String OPTION_IO_THREADS = "iothreads"; // llap-daemon-site
 
+  // Options for the pythin script that are here because our option parser cannot ignore the unknown ones
+  public static final String OPTION_ARGS = "args"; // forward as arg
+  public static final String OPTION_LOGLEVEL = "loglevel"; // forward as arg
+  public static final String OPTION_CHAOS_MONKEY = "chaosmonkey"; // forward as arg
+  public static final String OPTION_SLIDER_KEYTAB_DIR = "slider-keytab-dir";
+  public static final String OPTION_SLIDER_KEYTAB = "slider-keytab";
+  public static final String OPTION_SLIDER_PRINCIPAL = "slider-principal";
+  public static final String OPTION_SLIDER_DEFAULT_KEYTAB = "slider-default-keytab";
+
+
   public class LlapOptions {
     private final int instances;
     private final String directory;
@@ -171,6 +178,18 @@ public class LlapOptionsProcessor {
     options.addOption(OptionBuilder.hasArg().withArgName(OPTION_CHAOS_MONKEY).withLongOpt(OPTION_CHAOS_MONKEY)
         .withDescription("chaosmonkey interval").create('m'));
 
+    options.addOption(OptionBuilder.hasArg(false).withArgName(OPTION_SLIDER_DEFAULT_KEYTAB).withLongOpt(OPTION_SLIDER_DEFAULT_KEYTAB)
+        .withDescription("try to set default settings for Slider AM keytab; mostly for dev testing").create());
+
+    options.addOption(OptionBuilder.hasArg().withArgName(OPTION_SLIDER_KEYTAB_DIR).withLongOpt(OPTION_SLIDER_KEYTAB_DIR)
+        .withDescription("Slider AM keytab directory on HDFS (where the headless user keytab is stored by Slider keytab installation, e.g. .slider/keytabs/llap)").create());
+
+    options.addOption(OptionBuilder.hasArg().withArgName(OPTION_SLIDER_KEYTAB).withLongOpt(OPTION_SLIDER_KEYTAB)
+        .withDescription("Slider AM keytab file name inside " + OPTION_SLIDER_KEYTAB_DIR).create());
+
+    options.addOption(OptionBuilder.hasArg().withArgName(OPTION_SLIDER_PRINCIPAL).withLongOpt(OPTION_SLIDER_PRINCIPAL)
+        .withDescription("Slider AM principal; should be the user running the cluster, e.g. hive@EXAMPLE.COM").create());
+
     options.addOption(OptionBuilder.hasArg().withArgName(OPTION_EXECUTORS).withLongOpt(OPTION_EXECUTORS)
         .withDescription("executor per instance").create('e'));